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

[01/18] ignite git commit: IGNITE-4461: Hadoop: added automatic resolution of "raw" comparator for Text class.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.0 6090ebdfc -> 77ca2e636


IGNITE-4461: Hadoop: added automatic resolution of "raw" comparator for Text class.


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

Branch: refs/heads/ignite-2.0
Commit: f406887c274550317e1b6fbbe1bb302f53a5eaad
Parents: beb242b
Author: devozerov <vo...@gridgain.com>
Authored: Thu Jan 5 14:48:06 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Jan 5 14:48:35 2017 +0300

----------------------------------------------------------------------
 .../hadoop/impl/v2/HadoopV2TaskContext.java     | 64 ++++++++++++++------
 1 file changed, 46 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f406887c/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
index e9cae1c..d328550 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.hadoop.io.PartiallyRawComparator;
+import org.apache.ignite.hadoop.io.TextPartiallyRawComparator;
 import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit;
@@ -76,6 +77,8 @@ import java.io.File;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 
@@ -99,6 +102,9 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     private static final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fsMap
         = createHadoopLazyConcurrentMap();
 
+    /** Default partial comparator mappings. */
+    private static final Map<String, String> PARTIAL_COMPARATORS = new HashMap<>();
+
     /**
      * This method is called with reflection upon Job finish with class loader of each task.
      * This will clean up all the Fs created for specific task.
@@ -111,24 +117,6 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
         fsMap.close();
     }
 
-    /**
-     * Check for combiner grouping support (available since Hadoop 2.3).
-     */
-    static {
-        boolean ok;
-
-        try {
-            JobContext.class.getDeclaredMethod("getCombinerKeyGroupingComparator");
-
-            ok = true;
-        }
-        catch (NoSuchMethodException ignore) {
-            ok = false;
-        }
-
-        COMBINE_KEY_GROUPING_SUPPORTED = ok;
-    }
-
     /** Flag is set if new context-object code is used for running the mapper. */
     private final boolean useNewMapper;
 
@@ -153,6 +141,23 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     /** Counters for task. */
     private final HadoopCounters cntrs = new HadoopCountersImpl();
 
+    static {
+        boolean ok;
+
+        try {
+            JobContext.class.getDeclaredMethod("getCombinerKeyGroupingComparator");
+
+            ok = true;
+        }
+        catch (NoSuchMethodException ignore) {
+            ok = false;
+        }
+
+        COMBINE_KEY_GROUPING_SUPPORTED = ok;
+
+        PARTIAL_COMPARATORS.put(Text.class.getName(), TextPartiallyRawComparator.class.getName());
+    }
+
     /**
      * @param taskInfo Task info.
      * @param job Job.
@@ -181,6 +186,8 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
             // For map-reduce jobs prefer local writes.
             jobConf.setBooleanIfUnset(PARAM_IGFS_PREFER_LOCAL_WRITES, true);
 
+            initializePartiallyRawComparator(jobConf);
+
             jobCtx = new JobContextImpl(jobConf, new JobID(jobId.globalId().toString(), jobId.localId()));
 
             useNewMapper = jobConf.getUseNewMapper();
@@ -447,6 +454,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public Comparator<Object> groupComparator() {
         Comparator<?> res;
 
@@ -581,4 +589,24 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
             throw new IgniteCheckedException(e);
         }
     }
+
+    /**
+     * Try initializing partially raw comparator for job.
+     *
+     * @param conf Configuration.
+     */
+    private void initializePartiallyRawComparator(JobConf conf) {
+        String clsName = conf.get(HadoopJobProperty.JOB_PARTIALLY_RAW_COMPARATOR.propertyName(), null);
+
+        if (clsName == null) {
+            Class keyCls = conf.getMapOutputKeyClass();
+
+            if (keyCls != null) {
+                clsName = PARTIAL_COMPARATORS.get(keyCls.getName());
+
+                if (clsName != null)
+                    conf.set(HadoopJobProperty.JOB_PARTIALLY_RAW_COMPARATOR.propertyName(), clsName);
+            }
+        }
+    }
 }
\ No newline at end of file


[15/18] ignite git commit: IGNITE-4518 Fixed parallel load of cache. - Fixes #1426.

Posted by vo...@apache.org.
IGNITE-4518 Fixed parallel load of cache. - Fixes #1426.

Signed-off-by: Andrey Novikov <an...@gridgain.com>


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

Branch: refs/heads/ignite-2.0
Commit: 79401b2ebf814eeffa36d41a82d1238d8eccc7e9
Parents: 6045a24
Author: Andrey Novikov <an...@gridgain.com>
Authored: Mon Jan 16 10:33:16 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Mon Jan 16 10:33:16 2017 +0700

----------------------------------------------------------------------
 .../store/jdbc/dialect/BasicJdbcDialect.java    | 31 ++++++++-----
 .../store/jdbc/CacheJdbcPojoStoreTest.java      | 48 +++++++++++++++++++-
 2 files changed, 66 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/79401b2e/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
index 3ab112a..139f3fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
@@ -173,13 +173,15 @@ public class BasicJdbcDialect implements JdbcDialect {
         if (appendLowerBound) {
             sb.a("(");
 
-            for (int cnt = keyCols.size(); cnt > 0; cnt--) {
-                for (int j = 0; j < cnt; j++)
-                    if (j == cnt - 1)
-                        sb.a(cols[j]).a(" > ? ");
+            for (int keyCnt = keyCols.size(); keyCnt > 0; keyCnt--) {
+                for (int idx = 0; idx < keyCnt; idx++) {
+                    if (idx == keyCnt - 1)
+                        sb.a(cols[idx]).a(" > ? ");
                     else
-                        sb.a(cols[j]).a(" = ? AND ");
-                if (cnt != 1)
+                        sb.a(cols[idx]).a(" = ? AND ");
+                }
+
+                if (keyCnt != 1)
                     sb.a("OR ");
             }
 
@@ -192,13 +194,18 @@ public class BasicJdbcDialect implements JdbcDialect {
         if (appendUpperBound) {
             sb.a("(");
 
-            for (int cnt = keyCols.size(); cnt > 0; cnt--) {
-                for (int j = 0; j < cnt; j++)
-                    if (j == cnt - 1)
-                        sb.a(cols[j]).a(" <= ? ");
+            for (int keyCnt = keyCols.size(); keyCnt > 0; keyCnt--) {
+                for (int idx = 0, lastIdx = keyCnt - 1; idx < keyCnt; idx++) {
+                    sb.a(cols[idx]);
+
+                    // For composite key when not all of the key columns are constrained should use < (strictly less).
+                    if (idx == lastIdx)
+                        sb.a(keyCnt == keyCols.size() ? " <= ? " : " < ? ");
                     else
-                        sb.a(cols[j]).a(" = ? AND ");
-                if (cnt != 1)
+                        sb.a(" = ? AND ");
+                }
+
+                if (keyCnt != 1)
                     sb.a(" OR ");
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/79401b2e/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
index d8f75d3..4a0b1da 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
@@ -216,7 +216,7 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
 
         stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
             "Person_Complex (id integer not null, org_id integer not null, city_id integer not null, " +
-            "name varchar(50), salary integer, PRIMARY KEY(id))");
+            "name varchar(50), salary integer, PRIMARY KEY(id, org_id, city_id))");
 
         conn.commit();
 
@@ -352,6 +352,52 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
     /**
      * @throws Exception If failed.
      */
+    public void testParallelLoad() throws Exception {
+        Connection conn = store.openConnection(false);
+
+        PreparedStatement prnComplexStmt = conn.prepareStatement("INSERT INTO Person_Complex(id, org_id, city_id, name, salary) VALUES (?, ?, ?, ?, ?)");
+
+        for (int i = 0; i < 8; i++) {
+
+            prnComplexStmt.setInt(1, (i >> 2) & 1);
+            prnComplexStmt.setInt(2, (i >> 1) & 1);
+            prnComplexStmt.setInt(3, i % 2);
+
+            prnComplexStmt.setString(4, "name");
+            prnComplexStmt.setInt(5, 1000 + i * 500);
+
+            prnComplexStmt.addBatch();
+        }
+
+        prnComplexStmt.executeBatch();
+
+        U.closeQuiet(prnComplexStmt);
+
+        conn.commit();
+
+        U.closeQuiet(conn);
+
+        final Collection<PersonComplexKey> prnComplexKeys = new ConcurrentLinkedQueue<>();
+
+        IgniteBiInClosure<Object, Object> c = new CI2<Object, Object>() {
+            @Override public void apply(Object k, Object v) {
+                if (k instanceof PersonComplexKey && v instanceof Person)
+                    prnComplexKeys.add((PersonComplexKey)k);
+                else
+                    fail("Unexpected entry [key=" + k + ", value=" + v + "]");
+            }
+        };
+
+        store.setParallelLoadCacheMinimumThreshold(2);
+
+        store.loadCache(c);
+
+        assertEquals(8, prnComplexKeys.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testWriteRetry() throws Exception {
         CacheJdbcPojoStore<Object, Object> store = store();
 


[10/18] ignite git commit: updated 'setCollocated' flag documentation

Posted by vo...@apache.org.
updated 'setCollocated' flag documentation


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

Branch: refs/heads/ignite-2.0
Commit: 1f358db1eb3c9a28dd1a66042539796c1ef5ac45
Parents: ff0caf8
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jan 11 13:43:39 2017 -0800
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jan 11 13:43:39 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/ignite/cache/query/SqlFieldsQuery.java     | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f358db1/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
index 9b17e78..eac8cfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
@@ -164,6 +164,12 @@ public class SqlFieldsQuery extends Query<List<?>> {
     /**
      * Sets flag defining if this query is collocated.
      *
+     * Collocation flag is used for optimization purposes of queries with GROUP BY statements.
+     * Whenever Ignite executes a distributed query, it sends sub-queries to individual cluster members.
+     * If you know in advance that the elements of your query selection are collocated together on the same node and
+     * you group by collocated key (primary or affinity key), then Ignite can make significant performance and network
+     * optimizations by grouping data on remote nodes.
+     *
      * @param collocated Flag value.
      * @return {@code this} For chaining.
      */


[12/18] ignite git commit: ignite-4293 Do not need store deserialized value for BinaryMarshaller

Posted by vo...@apache.org.
ignite-4293 Do not need store deserialized value for BinaryMarshaller


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

Branch: refs/heads/ignite-2.0
Commit: d10946b8b107dee499a5798a4679ab5d14499764
Parents: abc8b90
Author: Alexandr Kuramshin <ak...@gridgain.com>
Authored: Fri Jan 13 12:26:39 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jan 13 12:26:39 2017 +0300

----------------------------------------------------------------------
 .../processors/cacheobject/IgniteCacheObjectProcessorImpl.java | 5 ++---
 .../java/org/apache/ignite/cache/store/jdbc/model/Person.java  | 2 +-
 .../processors/cache/CacheEntryProcessorCopySelfTest.java      | 6 ++++--
 .../processors/cache/GridCacheBasicStoreAbstractTest.java      | 2 +-
 .../distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java | 5 +----
 5 files changed, 9 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 208ec62..614c612 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -244,9 +244,8 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
         CacheMemoryMode memMode = ccfg.getMemoryMode();
 
-        boolean storeVal = ctx.config().isPeerClassLoadingEnabled() ||
-            GridQueryProcessor.isEnabled(ccfg) ||
-            !ccfg.isCopyOnRead();
+        boolean storeVal = !ccfg.isCopyOnRead() || (!isBinaryEnabled(ccfg) &&
+            (GridQueryProcessor.isEnabled(ccfg) || ctx.config().isPeerClassLoadingEnabled()));
 
         CacheObjectContext res = new CacheObjectContext(ctx,
             ccfg.getName(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
index ddf309b..52ddfc8 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
@@ -196,7 +196,7 @@ public class Person implements Serializable {
     @Override public String toString() {
         return "Person [id=" + id +
             ", orgId=" + orgId +
-            ", birthday=" + birthday.getTime() +
+            ", birthday=" + (birthday == null ? null : birthday.getTime()) +
             ", name=" + name +
             "]";
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
index 21395e6..f44889b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -93,8 +94,9 @@ public class CacheEntryProcessorCopySelfTest extends GridCommonAbstractTest {
             doTest(true, false, OLD_VAL, 1);
 
             // One deserialization due to copyOnRead == true.
-            // Additional deserialization in case p2p enabled due to storeValue == true on update entry.
-            doTest(true, true, NEW_VAL, p2pEnabled ? 2 : 1);
+            // Additional deserialization in case p2p enabled and not BinaryMarshaller due to storeValue == true on update entry.
+            doTest(true, true, NEW_VAL, p2pEnabled &&
+                !(grid.configuration().getMarshaller() instanceof BinaryMarshaller) ? 2 : 1);
 
             // No deserialization.
             doTest(false, false, NEW_VAL, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
index 8ddd737..026b618 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
@@ -571,7 +571,7 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
 
             assert cached != null;
 
-            assert cached == val : "Cached value mismatch [expected=" + val + ", cached=" + cached + ']';
+            assert cached.equals(val) : "Cached value mismatch [expected=" + val + ", cached=" + cached + ']';
 
             // Make sure that value is coming from cache, not from store.
             checkLastMethod(null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java
index 3f3f84f..e8a6cfb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java
@@ -117,10 +117,7 @@ public class GridCacheDhtEvictionsDisabledSelfTest extends GridCommonAbstractTes
             assertNotNull(v1);
             assertNotNull(v2);
 
-            if (affinity(cache).mapKeyToNode(key).isLocal())
-                assertSame(v1, v2);
-            else
-                assertEquals(v1, v2);
+            assertEquals(v1, v2);
         }
     }
 }
\ No newline at end of file


[07/18] ignite git commit: .NET: Fix flaky tests - improve assertions

Posted by vo...@apache.org.
.NET: Fix flaky tests - improve assertions


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

Branch: refs/heads/ignite-2.0
Commit: c4d67d2ab737c398c9158dfeb67dc979f5c052af
Parents: 87fbb27
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jan 10 16:07:36 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 16:07:36 2017 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs | 2 ++
 .../platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs | 6 +++++-
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c4d67d2a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
index 250f974..1239794 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -309,6 +309,8 @@ namespace Apache.Ignite.Core.Tests.Cache
 
                 Ignition.Start(cfg);
             }
+
+            Assert.AreEqual(GridCount(), GetIgnite(0).GetCluster().GetNodes().Count);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4d67d2a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
index 82c6742..8b57c81 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
@@ -56,6 +56,8 @@ namespace Apache.Ignite.Core.Tests
             _grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
             _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
             _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+
+            Assert.AreEqual(3, _grid1.GetCluster().GetNodes().Count);
         }
 
         /// <summary>
@@ -556,7 +558,9 @@ namespace Apache.Ignite.Core.Tests
             Func<IEnumerable<string>, IEnumerable<string>> resultFunc, int expectedRepeat)
         {
             // check if expected message count has been received; Wait returns false if there were none.
-            Assert.IsTrue(ReceivedEvent.Wait(MessageTimeout));
+            Assert.IsTrue(ReceivedEvent.Wait(MessageTimeout),
+                string.Format("expectedMessages: {0}, expectedRepeat: {1}, remaining: {2}",
+                    expectedMessages, expectedRepeat, ReceivedEvent.CurrentCount));
 
             expectedMessages = expectedMessages.SelectMany(x => Enumerable.Repeat(x, expectedRepeat));
 


[09/18] ignite git commit: IGNITE-3886 .NET: Fix build script to use latest build tools

Posted by vo...@apache.org.
IGNITE-3886 .NET: Fix build script to use latest build tools

This fixes C++ compilation issue with retargeted projects


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

Branch: refs/heads/ignite-2.0
Commit: ff0caf810f3bb76f284555dd37f93706c9edf1e7
Parents: 58188e8
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jan 10 17:49:07 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 17:49:07 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/build.ps1 | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ff0caf81/modules/platforms/dotnet/build.ps1
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/build.ps1 b/modules/platforms/dotnet/build.ps1
index be7e638..4b5d937 100644
--- a/modules/platforms/dotnet/build.ps1
+++ b/modules/platforms/dotnet/build.ps1
@@ -127,7 +127,7 @@ else {
 
 # 2) Build .NET
 # Detect MSBuild 4.0+
-for ($i=4; $i -le 20; $i++) {
+for ($i=20; $i -ge 4; $i--) {
     $regKey = "HKLM:\software\Microsoft\MSBuild\ToolsVersions\$i.0"
     if (Test-Path $regKey) { break }
 }


[13/18] ignite git commit: .NET: Improve exception messages for binary misconfiguration and missing assemblies

Posted by vo...@apache.org.
.NET: Improve exception messages for binary misconfiguration and missing assemblies


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

Branch: refs/heads/ignite-2.0
Commit: e304b48b92fe75027442b7f7cfb46b421d58ae49
Parents: d10946b
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Jan 13 16:49:48 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Jan 13 16:49:48 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core/Impl/Binary/BinaryReader.cs  | 16 ++++++++++++----
 1 file changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e304b48b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
index 100091f..d9facc3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -697,10 +697,18 @@ namespace Apache.Ignite.Core.Impl.Binary
                     if (desc.Type == null)
                     {
                         if (desc is BinarySurrogateTypeDescriptor)
-                            throw new BinaryObjectException("Unknown type ID: " + hdr.TypeId);
-
-                        throw new BinaryObjectException("No matching type found for object [typeId=" +
-                                                        desc.TypeId + ", typeName=" + desc.TypeName + ']');
+                        {
+                            throw new BinaryObjectException(string.Format(
+                                "Unknown type ID: {0}. " +
+                                "This usually indicates missing BinaryConfiguration." +
+                                "Make sure that all nodes have the same BinaryConfiguration.", hdr.TypeId));
+                        }
+
+                        throw new BinaryObjectException(string.Format(
+                            "No matching type found for object [typeId={0}, typeName={1}]." +
+                            "This usually indicates that assembly with specified type is not loaded on a node." +
+                            "When using Apache.Ignite.exe, make sure to load assemblies with -assembly parameter.",
+                            desc.TypeId, desc.TypeName));
                     }
 
                     // Preserve old frame.


[14/18] ignite git commit: Added more metrics to Flume IgniteSink.

Posted by vo...@apache.org.
Added more metrics to Flume IgniteSink.


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

Branch: refs/heads/ignite-2.0
Commit: 6045a2469a4d2e23cad918b44e002c7de237abcb
Parents: e304b48
Author: shroman <rs...@yahoo.com>
Authored: Sat Jan 14 21:14:34 2017 +0900
Committer: shroman <rs...@yahoo.com>
Committed: Sat Jan 14 21:14:34 2017 +0900

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/stream/flume/IgniteSink.java  | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6045a246/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java
----------------------------------------------------------------------
diff --git a/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java b/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java
index e6e7e90..083e833 100644
--- a/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java
+++ b/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java
@@ -85,7 +85,7 @@ public class IgniteSink extends AbstractSink implements Configurable {
     }
 
     /**
-     * Starts a grid and initializes na event transformer.
+     * Starts a grid and initializes an event transformer.
      */
     @SuppressWarnings("unchecked")
     @Override synchronized public void start() {
@@ -109,9 +109,13 @@ public class IgniteSink extends AbstractSink implements Configurable {
         catch (Exception e) {
             log.error("Failed to start grid", e);
 
+            sinkCounter.incrementConnectionFailedCount();
+
             throw new FlumeException("Failed to start grid", e);
         }
 
+        sinkCounter.incrementConnectionCreatedCount();
+
         super.start();
     }
 
@@ -122,6 +126,7 @@ public class IgniteSink extends AbstractSink implements Configurable {
         if (ignite != null)
             ignite.close();
 
+        sinkCounter.incrementConnectionClosedCount();
         sinkCounter.stop();
 
         super.stop();


[03/18] ignite git commit: IGNITE-4412 fix NLogLoggerTest thread safety issue

Posted by vo...@apache.org.
IGNITE-4412 fix NLogLoggerTest thread safety issue

This closes #1401


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

Branch: refs/heads/ignite-2.0
Commit: bf118aad8d8251144062d97c476fbe5f817d8018
Parents: d2e6007
Author: Sergey Stronchinskiy <se...@kraftvaerk.com>
Authored: Mon Jan 9 15:36:11 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Jan 9 15:36:11 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.csproj             |  1 +
 .../Log/ConcurrentMemoryTarget.cs               | 73 ++++++++++++++++++++
 .../Log/NLogLoggerTest.cs                       |  5 +-
 3 files changed, 76 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bf118aad/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 5948593..55adfe4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -77,6 +77,7 @@
     <Compile Include="Collections\ReadOnlyCollectionTest.cs" />
     <Compile Include="Collections\ReadOnlyDictionaryTest.cs" />
     <Compile Include="Common\IgniteGuidTest.cs" />
+    <Compile Include="Log\ConcurrentMemoryTarget.cs" />
     <Compile Include="Log\DefaultLoggerTest.cs" />
     <Compile Include="Log\Log4NetLoggerTest.cs" />
     <Compile Include="Log\NLogLoggerTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf118aad/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/ConcurrentMemoryTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/ConcurrentMemoryTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/ConcurrentMemoryTarget.cs
new file mode 100644
index 0000000..66bdbe2
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/ConcurrentMemoryTarget.cs
@@ -0,0 +1,73 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Log
+{
+    using System.Collections.Generic;
+    using System.Linq;
+    using global::NLog;
+    using global::NLog.Targets;
+
+    /// <summary>
+    /// NLog target which supports logging from multiple threads.
+    /// </summary>
+    public class ConcurrentMemoryTarget : TargetWithLayout
+    {
+        /// <summary> Object used for locking. </summary>
+        private readonly object _locker = new object();
+
+        /// <summary> Logs. </summary>
+        private readonly IList<string> _logs;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ConcurrentMemoryTarget" /> class.
+        /// </summary>
+        public ConcurrentMemoryTarget()
+        {
+            _logs = new List<string>();
+            Name = "ConcurrentMemoryTarget";
+        }
+
+        /// <summary>
+        /// Gets the collection of logs gathered in the <see cref="ConcurrentMemoryTarget" />.
+        /// </summary>
+        public IEnumerable<string> Logs
+        {
+            get
+            {
+                lock (_locker)
+                {
+                    return _logs.ToList();
+                }
+            }
+        }
+
+        /// <summary>
+        /// Renders the logging event message and adds it to the internal ArrayList of log messages.
+        /// </summary>
+        /// <param name="logEvent">The logging event.</param>
+        protected override void Write(LogEventInfo logEvent)
+        {
+            lock (_locker)
+            {
+                var msg = Layout.Render(logEvent);
+
+                _logs.Add(msg);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf118aad/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs
index 7806ecd..2743353 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs
@@ -25,7 +25,6 @@ namespace Apache.Ignite.Core.Tests.Log
     using global::NLog;
     using global::NLog.Config;
     using global::NLog.Layouts;
-    using global::NLog.Targets;
     using NUnit.Framework;
     using LogLevel = Apache.Ignite.Core.Log.LogLevel;
 
@@ -35,7 +34,7 @@ namespace Apache.Ignite.Core.Tests.Log
     public class NLogLoggerTest
     {
         /** */
-        private MemoryTarget _logTarget;
+        private ConcurrentMemoryTarget _logTarget;
 
         /// <summary>
         /// Test set up.
@@ -45,7 +44,7 @@ namespace Apache.Ignite.Core.Tests.Log
         {
             var cfg = new LoggingConfiguration();
 
-            _logTarget = new MemoryTarget("mem")
+            _logTarget = new ConcurrentMemoryTarget
             {
                 Layout = new SimpleLayout("${Logger}|${Level}|${Message}|${exception}|${all-event-properties}")
             };


[04/18] ignite git commit: IGNITE-4493: ODBC: Added missing diagnostic records for error cases

Posted by vo...@apache.org.
IGNITE-4493: ODBC: Added missing diagnostic records for error cases

This closes #1396


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

Branch: refs/heads/ignite-2.0
Commit: 9e8e9798dae8b92cfbf4d940856dec35d3e30f54
Parents: bf118aa
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Tue Jan 10 11:10:56 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 11:10:56 2017 +0300

----------------------------------------------------------------------
 .../cpp/odbc-test/src/api_robustness_test.cpp   | 113 +++++++++++-
 .../cpp/odbc/include/ignite/odbc/common_types.h |  29 ++-
 .../cpp/odbc/include/ignite/odbc/statement.h    | 143 +++++++++++----
 modules/platforms/cpp/odbc/src/connection.cpp   |   2 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |  38 ++++
 modules/platforms/cpp/odbc/src/odbc.cpp         | 118 ++++---------
 modules/platforms/cpp/odbc/src/statement.cpp    | 176 ++++++++++++++++---
 7 files changed, 484 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/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 13a5ea6..0b6df93 100644
--- a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
@@ -50,7 +50,7 @@ using ignite::impl::binary::BinaryUtils;
 /**
  * Test setup fixture.
  */
-struct ApiRobustnessTestSuiteFixture 
+struct ApiRobustnessTestSuiteFixture
 {
     void Prepare()
     {
@@ -206,6 +206,33 @@ struct ApiRobustnessTestSuiteFixture
 
         // Operation is not supported. However, there should be no crash.
         BOOST_CHECK(ret == SQL_ERROR);
+
+        CheckSQLStatementDiagnosticError("HY106");
+    }
+
+    void CheckSQLDiagnosticError(int16_t handleType, SQLHANDLE handle, const std::string& expectSqlState)
+    {
+        SQLCHAR state[ODBC_BUFFER_SIZE];
+        SQLINTEGER nativeError = 0;
+        SQLCHAR message[ODBC_BUFFER_SIZE];
+        SQLSMALLINT messageLen = 0;
+
+        SQLRETURN ret = SQLGetDiagRec(handleType, handle, 1, state, &nativeError, message, sizeof(message), &messageLen);
+
+        const std::string sqlState = reinterpret_cast<char*>(state);
+        BOOST_REQUIRE_EQUAL(ret, SQL_SUCCESS);
+        BOOST_REQUIRE_EQUAL(sqlState, expectSqlState);
+        BOOST_REQUIRE(messageLen > 0);
+    }
+
+    void CheckSQLStatementDiagnosticError(const std::string& expectSqlState)
+    {
+        CheckSQLDiagnosticError(SQL_HANDLE_STMT, stmt, expectSqlState);
+    }
+
+    void CheckSQLConnectionDiagnosticError(const std::string& expectSqlState)
+    {
+        CheckSQLDiagnosticError(SQL_HANDLE_DBC, dbc, expectSqlState);
     }
 
     /**
@@ -234,6 +261,43 @@ struct ApiRobustnessTestSuiteFixture
     SQLHSTMT stmt;
 };
 
+SQLSMALLINT unsupportedC[] = {
+        SQL_C_INTERVAL_YEAR,
+        SQL_C_INTERVAL_MONTH,
+        SQL_C_INTERVAL_DAY,
+        SQL_C_INTERVAL_HOUR,
+        SQL_C_INTERVAL_MINUTE,
+        SQL_C_INTERVAL_SECOND,
+        SQL_C_INTERVAL_YEAR_TO_MONTH,
+        SQL_C_INTERVAL_DAY_TO_HOUR,
+        SQL_C_INTERVAL_DAY_TO_MINUTE,
+        SQL_C_INTERVAL_DAY_TO_SECOND,
+        SQL_C_INTERVAL_HOUR_TO_MINUTE,
+        SQL_C_INTERVAL_HOUR_TO_SECOND,
+        SQL_C_INTERVAL_MINUTE_TO_SECOND
+    };
+
+SQLSMALLINT unsupportedSql[] = {
+        SQL_WVARCHAR,
+        SQL_WLONGVARCHAR,
+        SQL_REAL,
+        SQL_NUMERIC,
+        SQL_TYPE_TIME,
+        SQL_INTERVAL_MONTH,
+        SQL_INTERVAL_YEAR,
+        SQL_INTERVAL_YEAR_TO_MONTH,
+        SQL_INTERVAL_DAY,
+        SQL_INTERVAL_HOUR,
+        SQL_INTERVAL_MINUTE,
+        SQL_INTERVAL_SECOND,
+        SQL_INTERVAL_DAY_TO_HOUR,
+        SQL_INTERVAL_DAY_TO_MINUTE,
+        SQL_INTERVAL_DAY_TO_SECOND,
+        SQL_INTERVAL_HOUR_TO_MINUTE,
+        SQL_INTERVAL_HOUR_TO_SECOND,
+        SQL_INTERVAL_MINUTE_TO_SECOND
+    };
+
 BOOST_FIXTURE_TEST_SUITE(ApiRobustnessTestSuite, ApiRobustnessTestSuiteFixture)
 
 BOOST_AUTO_TEST_CASE(TestSQLDriverConnect)
@@ -516,6 +580,19 @@ BOOST_AUTO_TEST_CASE(TestSQLBindCol)
 
     ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt);
 
+    //Unsupported data types
+    for(int i = 0; i < sizeof(unsupportedC)/sizeof(unsupportedC[0]); ++i)
+    {
+        ret = SQLBindCol(stmt, 1, unsupportedC[i], &ind1, sizeof(ind1), &len1);
+        BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+        CheckSQLStatementDiagnosticError("HY003");
+    }
+
+    // Size is negative.
+    ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &ind1, -1, &len1);
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    CheckSQLStatementDiagnosticError("HY090");
+
     // Size is null.
     SQLBindCol(stmt, 1, SQL_C_SLONG, &ind1, 0, &len1);
 
@@ -545,6 +622,24 @@ BOOST_AUTO_TEST_CASE(TestSQLBindParameter)
 
     ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt);
 
+    //Unsupported parameter type : output
+    SQLBindParameter(stmt, 2, SQL_PARAM_OUTPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, sizeof(ind1), &len1);
+    CheckSQLStatementDiagnosticError("HY105");
+
+    //Unsupported parameter type : input/output
+    SQLBindParameter(stmt, 2, SQL_PARAM_INPUT_OUTPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, sizeof(ind1), &len1);
+    CheckSQLStatementDiagnosticError("HY105");
+
+
+    //Unsupported data types
+    for(int i = 0; i < sizeof(unsupportedSql)/sizeof(unsupportedSql[0]); ++i)
+    {
+        ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_SLONG, unsupportedSql[i], 100, 100, &ind1, sizeof(ind1), &len1);
+        BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+        CheckSQLStatementDiagnosticError("HYC00");
+    }
+
+
     // Size is null.
     SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, 0, &len1);
 
@@ -1111,4 +1206,20 @@ BOOST_AUTO_TEST_CASE(TestSQLError)
     SQLError(0, 0, 0, 0, 0, 0, 0, 0);
 }
 
+BOOST_AUTO_TEST_CASE(TestSQLDiagnosticRecords)
+{
+    Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache");
+
+    SQLHANDLE hnd;
+    SQLRETURN ret;
+
+    ret = SQLAllocHandle(SQL_HANDLE_DESC, dbc, &hnd);
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    CheckSQLConnectionDiagnosticError("IM001");
+
+    ret = SQLFreeStmt(stmt, 4);
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    CheckSQLStatementDiagnosticError("HY092");
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
index 517fe4e..5d8901d 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
@@ -113,15 +113,37 @@ namespace ignite
              */
             SQL_STATE_HY001_MEMORY_ALLOCATION,
 
+            /**
+             * The argument TargetType was neither a valid data type
+             * nor SQL_C_DEFAULT
+             */
+            SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE,
+
             /** Invalid use of null pointer. */
             SQL_STATE_HY009_INVALID_USE_OF_NULL_POINTER,
 
             /** Function sequence error. */
             SQL_STATE_HY010_SEQUENCE_ERROR,
 
+            /**
+             * Invalid string or buffer length
+             */
+            SQL_STATE_HY090_INVALID_STRING_OR_BUFFER_LENGTH,
+
+            /**
+             * Option type was out of range.
+             */
+            SQL_STATE_HY092_OPTION_TYPE_OUT_OF_RANGE,
+
             /** Column type out of range. */
             SQL_STATE_HY097_COLUMN_TYPE_OUT_OF_RANGE,
 
+            /** The value specified for the argument InputOutputType was invalid. */
+            SQL_STATE_HY105_INVALID_PARAMETER_TYPE,
+
+            /** The value specified for the argument FetchOrientation was invalid. */
+            SQL_STATE_HY106_FETCH_TYPE_OUT_OF_RANGE,
+
             /**
              * The driver does not support the feature of ODBC behavior that
              * the application requested.
@@ -132,7 +154,12 @@ namespace ignite
              * The connection timeout period expired before the data source
              * responded to the request.
              */
-            SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT
+            SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT,
+
+            /**
+             * Driver does not support this function.
+             */
+            SQL_STATE_IM001_FUNCTION_NOT_SUPPORTED
         };
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
index db56660..596fc66 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
@@ -57,24 +57,15 @@ namespace ignite
             ~Statement();
 
             /**
-             * Bind result column to specified data buffer.
+             * Bind result column to data buffer provided by application
              *
              * @param columnIdx Column index.
-             * @param buffer Buffer to put column data to.
+             * @param targetType Type of target buffer.
+             * @param targetValue Pointer to target buffer.
+             * @param bufferLength Length of target buffer.
+             * @param strLengthOrIndicator Pointer to the length/indicator buffer.
              */
-            void BindColumn(uint16_t columnIdx, const app::ApplicationDataBuffer& buffer);
-
-            /**
-             * Unbind specified column buffer.
-             *
-             * @param columnIdx Column index.
-             */
-            void UnbindColumn(uint16_t columnIdx);
-
-            /**
-             * Unbind all column buffers.
-             */
-            void UnbindAllColumns();
+            void BindColumn(uint16_t columnIdx, int16_t targetType, void* targetValue, SqlLen bufferLength, SqlLen* strLengthOrIndicator);
 
             /**
              * Set column binding offset pointer.
@@ -101,21 +92,17 @@ namespace ignite
              * Bind parameter.
              *
              * @param paramIdx Parameter index.
-             * @param param Parameter.
-             */
-            void BindParameter(uint16_t paramIdx, const app::Parameter& param);
-
-            /**
-             * Unbind specified parameter.
-             *
-             * @param paramIdx Parameter index.
+             * @param ioType Type of the parameter (input/output).
+             * @param bufferType The data type of the parameter.
+             * @param paramSqlType The SQL data type of the parameter.
+             * @param columnSize  The size of the column or expression of the corresponding parameter marker.
+             * @param decDigits  The decimal digits of the column or expression of the corresponding parameter marker.
+             * @param buffer A pointer to a buffer for the parameter's data.
+             * @param bufferLen Length of the ParameterValuePtr buffer in bytes.
+             * @param resLen A pointer to a buffer for the parameter's length.
              */
-            void UnbindParameter(uint16_t paramIdx);
-
-            /**
-             * Unbind all parameters.
-             */
-            void UnbindAllParameters();
+            void BindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType,
+                               SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen);
 
             /**
              * Set statement attribute.
@@ -171,7 +158,7 @@ namespace ignite
              * @param query SQL query.
              */
             void PrepareSqlQuery(const std::string& query);
-            
+
             /**
              * Execute SQL query.
              *
@@ -254,11 +241,24 @@ namespace ignite
             void ExecuteGetTypeInfoQuery(int16_t sqlType);
 
             /**
+             * Free resources
+             * @param option indicates what needs to be freed
+             */
+            void FreeResources(int16_t option);
+
+            /**
              * Close statement.
              */
             void Close();
 
             /**
+             * Fetch query result row with offset
+             * @param orientation Fetch type
+             * @param offset Fetch offset
+             */
+            void FetchScroll(int16_t orientation, int64_t offset);
+
+            /**
              * Fetch query result row.
              */
             void FetchRow();
@@ -362,14 +362,75 @@ namespace ignite
         private:
             IGNITE_NO_COPY_ASSIGNMENT(Statement);
 
+
+            /**
+             * Bind result column to specified data buffer.
+             *
+             * @param columnIdx Column index.
+             * @param buffer Buffer to put column data to.
+             */
+            void SafeBindColumn(uint16_t columnIdx, const app::ApplicationDataBuffer& buffer);
+
+            /**
+             * Unbind specified column buffer.
+             *
+             * @param columnIdx Column index.
+             */
+            void SafeUnbindColumn(uint16_t columnIdx);
+
+            /**
+             * Unbind all column buffers.
+             */
+            void SafeUnbindAllColumns();
+
+            /**
+             * Bind result column to data buffer provided by application
+             *
+             * @param columnIdx Column index.
+             * @param targetType Type of target buffer.
+             * @param targetValue Pointer to target buffer.
+             * @param bufferLength Length of target buffer.
+             * @param strLengthOrIndicator Pointer to the length/indicator buffer.
+             * @return Operation result.
+             */
+            SqlResult InternalBindColumn(uint16_t columnIdx, int16_t targetType, void* targetValue, SqlLen bufferLength, SqlLen* strLengthOrIndicator);
+
             /**
              * Bind parameter.
              *
              * @param paramIdx Parameter index.
              * @param param Parameter.
+             */
+            void SafeBindParameter(uint16_t paramIdx, const app::Parameter& param);
+
+            /**
+             * Unbind specified parameter.
+             *
+             * @param paramIdx Parameter index.
+             */
+            void SafeUnbindParameter(uint16_t paramIdx);
+
+            /**
+             * Unbind all parameters.
+             */
+            void SafeUnbindAllParameters();
+
+            /**
+             * Bind parameter.
+             *
+             * @param paramIdx Parameter index.
+             * @param ioType Type of the parameter (input/output).
+             * @param bufferType The data type of the parameter.
+             * @param paramSqlType The SQL data type of the parameter.
+             * @param columnSize  The size of the column or expression of the corresponding parameter marker.
+             * @param decDigits  The decimal digits of the column or expression of the corresponding parameter marker.
+             * @param buffer A pointer to a buffer for the parameter's data.
+             * @param bufferLen Length of the ParameterValuePtr buffer in bytes.
+             * @param resLen A pointer to a buffer for the parameter's length.
              * @return Operation result.
              */
-            SqlResult InternalBindParameter(uint16_t paramIdx, const app::Parameter& param);
+            SqlResult InternalBindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType,
+                                            SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen);
 
             /**
              * Set statement attribute.
@@ -403,6 +464,14 @@ namespace ignite
              */
             SqlResult InternalGetColumnData(uint16_t columnIdx, app::ApplicationDataBuffer& buffer);
 
+
+            /**
+             * Free resources
+             * @param option indicates what needs to be freed
+             * @return Operation result.
+             */
+            SqlResult InternalFreeResources(int16_t option);
+
             /**
              * Close statement.
              * Internal call.
@@ -418,7 +487,7 @@ namespace ignite
              * @return Operation result.
              */
             SqlResult InternalPrepareSqlQuery(const std::string& query);
-            
+
             /**
              * Execute SQL query.
              *
@@ -435,6 +504,14 @@ namespace ignite
             SqlResult InternalExecuteSqlQuery();
 
             /**
+             * Fetch query result row with offset
+             * @param orientation Fetch type
+             * @param offset Fetch offset
+             * @return Operation result.
+             */
+            SqlResult InternalFetchScroll(int16_t orientation, int64_t offset);
+
+            /**
              * Fetch query result row.
              *
              * @return Operation result.
@@ -621,7 +698,7 @@ namespace ignite
 
             /** Offset added to pointers to change binding of parameters. */
             int* paramBindOffset;
-            
+
             /** Offset added to pointers to change binding of column data. */
             int* columnBindOffset;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/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 2afa42d..4a6de5e 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -226,7 +226,7 @@ namespace ignite
 
             OdbcProtocolHeader hdr;
 
-            int64_t received = ReceiveAll(reinterpret_cast<int8_t*>(&hdr), sizeof(hdr));
+            size_t received = ReceiveAll(reinterpret_cast<int8_t*>(&hdr), sizeof(hdr));
 
             if (received != sizeof(hdr))
                 IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not receive message header");

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/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 215d77f..3c411d2 100644
--- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
+++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
@@ -64,17 +64,35 @@ namespace
     /** SQL state HY001 constant. */
     const std::string STATE_HY001 = "HY001";
 
+    /** SQL state HY003 constant. */
+    const std::string STATE_HY003 = "HY003";
+
     /** SQL state HY009 constant. */
     const std::string STATE_HY009 = "HY009";
 
     /** SQL state HY010 constant. */
     const std::string STATE_HY010 = "HY010";
 
+    /** SQL state HY092 constant. */
+    const std::string STATE_HY092 = "HY092";
+
+    /** SQL state HY105 constant. */
+    const std::string STATE_HY105 = "HY105";
+
+    /** SQL state HY106 constant. */
+    const std::string STATE_HY106 = "HY106";
+
     /** SQL state HYC00 constant. */
     const std::string STATE_HYC00 = "HYC00";
 
     /** SQL state HYT01 constant. */
     const std::string STATE_HYT01 = "HYT01";
+
+    /** SQL state HY090 constant. */
+    const std::string STATE_HY090 = "HY090";
+
+    /** SQL state IM001 constant. */
+    const std::string STATE_IM001 = "IM001";
 }
 
 namespace ignite
@@ -130,6 +148,8 @@ namespace ignite
 
                 if (odbcSubclasses.empty())
                 {
+                    // This is a fixed list taken from ODBC doc.
+                    // Please do not add/remove values here.
                     odbcSubclasses.insert("01S00");
                     odbcSubclasses.insert("01S01");
                     odbcSubclasses.insert("01S02");
@@ -234,18 +254,36 @@ namespace ignite
                     case SQL_STATE_HY001_MEMORY_ALLOCATION:
                         return STATE_HY001;
 
+                    case SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE:
+                        return STATE_HY003;
+
                     case SQL_STATE_HY009_INVALID_USE_OF_NULL_POINTER:
                         return STATE_HY009;
 
                     case SQL_STATE_HY010_SEQUENCE_ERROR:
                         return STATE_HY010;
 
+                    case SQL_STATE_HY090_INVALID_STRING_OR_BUFFER_LENGTH:
+                        return STATE_HY090;
+
+                    case SQL_STATE_HY092_OPTION_TYPE_OUT_OF_RANGE:
+                        return STATE_HY092;
+
+                    case SQL_STATE_HY105_INVALID_PARAMETER_TYPE:
+                        return STATE_HY105;
+
+                    case SQL_STATE_HY106_FETCH_TYPE_OUT_OF_RANGE:
+                        return STATE_HY106;
+
                     case SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED:
                         return STATE_HYC00;
 
                     case SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT:
                         return STATE_HYT01;
 
+                    case SQL_STATE_IM001_FUNCTION_NOT_SUPPORTED:
+                        return STATE_IM001;
+
                     default:
                         break;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/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 dbec55f..fd69c0d 100644
--- a/modules/platforms/cpp/odbc/src/odbc.cpp
+++ b/modules/platforms/cpp/odbc/src/odbc.cpp
@@ -31,6 +31,7 @@
 #include "ignite/odbc/dsn_config.h"
 #include "ignite/odbc.h"
 
+
 namespace ignite
 {
     SQLRETURN SQLGetInfo(SQLHDBC        conn,
@@ -71,6 +72,23 @@ namespace ignite
                 return SQLAllocStmt(parent, result);
 
             case SQL_HANDLE_DESC:
+            {
+                using odbc::Connection;
+                Connection *connection = reinterpret_cast<Connection*>(parent);
+
+                if (!connection)
+                    return SQL_INVALID_HANDLE;
+
+                if (result)
+                    *result = 0;
+
+                connection->GetDiagnosticRecords().Reset();
+                connection->AddStatusRecord(odbc::SQL_STATE_IM001_FUNCTION_NOT_SUPPORTED,
+                                            "The HandleType argument was SQL_HANDLE_DESC, and "
+                                            "the driver does not support allocating a descriptor handle");
+
+                return SQL_ERROR;
+            }
             default:
                 break;
         }
@@ -199,39 +217,15 @@ namespace ignite
         if (!statement)
             return SQL_INVALID_HANDLE;
 
-        switch (option)
+        if (option == SQL_DROP)
         {
-            case SQL_DROP:
-            {
-                delete statement;
-
-                break;
-            }
-
-            case SQL_CLOSE:
-            {
-                return SQLCloseCursor(stmt);
-            }
-
-            case SQL_UNBIND:
-            {
-                statement->UnbindAllColumns();
-
-                break;
-            }
-
-            case SQL_RESET_PARAMS:
-            {
-                statement->UnbindAllParameters();
-
-                break;
-            }
-
-            default:
-                return SQL_ERROR;
+            delete statement;
+            return SQL_SUCCESS;
         }
 
-        return SQL_SUCCESS;
+        statement->FreeResources(option);
+
+        return statement->GetDiagnosticRecords().GetReturnCode();
     }
 
     SQLRETURN SQLCloseCursor(SQLHSTMT stmt)
@@ -419,29 +413,14 @@ namespace ignite
         using odbc::Statement;
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLBindCol called: index=%d, type=%d\n", colNum, targetType);
+        LOG_MSG("SQLBindCol called: index=%d, type=%d targetValue=%p bufferLength=%d\n", colNum, targetType, targetValue, bufferLength);
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
         if (!statement)
             return SQL_INVALID_HANDLE;
 
-        IgniteSqlType driverType = ToDriverType(targetType);
-
-        if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED)
-            return SQL_ERROR;
-
-        if (bufferLength < 0)
-            return SQL_ERROR;
-
-        if (targetValue || strLengthOrIndicator)
-        {
-            ApplicationDataBuffer dataBuffer(driverType, targetValue, bufferLength, strLengthOrIndicator);
-
-            statement->BindColumn(colNum, dataBuffer);
-        }
-        else
-            statement->UnbindColumn(colNum);
+        statement->BindColumn(colNum, targetType, targetValue, bufferLength, strLengthOrIndicator);
 
         return statement->GetDiagnosticRecords().GetReturnCode();
     }
@@ -464,13 +443,19 @@ namespace ignite
 
     SQLRETURN SQLFetchScroll(SQLHSTMT stmt, SQLSMALLINT orientation, SQLLEN offset)
     {
+        using odbc::Statement;
+
         LOG_MSG("SQLFetchScroll called\n");
         LOG_MSG("Orientation: %d, Offset: %d\n", orientation, offset);
 
-        if (orientation != SQL_FETCH_NEXT)
-            return SQL_ERROR;
+        Statement *statement = reinterpret_cast<Statement*>(stmt);
+
+        if (!statement)
+            return SQL_INVALID_HANDLE;
+
+        statement->FetchScroll(orientation, offset);
 
-        return SQLFetch(stmt);
+        return statement->GetDiagnosticRecords().GetReturnCode();
     }
 
     SQLRETURN SQLExtendedFetch(SQLHSTMT         stmt,
@@ -617,12 +602,7 @@ namespace ignite
                                SQLLEN       bufferLen,
                                SQLLEN*      resLen)
     {
-        using namespace odbc::type_traits;
-
         using odbc::Statement;
-        using odbc::app::ApplicationDataBuffer;
-        using odbc::app::Parameter;
-        using odbc::type_traits::IsSqlTypeSupported;
 
         LOG_MSG("SQLBindParameter called: %d, %d, %d\n", paramIdx, bufferType, paramSqlType);
 
@@ -631,27 +611,7 @@ namespace ignite
         if (!statement)
             return SQL_INVALID_HANDLE;
 
-        if (ioType != SQL_PARAM_INPUT)
-            return SQL_ERROR;
-
-        if (!IsSqlTypeSupported(paramSqlType))
-            return SQL_ERROR;
-
-        IgniteSqlType driverType = ToDriverType(bufferType);
-
-        if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED)
-            return SQL_ERROR;
-
-        if (buffer)
-        {
-            ApplicationDataBuffer dataBuffer(driverType, buffer, bufferLen, resLen);
-
-            Parameter param(dataBuffer, paramSqlType, columnSize, decDigits);
-
-            statement->BindParameter(paramIdx, param);
-        }
-        else
-            statement->UnbindParameter(paramIdx);
+        statement->BindParameter(paramIdx, ioType, bufferType, paramSqlType, columnSize, decDigits, buffer, bufferLen, resLen);
 
         return statement->GetDiagnosticRecords().GetReturnCode();
     }
@@ -717,13 +677,13 @@ namespace ignite
     }
 
     SQLRETURN SQLDescribeCol(SQLHSTMT       stmt,
-                             SQLUSMALLINT   columnNum, 
+                             SQLUSMALLINT   columnNum,
                              SQLCHAR*       columnNameBuf,
                              SQLSMALLINT    columnNameBufLen,
                              SQLSMALLINT*   columnNameLen,
-                             SQLSMALLINT*   dataType, 
+                             SQLSMALLINT*   dataType,
                              SQLULEN*       columnSize,
-                             SQLSMALLINT*   decimalDigits, 
+                             SQLSMALLINT*   decimalDigits,
                              SQLSMALLINT*   nullable)
     {
         using odbc::Statement;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc/src/statement.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp
index 02c6dd9..09ad81c 100644
--- a/modules/platforms/cpp/odbc/src/statement.cpp
+++ b/modules/platforms/cpp/odbc/src/statement.cpp
@@ -50,26 +50,57 @@ namespace ignite
             // No-op.
         }
 
-        void Statement::BindColumn(uint16_t columnIdx, const app::ApplicationDataBuffer& buffer)
+        void Statement::BindColumn(uint16_t columnIdx, int16_t targetType, void* targetValue, SqlLen bufferLength, SqlLen* strLengthOrIndicator)
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
+            IGNITE_ODBC_API_CALL(InternalBindColumn(columnIdx, targetType, targetValue, bufferLength, strLengthOrIndicator));
+        }
+
+        SqlResult Statement::InternalBindColumn(uint16_t columnIdx, int16_t targetType, void* targetValue, SqlLen bufferLength, SqlLen* strLengthOrIndicator)
+        {
+            using namespace odbc::type_traits;
+            IgniteSqlType driverType = ToDriverType(targetType);
+
+            if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED)
+            {
+                AddStatusRecord(odbc::SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE, "The argument TargetType was not a valid data type.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            if (bufferLength < 0)
+            {
+                AddStatusRecord(odbc::SQL_STATE_HY090_INVALID_STRING_OR_BUFFER_LENGTH,
+                    "The value specified for the argument BufferLength was less than 0.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            if (targetValue || strLengthOrIndicator)
+            {
+                app::ApplicationDataBuffer dataBuffer(driverType, targetValue, bufferLength, strLengthOrIndicator);
 
+                SafeBindColumn(columnIdx, dataBuffer);
+            }
+            else
+                SafeUnbindColumn(columnIdx);
+
+            return SQL_RESULT_SUCCESS;
+        }
+
+        void Statement::SafeBindColumn(uint16_t columnIdx, const app::ApplicationDataBuffer& buffer)
+        {
             columnBindings[columnIdx] = buffer;
 
             columnBindings[columnIdx].SetPtrToOffsetPtr(&columnBindOffset);
         }
 
-        void Statement::UnbindColumn(uint16_t columnIdx)
+        void Statement::SafeUnbindColumn(uint16_t columnIdx)
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
-
             columnBindings.erase(columnIdx);
         }
 
-        void Statement::UnbindAllColumns()
+        void Statement::SafeUnbindAllColumns()
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
-
             columnBindings.clear();
         }
 
@@ -108,14 +139,21 @@ namespace ignite
             return SQL_RESULT_SUCCESS;
         }
 
-        void Statement::BindParameter(uint16_t paramIdx, const app::Parameter& param)
+        void Statement::BindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType,
+                                      SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen)
         {
-            IGNITE_ODBC_API_CALL(InternalBindParameter(paramIdx, param));
+            IGNITE_ODBC_API_CALL(InternalBindParameter(paramIdx, ioType, bufferType, paramSqlType, columnSize, decDigits, buffer, bufferLen, resLen));
         }
 
-
-        SqlResult Statement::InternalBindParameter(uint16_t paramIdx, const app::Parameter& param)
+        SqlResult Statement::InternalBindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType,
+                                                   SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen)
         {
+            using namespace odbc::type_traits;
+            using odbc::Statement;
+            using odbc::app::ApplicationDataBuffer;
+            using odbc::app::Parameter;
+            using odbc::type_traits::IsSqlTypeSupported;
+
             if (paramIdx == 0)
             {
                 AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE,
@@ -124,24 +162,60 @@ namespace ignite
                 return SQL_RESULT_ERROR;
             }
 
-            paramBindings[paramIdx] = param;
+            if (ioType != SQL_PARAM_INPUT)
+            {
+                AddStatusRecord(SQL_STATE_HY105_INVALID_PARAMETER_TYPE,
+                    "The value specified for the argument InputOutputType was not SQL_PARAM_INPUT.");
 
-            paramBindings[paramIdx].GetBuffer().SetPtrToOffsetPtr(&paramBindOffset);
+                return SQL_RESULT_ERROR;
+            }
+
+            if (!IsSqlTypeSupported(paramSqlType))
+            {
+                AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED,
+                    "Data type is not supported.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            IgniteSqlType driverType = ToDriverType(bufferType);
+
+            if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED)
+            {
+                AddStatusRecord(odbc::SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE,
+                    "The argument TargetType was not a valid data type.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            if (buffer)
+            {
+                ApplicationDataBuffer dataBuffer(driverType, buffer, bufferLen, resLen);
+
+                Parameter param(dataBuffer, paramSqlType, columnSize, decDigits);
+
+                SafeBindParameter(paramIdx, param);
+            }
+            else
+                SafeUnbindParameter(paramIdx);
 
             return SQL_RESULT_SUCCESS;
         }
 
-        void Statement::UnbindParameter(uint16_t paramIdx)
+        void Statement::SafeBindParameter(uint16_t paramIdx, const app::Parameter& param)
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
+            paramBindings[paramIdx] = param;
 
-            paramBindings.erase(paramIdx);
+            paramBindings[paramIdx].GetBuffer().SetPtrToOffsetPtr(&paramBindOffset);
         }
 
-        void Statement::UnbindAllParameters()
+        void Statement::SafeUnbindParameter(uint16_t paramIdx)
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
+            paramBindings.erase(paramIdx);
+        }
 
+        void Statement::SafeUnbindAllParameters()
+        {
             paramBindings.clear();
         }
 
@@ -536,6 +610,50 @@ namespace ignite
             return currentQuery->Execute();
         }
 
+        void Statement::FreeResources(int16_t option)
+        {
+            IGNITE_ODBC_API_CALL(InternalFreeResources(option));
+        }
+
+        SqlResult Statement::InternalFreeResources(int16_t option)
+        {
+            switch (option)
+            {
+                case SQL_DROP:
+                {
+                    AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Deprecated, call SQLFreeHandle instead");
+
+                    return SQL_RESULT_ERROR;
+                }
+
+                case SQL_CLOSE:
+                {
+                    return InternalClose();
+                }
+
+                case SQL_UNBIND:
+                {
+                    SafeUnbindAllColumns();
+
+                    break;
+                }
+
+                case SQL_RESET_PARAMS:
+                {
+                    SafeUnbindAllParameters();
+
+                    break;
+                }
+
+                default:
+                {
+                    AddStatusRecord(SQL_STATE_HY092_OPTION_TYPE_OUT_OF_RANGE, "The value specified for the argument Option was invalid");
+                    return SQL_RESULT_ERROR;
+                }
+            }
+            return SQL_RESULT_SUCCESS;
+        }
+
         void Statement::Close()
         {
             IGNITE_ODBC_API_CALL(InternalClose());
@@ -545,7 +663,7 @@ namespace ignite
         {
             if (!currentQuery.get())
                 return SQL_RESULT_SUCCESS;
-            
+
             SqlResult result = currentQuery->Close();
 
             if (result == SQL_RESULT_SUCCESS)
@@ -554,6 +672,24 @@ namespace ignite
             return result;
         }
 
+        void Statement::FetchScroll(int16_t orientation, int64_t offset)
+        {
+            IGNITE_ODBC_API_CALL(InternalFetchScroll(orientation, offset));
+        }
+
+        SqlResult Statement::InternalFetchScroll(int16_t orientation, int64_t offset)
+        {
+            UNREFERENCED_PARAMETER(offset);
+
+            if (orientation != SQL_FETCH_NEXT)
+            {
+                AddStatusRecord(SQL_STATE_HY106_FETCH_TYPE_OUT_OF_RANGE, "The value specified for the argument FetchOrientation was not SQL_FETCH_NEXT.");
+                return SQL_RESULT_ERROR;
+            }
+
+            return InternalFetchRow();
+        }
+
         void Statement::FetchRow()
         {
             IGNITE_ODBC_API_CALL(InternalFetchRow());


[08/18] ignite git commit: IGNITE-3885 .NET: Add link to Ignite.NET development page in readme.md

Posted by vo...@apache.org.
IGNITE-3885 .NET: Add link to Ignite.NET development page in readme.md


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

Branch: refs/heads/ignite-2.0
Commit: 58188e81b262c26f93b29f3facd3e77f4325e359
Parents: c4d67d2
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jan 10 17:32:57 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 17:32:57 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/README.md | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/58188e81/modules/platforms/dotnet/README.md
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/README.md b/modules/platforms/dotnet/README.md
index 528a219..cbdc7c9 100644
--- a/modules/platforms/dotnet/README.md
+++ b/modules/platforms/dotnet/README.md
@@ -147,4 +147,8 @@ You can find the full Apache Ignite documentation here: [Full documentation][doc
 
 [apache-homepage]: https://ignite.apache.org/
 [getting-started]: https://apacheignite-net.readme.io/docs/getting-started-2
-[docs]: https://apacheignite-net.readme.io/docs
\ No newline at end of file
+[docs]: https://apacheignite-net.readme.io/docs
+
+## Contribute to Ignite.NET
+
+See [Ignite.NET Development](https://cwiki.apache.org/confluence/display/IGNITE/Ignite.NET+Development) on wiki.
\ No newline at end of file


[11/18] ignite git commit: IGNITE-4531: SQL: Use correct property name in BinaryProperty. This closes 1419.

Posted by vo...@apache.org.
IGNITE-4531: SQL: Use correct property name in BinaryProperty. This closes 1419.


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

Branch: refs/heads/ignite-2.0
Commit: abc8b9091e9eda8cc43f398939bf1fb6035f7a3f
Parents: 1f358db
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu Jan 12 10:50:14 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Jan 12 10:50:14 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java      |  2 +-
 ...IgniteCacheAbstractInsertSqlQuerySelfTest.java | 14 +++++++-------
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java   |  6 +++---
 .../cache/IgniteCacheInsertSqlQuerySelfTest.java  | 18 ++++++++++--------
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java   | 14 +++++++-------
 .../cache/IgniteCacheUpdateSqlQuerySelfTest.java  |  4 ++--
 6 files changed, 30 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 58f94f4..3286bac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -2147,7 +2147,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (!(obj instanceof BinaryObjectBuilder))
                 throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
 
-            setValue0((BinaryObjectBuilder) obj, name(), propVal, type());
+            setValue0((BinaryObjectBuilder) obj, propName, propVal, type());
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
index df4259e..86d01c7 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
@@ -151,7 +151,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             LinkedHashMap<String, String> flds = new LinkedHashMap<>();
 
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             s2p.setFields(flds);
 
@@ -172,7 +172,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             LinkedHashMap<String, String> flds = new LinkedHashMap<>();
 
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             i2p.setFields(flds);
 
@@ -194,7 +194,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
 
             flds.put("key", Integer.class.getName());
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             k2p.setFields(flds);
 
@@ -216,7 +216,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
 
             flds.put("Id", Integer.class.getName());
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
             flds.put("IntVal", Integer.class.getName());
 
             k22p.setFields(flds);
@@ -240,7 +240,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             flds.put("key", Integer.class.getName());
             flds.put("strKey", String.class.getName());
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             k32p.setFields(flds);
 
@@ -263,7 +263,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             flds.put("key", Integer.class.getName());
             flds.put("strKey", String.class.getName());
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             k42p.setFields(flds);
 
@@ -515,7 +515,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
         protected int id;
 
         /** */
-        @QuerySqlField
+        @QuerySqlField(name = "firstName")
         protected String name;
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
index 7f79ec4..649012f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
@@ -139,7 +139,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
         BinaryObjectBuilder bldr = ignite(0).binary().builder("Person");
 
         bldr.setField("id", id);
-        bldr.setField("name", name);
+        bldr.setField("firstName", name);
         bldr.setField("secondName", secondName);
 
         return bldr.build();
@@ -186,7 +186,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
         LinkedHashMap<String, String> flds = new LinkedHashMap<>();
 
         flds.put("id", Integer.class.getName());
-        flds.put("name", String.class.getName());
+        flds.put("firstName", String.class.getName());
         flds.put("secondName", String.class.getName());
 
         e.setFields(flds);
@@ -214,7 +214,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
         protected int id;
 
         /** */
-        @QuerySqlField
+        @QuerySqlField(name = "firstName")
         protected final String name;
 
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
index 04a352f..e9c21dc 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
@@ -56,7 +56,7 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
     public void testInsertWithExplicitKey() {
         IgniteCache<String, Person> p = ignite(0).cache("S2P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("insert into Person (_key, id, name) values ('s', ?, ?), " +
+        p.query(new SqlFieldsQuery("insert into Person (_key, id, firstName) values ('s', ?, ?), " +
             "('a', 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get("s"));
@@ -76,7 +76,7 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         assertEquals("Sergi", p.get("s"));
         assertEquals("Alex", p.get("a"));
 
-        p.query(new SqlFieldsQuery("insert into Person(_key, id, name) " +
+        p.query(new SqlFieldsQuery("insert into Person(_key, id, firstName) " +
             "(select substring(lower(_val), 0, 2), cast(length(_val) as int), _val from String)"));
 
         assertEquals(createPerson(5, "Sergi"), p.get("se"));
@@ -91,7 +91,7 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         IgniteCache<Integer, Person> p = ignite(0).cache("I2P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "insert into Person (_key, id, name) values (cast('1' as int), ?, ?), (2, (5 - 3), 'Alex')")
+            "insert into Person (_key, id, firstName) values (cast('1' as int), ?, ?), (2, (5 - 3), 'Alex')")
             .setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(1));
@@ -106,7 +106,7 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         IgniteCache<Key, Person> p = ignite(0).cache("K2P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "insert into Person (key, id, name) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
+            "insert into Person (key, id, firstName) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key(1)));
 
@@ -119,8 +119,8 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
     public void testFieldsCaseSensitivity() {
         IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("insert into \"Person2\" (\"Id\", \"id\", \"name\", \"IntVal\") values (1, ?, ?, 5), " +
-            "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
+        p.query(new SqlFieldsQuery("insert into \"Person2\" (\"Id\", \"id\", \"firstName\", \"IntVal\") " +
+            "values (1, ?, ?, 5),  (2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
 
         assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));
 
@@ -177,7 +177,8 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         IgniteCache<Key3, Person> p = ignite(0).cache("K32P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "insert into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+            "insert into Person (key, strKey, id, firstName) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')")
+            .setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key3(1)));
 
@@ -194,7 +195,8 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         IgniteCache<Key4, Person> p = ignite(0).cache("K42P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "insert into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+            "insert into Person (key, strKey, id, firstName) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')")
+            .setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key4(1)));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
index 0ff3fda..58d07af 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
@@ -32,7 +32,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
     public void testMergeWithExplicitKey() {
         IgniteCache<String, Person> p = ignite(0).cache("S2P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("merge into Person (_key, id, name) values ('s', ?, ?), " +
+        p.query(new SqlFieldsQuery("merge into Person (_key, id, firstName) values ('s', ?, ?), " +
             "('a', 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get("s"));
@@ -52,7 +52,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         assertEquals("Sergi", p.get("s"));
         assertEquals("Alex", p.get("a"));
 
-        p.query(new SqlFieldsQuery("merge into Person(_key, id, name) " +
+        p.query(new SqlFieldsQuery("merge into Person(_key, id, firstName) " +
             "(select substring(lower(_val), 0, 2), cast(length(_val) as int), _val from String)"));
 
         assertEquals(createPerson(5, "Sergi"), p.get("se"));
@@ -67,7 +67,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         IgniteCache<Integer, Person> p = ignite(0).cache("I2P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "merge into Person (_key, id, name) values (cast(? as int), ?, ?), (2, (5 - 3), 'Alex')")
+            "merge into Person (_key, id, firstName) values (cast(? as int), ?, ?), (2, (5 - 3), 'Alex')")
             .setArgs("1", 1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(1));
@@ -82,7 +82,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         IgniteCache<Key, Person> p = ignite(0).cache("K2P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "merge into Person (key, id, name) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
+            "merge into Person (key, id, firstName) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key(1)));
 
@@ -95,7 +95,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
     public void testFieldsCaseSensitivity() {
         IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("merge into \"Person2\" (\"Id\", \"id\", \"name\", \"IntVal\") values (1, ?, ?, 5), " +
+        p.query(new SqlFieldsQuery("merge into \"Person2\" (\"Id\", \"id\", \"firstName\", \"IntVal\") values (1, ?, ?, 5), " +
             "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
 
         assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));
@@ -127,7 +127,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         IgniteCache<Key3, Person> p = ignite(0).cache("K32P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "merge into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+            "merge into Person (key, strKey, id, firstName) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key3(1)));
 
@@ -144,7 +144,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         IgniteCache<Key4, Person> p = ignite(0).cache("K42P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "merge into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+            "merge into Person (key, strKey, id, firstName) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key4(1)));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
index 332a082..58bcaac 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
@@ -61,8 +61,8 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
     public void testUpdateSimple() {
         IgniteCache p = cache();
 
-        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set p.id = p.id * 2, p.name = " +
-            "substring(p.name, 0, 2) where length(p._key) = ? or p.secondName like ?").setArgs(2, "%ite"));
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set p.id = p.id * 2, p.firstName = " +
+            "substring(p.firstName, 0, 2) where length(p._key) = ? or p.secondName like ?").setArgs(2, "%ite"));
 
         c.iterator();
 


[17/18] ignite git commit: IGNITE-4424 REPLICATED cache isn't synced across nodes

Posted by vo...@apache.org.
IGNITE-4424 REPLICATED cache isn't synced across nodes


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

Branch: refs/heads/ignite-2.0
Commit: 82dd912889b0dfca213edb1374c1fa0ed79411fd
Parents: 27ba69b
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Dec 30 13:41:34 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Jan 16 12:18:14 2017 +0300

----------------------------------------------------------------------
 .../GridNearAtomicAbstractUpdateFuture.java     |  34 ++-
 .../GridNearAtomicSingleUpdateFuture.java       |  44 ++--
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  57 ++---
 .../AtomicPutAllChangingTopologyTest.java       | 212 +++++++++++++++++++
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 5 files changed, 284 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/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 2fbabaa..c92e0f5 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
@@ -212,14 +212,18 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
             // Cannot remap.
             remapCnt = 1;
 
-            map(topVer);
+            GridCacheVersion futVer = addAtomicFuture(topVer);
+
+            if (futVer != null)
+                map(topVer, futVer);
         }
     }
 
     /**
      * @param topVer Topology version.
+     * @param futVer Future version
      */
-    protected abstract void map(AffinityTopologyVersion topVer);
+    protected abstract void map(AffinityTopologyVersion topVer, GridCacheVersion futVer);
 
     /**
      * Maps future on ready topology.
@@ -302,7 +306,7 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
      * @param req Request.
      * @param e Error.
      */
-    protected void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteCheckedException e) {
+    protected final void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteCheckedException e) {
         synchronized (mux) {
             GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
                 req.nodeId(),
@@ -314,4 +318,28 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
             onResult(req.nodeId(), res, true);
         }
     }
+
+    /**
+     * Adds future prevents topology change before operation complete.
+     * Should be invoked before topology lock released.
+     *
+     * @param topVer Topology version.
+     * @return Future version in case future added.
+     */
+    protected final GridCacheVersion addAtomicFuture(AffinityTopologyVersion topVer) {
+        GridCacheVersion futVer = cctx.versions().next(topVer);
+
+        synchronized (mux) {
+            assert this.futVer == null : this;
+            assert this.topVer == AffinityTopologyVersion.ZERO : this;
+
+            this.topVer = topVer;
+            this.futVer = futVer;
+        }
+
+        if (storeFuture() && !cctx.mvcc().addAtomicFuture(futVer, this))
+            return null;
+
+        return futVer;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/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 bd231cf..7376aff 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
@@ -348,14 +348,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
                 @Override public void apply(final IgniteInternalFuture<AffinityTopologyVersion> fut) {
                     cctx.kernalContext().closure().runLocalSafe(new Runnable() {
                         @Override public void run() {
-                            try {
-                                AffinityTopologyVersion topVer = fut.get();
-
-                                map(topVer);
-                            }
-                            catch (IgniteCheckedException e) {
-                                onDone(e);
-                            }
+                            mapOnTopology();
                         }
                     });
                 }
@@ -388,7 +381,9 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
     @Override protected void mapOnTopology() {
         cache.topology().readLock();
 
-        AffinityTopologyVersion topVer = null;
+        AffinityTopologyVersion topVer;
+
+        GridCacheVersion futVer;
 
         try {
             if (cache.topology().stopping()) {
@@ -410,6 +405,8 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
                 }
 
                 topVer = fut.topologyVersion();
+
+                futVer = addAtomicFuture(topVer);
             }
             else {
                 if (waitTopFut) {
@@ -435,11 +432,12 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             cache.topology().readUnlock();
         }
 
-        map(topVer);
+        if (futVer != null)
+            map(topVer, futVer);
     }
 
     /** {@inheritDoc} */
-    protected void map(AffinityTopologyVersion topVer) {
+    @Override protected void map(AffinityTopologyVersion topVer, GridCacheVersion futVer) {
         Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);
 
         if (F.isEmpty(topNodes)) {
@@ -449,11 +447,6 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             return;
         }
 
-        Exception err = null;
-        GridNearAtomicAbstractUpdateRequest singleReq0 = null;
-
-        GridCacheVersion futVer = cctx.versions().next(topVer);
-
         GridCacheVersion updVer;
 
         // Assign version on near node in CLOCK ordering mode even if fastMap is false.
@@ -470,16 +463,17 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
         else
             updVer = null;
 
+        Exception err = null;
+        GridNearAtomicAbstractUpdateRequest singleReq0 = null;
+
         try {
             singleReq0 = mapSingleUpdate(topVer, futVer, updVer);
 
             synchronized (mux) {
-                assert this.futVer == null : this;
-                assert this.topVer == AffinityTopologyVersion.ZERO : this;
+                assert this.futVer == futVer || (this.isDone() && this.error() != null);
+                assert this.topVer == topVer;
 
-                this.topVer = topVer;
                 this.updVer = updVer;
-                this.futVer = futVer;
 
                 resCnt = 0;
 
@@ -496,14 +490,6 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             return;
         }
 
-        if (storeFuture()) {
-            if (!cctx.mvcc().addAtomicFuture(futVer, this)) {
-                assert isDone() : this;
-
-                return;
-            }
-        }
-
         // Optimize mapping for single key.
         mapSingle(singleReq0.nodeId(), singleReq0);
     }
@@ -511,7 +497,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
     /**
      * @return Future version.
      */
-    GridCacheVersion onFutureDone() {
+    private GridCacheVersion onFutureDone() {
         GridCacheVersion ver0;
 
         GridFutureAdapter<Void> fut0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/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 cd64117..950e5bd 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
@@ -456,14 +456,7 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
                 @Override public void apply(final IgniteInternalFuture<AffinityTopologyVersion> fut) {
                     cctx.kernalContext().closure().runLocalSafe(new Runnable() {
                         @Override public void run() {
-                            try {
-                                AffinityTopologyVersion topVer = fut.get();
-
-                                map(topVer, remapKeys);
-                            }
-                            catch (IgniteCheckedException e) {
-                                onDone(e);
-                            }
+                            mapOnTopology();
                         }
                     });
                 }
@@ -497,7 +490,9 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     @Override protected void mapOnTopology() {
         cache.topology().readLock();
 
-        AffinityTopologyVersion topVer = null;
+        AffinityTopologyVersion topVer;
+
+        GridCacheVersion futVer;
 
         try {
             if (cache.topology().stopping()) {
@@ -519,6 +514,8 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
                 }
 
                 topVer = fut.topologyVersion();
+
+                futVer = addAtomicFuture(topVer);
             }
             else {
                 if (waitTopFut) {
@@ -544,7 +541,8 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             cache.topology().readUnlock();
         }
 
-        map(topVer, null);
+        if (futVer != null)
+            map(topVer, futVer, remapKeys);
     }
 
     /**
@@ -602,15 +600,18 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     }
 
     /** {@inheritDoc} */
-    protected void map(AffinityTopologyVersion topVer) {
-        map(topVer, null);
+    @Override protected void map(AffinityTopologyVersion topVer, GridCacheVersion futVer) {
+        map(topVer, futVer, null);
     }
 
     /**
      * @param topVer Topology version.
+     * @param futVer Future ID.
      * @param remapKeys Keys to remap.
      */
-    void map(AffinityTopologyVersion topVer, @Nullable Collection<KeyCacheObject> remapKeys) {
+    void map(AffinityTopologyVersion topVer,
+        GridCacheVersion futVer,
+        @Nullable Collection<KeyCacheObject> remapKeys) {
         Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);
 
         if (F.isEmpty(topNodes)) {
@@ -620,14 +621,6 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             return;
         }
 
-        Exception err = null;
-        GridNearAtomicFullUpdateRequest singleReq0 = null;
-        Map<UUID, GridNearAtomicFullUpdateRequest> mappings0 = null;
-
-        int size = keys.size();
-
-        GridCacheVersion futVer = cctx.versions().next(topVer);
-
         GridCacheVersion updVer;
 
         // Assign version on near node in CLOCK ordering mode even if fastMap is false.
@@ -644,6 +637,12 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
         else
             updVer = null;
 
+        Exception err = null;
+        GridNearAtomicFullUpdateRequest singleReq0 = null;
+        Map<UUID, GridNearAtomicFullUpdateRequest> mappings0 = null;
+
+        int size = keys.size();
+
         try {
             if (size == 1 && !fastMap) {
                 assert remapKeys == null || remapKeys.size() == 1;
@@ -676,12 +675,10 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             }
 
             synchronized (mux) {
-                assert this.futVer == null : this;
-                assert this.topVer == AffinityTopologyVersion.ZERO : this;
+                assert this.futVer == futVer || (this.isDone() && this.error() != null);
+                assert this.topVer == topVer;
 
-                this.topVer = topVer;
                 this.updVer = updVer;
-                this.futVer = futVer;
 
                 resCnt = 0;
 
@@ -701,14 +698,6 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             return;
         }
 
-        if (storeFuture()) {
-            if (!cctx.mvcc().addAtomicFuture(futVer, this)) {
-                assert isDone() : this;
-
-                return;
-            }
-        }
-
         // Optimize mapping for single key.
         if (singleReq0 != null)
             mapSingle(singleReq0.nodeId(), singleReq0);
@@ -725,7 +714,7 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     /**
      * @return Future version.
      */
-    GridCacheVersion onFutureDone() {
+    private GridCacheVersion onFutureDone() {
         GridCacheVersion ver0;
 
         GridFutureAdapter<Void> fut0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
new file mode 100644
index 0000000..878cb17
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
@@ -0,0 +1,212 @@
+/*
+ * 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.dht.atomic;
+
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CachePeekMode.BACKUP;
+import static org.apache.ignite.cache.CachePeekMode.PRIMARY;
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/** */
+public class AtomicPutAllChangingTopologyTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES_CNT = 3;
+
+    /** */
+    public static final String CACHE_NAME = "test-cache";
+
+    /** */
+    private static final int CACHE_SIZE = 20_000;
+
+    /** */
+    private static volatile CountDownLatch FILLED_LATCH;
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfig() {
+        return new CacheConfiguration<Integer, Integer>()
+            .setAtomicityMode(ATOMIC)
+            .setCacheMode(REPLICATED)
+            .setAffinity(new FairAffinityFunction(false, 1))
+            .setWriteSynchronizationMode(FULL_SYNC)
+            .setRebalanceMode(SYNC)
+            .setName(CACHE_NAME);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllOnChangingTopology() throws Exception {
+        List<IgniteInternalFuture> futs = new LinkedList<>();
+
+        for (int i = 1; i < NODES_CNT; i++)
+            futs.add(startNodeAsync(i));
+
+        futs.add(startSeedNodeAsync());
+
+        boolean failed = false;
+
+        for (IgniteInternalFuture fut : futs) {
+            try {
+                fut.get();
+            }
+            catch (Throwable th) {
+                log.error("Check failed.", th);
+
+                failed = true;
+            }
+        }
+
+        if (failed)
+            throw new RuntimeException("Test Failed.");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        FILLED_LATCH = new CountDownLatch(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @return Future.
+     * @throws IgniteCheckedException If failed.
+     */
+    private IgniteInternalFuture startSeedNodeAsync() throws IgniteCheckedException {
+        return GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Boolean call() throws Exception {
+                Ignite node = startGrid(0);
+
+                log.info("Creating cache.");
+
+                IgniteCache<Integer, Integer> cache = node.getOrCreateCache(cacheConfig());
+
+                log.info("Created cache.");
+
+                Map<Integer, Integer> data = new HashMap<>(CACHE_SIZE);
+
+                for (int i = 0; i < CACHE_SIZE; i++)
+                    data.put(i, i);
+
+                log.info("Filling.");
+
+                cache.putAll(data);
+
+                log.info("Filled.");
+
+                FILLED_LATCH.countDown();
+
+                checkCacheState(node, cache);
+
+                return true;
+            }
+        });
+    }
+
+    /**
+     * @param nodeId Node index.
+     * @return Future.
+     * @throws IgniteCheckedException If failed.
+     */
+    private IgniteInternalFuture startNodeAsync(final int nodeId) throws IgniteCheckedException {
+        return GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Boolean call() throws Exception {
+                Ignite node = startGrid(nodeId);
+
+                log.info("Getting cache.");
+
+                IgniteCache<Integer, Integer> cache = node.getOrCreateCache(cacheConfig());
+
+                log.info("Got cache.");
+
+                FILLED_LATCH.await();
+
+                log.info("Got Filled.");
+
+                cache.put(1, nodeId);
+
+                checkCacheState(node, cache);
+
+                return true;
+            }
+        });
+    }
+
+    /**
+     * @param node Node.
+     * @param cache Cache.
+     * @throws Exception If failed.
+     */
+    private void checkCacheState(Ignite node, IgniteCache<Integer, Integer> cache) throws Exception {
+        int locSize = cache.localSize(PRIMARY, BACKUP);
+        int locSize2 = -1;
+
+        if (locSize != CACHE_SIZE) {
+            U.sleep(5000);
+
+            // Rechecking.
+            locSize2 = cache.localSize(PRIMARY, BACKUP);
+        }
+
+        assertEquals("Wrong cache size on node [node=" + node.configuration().getGridName() +
+            ", expected= " + CACHE_SIZE +
+            ", actual=" + locSize +
+            ", actual2=" + locSize2 + "]",
+            locSize, CACHE_SIZE);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
index 26cea39..986b8d4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtR
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheTxNodeFailureSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridNearCacheTxNodeFailureSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteAtomicLongChangingTopologySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.AtomicPutAllChangingTopologyTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicClientInvalidPartitionHandlingSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicClientRemoveFailureTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicInvalidPartitionHandlingSelfTest;
@@ -97,6 +98,8 @@ public class IgniteCacheFailoverTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheTxNodeFailureSelfTest.class);
         suite.addTestSuite(GridNearCacheTxNodeFailureSelfTest.class);
 
+        suite.addTestSuite(AtomicPutAllChangingTopologyTest.class);
+
         return suite;
     }
 }


[06/18] ignite git commit: IGNITE-4532 .NET: Fix build warnings

Posted by vo...@apache.org.
IGNITE-4532 .NET: Fix build warnings


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

Branch: refs/heads/ignite-2.0
Commit: 87fbb275eb9d11084485e6b4314b5bf7f8d42a1a
Parents: 825fe9c
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jan 10 16:02:26 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 16:02:26 2017 +0300

----------------------------------------------------------------------
 .../platforms/cpp/common/project/vs/common.vcxproj  |  1 +
 modules/platforms/cpp/jni/project/vs/jni.vcxproj    |  1 +
 .../Apache.Ignite.AspNet.Tests.csproj               |  7 ++++++-
 .../Apache.Ignite.AspNet.csproj                     |  5 +++++
 .../Apache.Ignite.Benchmarks.csproj                 |  5 +++++
 .../Apache.Ignite.Core.Tests.TestDll.csproj         |  5 +++++
 .../Apache.Ignite.Core.Tests.csproj                 |  5 +++++
 .../dotnet/Apache.Ignite.Core.Tests/TestUtils.cs    |  1 -
 .../Apache.Ignite.Core/Apache.Ignite.Core.csproj    | 16 ++++++++--------
 .../Impl/Unmanaged/UnmanagedUtils.cs                |  4 ++--
 .../Apache.Ignite.EntityFramework.Tests.csproj      |  5 +++++
 .../Apache.Ignite.EntityFramework.csproj            |  9 +++++++--
 .../Impl/ArrayDbDataReader.cs                       |  5 +++++
 .../Impl/DataReaderResult.cs                        |  2 ++
 .../Impl/DbCommandInfo.cs                           | 11 +++++++++++
 .../Impl/DbCommandProxy.cs                          |  1 +
 .../Apache.Ignite.Linq/Apache.Ignite.Linq.csproj    |  5 +++++
 .../Apache.Ignite.Log4Net.csproj                    |  5 +++++
 .../Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs    | 16 ++++++++--------
 .../Apache.Ignite.NLog/Apache.Ignite.NLog.csproj    |  5 +++++
 modules/platforms/dotnet/Apache.Ignite.sln          |  3 +++
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj       |  5 +++++
 .../Apache.Ignite.Examples.csproj                   |  3 +++
 .../Apache.Ignite.ExamplesDll.csproj                |  4 ++++
 .../Services/MapService.cs                          |  2 --
 25 files changed, 107 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/cpp/common/project/vs/common.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj b/modules/platforms/cpp/common/project/vs/common.vcxproj
index 99fd551..b4431f8 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj
@@ -97,6 +97,7 @@
       <Optimization>Disabled</Optimization>
       <PreprocessorDefinitions>_SCL_SECURE_NO_WARNINGS;WIN32;_DEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions)</PreprocessorDefinitions>
       <AdditionalIncludeDirectories>$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+      <DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
     </ClCompile>
     <Link>
       <SubSystem>Windows</SubSystem>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/cpp/jni/project/vs/jni.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/project/vs/jni.vcxproj b/modules/platforms/cpp/jni/project/vs/jni.vcxproj
index f080a68..cb03197 100644
--- a/modules/platforms/cpp/jni/project/vs/jni.vcxproj
+++ b/modules/platforms/cpp/jni/project/vs/jni.vcxproj
@@ -97,6 +97,7 @@
       <Optimization>Disabled</Optimization>
       <PreprocessorDefinitions>WIN32;_DEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions)</PreprocessorDefinitions>
       <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+      <DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
     </ClCompile>
     <Link>
       <SubSystem>Windows</SubSystem>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj
index aed74db..628942d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj
@@ -21,7 +21,8 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <RunCodeAnalysis>true</RunCodeAnalysis>
-    <CodeAnalysisRuleSet>AllRules.ruleset</CodeAnalysisRuleSet>
+    <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>pdbonly</DebugType>
@@ -30,6 +31,7 @@
     <DefineConstants>TRACE</DefineConstants>
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -37,6 +39,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.AspNet.Tests.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="nunit.framework">
       <HintPath>..\packages\NUnit.Runners.2.6.3\tools\nunit.framework.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj
index 2e501c1..4508776 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj
@@ -28,6 +28,7 @@
     <RunCodeAnalysis>true</RunCodeAnalysis>
     <ErrorReport>prompt</ErrorReport>
     <CodeAnalysisRuleSet>Apache.Ignite.AspNet.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
@@ -37,6 +38,10 @@
     <ErrorReport>prompt</ErrorReport>
     <CodeAnalysisRuleSet>Apache.Ignite.AspNet.ruleset</CodeAnalysisRuleSet>
     <DocumentationFile>bin\Release\Apache.Ignite.AspNet.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj
index 811047c..65dfc1e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj
@@ -27,12 +27,17 @@
     <DefineConstants>DEBUG</DefineConstants>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
     <Optimize>true</Optimize>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
index b33023a..db7a4d1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
@@ -23,12 +23,17 @@
     <OutputPath>bin\Debug\</OutputPath>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
     <Optimize>true</Optimize>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 55adfe4..78a08d2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -28,6 +28,7 @@
     <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
@@ -35,6 +36,10 @@
     <Optimize>true</Optimize>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="log4net, Version=1.2.15.0, Culture=neutral, PublicKeyToken=669e0ddf0bb1aa2a, processorArchitecture=MSIL">

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index ad27398..8360bf1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-#pragma warning disable S2360 // Optional parameters should not be used
 namespace Apache.Ignite.Core.Tests
 {
     using System;

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index b2db0a7..a80dfc0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -21,6 +21,7 @@
     <DocumentationFile>bin\x64\Debug\Apache.Ignite.Core.XML</DocumentationFile>
     <RunCodeAnalysis>false</RunCodeAnalysis>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x64'">
     <PlatformTarget>x64</PlatformTarget>
@@ -30,6 +31,7 @@
     <Optimize>true</Optimize>
     <DocumentationFile>bin\x64\Release\Apache.Ignite.Core.XML</DocumentationFile>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x86'">
     <PlatformTarget>x86</PlatformTarget>
@@ -39,6 +41,7 @@
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
     <DocumentationFile>bin\x86\Debug\Apache.Ignite.Core.XML</DocumentationFile>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x86'">
     <PlatformTarget>x86</PlatformTarget>
@@ -48,6 +51,7 @@
     <Optimize>true</Optimize>
     <DocumentationFile>bin\x86\Release\Apache.Ignite.Core.XML</DocumentationFile>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -67,6 +71,7 @@
     <PlatformTarget>AnyCPU</PlatformTarget>
     <RunCodeAnalysis>true</RunCodeAnalysis>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
@@ -76,6 +81,7 @@
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
     <DebugType>none</DebugType>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />
@@ -514,10 +520,10 @@
   </ItemGroup>
   <ItemGroup Condition="'$(Platform)' == 'AnyCPU'">
     <EmbeddedResource Include="$(ProjectDir)..\..\cpp\jni\project\vs\x64\$(Configuration)\ignite.jni.dll">
-      <Link>resources\$(Configuration)\x64\ignite.jni.dll</Link>
+      <Link>resources\x64\ignite.jni.dll</Link>
     </EmbeddedResource>
     <EmbeddedResource Include="$(ProjectDir)..\..\cpp\jni\project\vs\Win32\$(Configuration)\ignite.jni.dll" Condition="'$(Configuration)' == 'Release' Or Exists('$(ProjectDir)..\..\cpp\jni\project\vs\Win32\$(Configuration)\ignite.jni.dll')">
-      <Link>resources\$(Configuration)\x86\ignite.jni.dll</Link>
+      <Link>resources\x86\ignite.jni.dll</Link>
     </EmbeddedResource>
   </ItemGroup>
   <ItemGroup>
@@ -537,12 +543,6 @@
   <ItemGroup>
     <Folder Include="Impl\Common\JavaObjects\" />
   </ItemGroup>
-  <ItemGroup>
-    <ProjectReference Include="..\..\cpp\jni\project\vs\jni.vcxproj">
-      <Project>{4f7e4917-4612-4b96-9838-025711ade391}</Project>
-      <Name>jni</Name>
-    </ProjectReference>
-  </ItemGroup>
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <PropertyGroup Condition="'$(Platform)' == 'AnyCPU'">
     <PreBuildEvent>cd "$(ProjectDir)"

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
index 0a2a1f0..34baee0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
@@ -38,9 +38,9 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         [SuppressMessage("Microsoft.Design", "CA1065:DoNotRaiseExceptionsInUnexpectedLocations")]
         static UnmanagedUtils()
         {
-            var platfrom = Environment.Is64BitProcess ? "x64" : "x86";
+            var platform = Environment.Is64BitProcess ? "x64" : "x86";
 
-            var resName = string.Format("{0}.{1}", platfrom, IgniteUtils.FileIgniteJniDll);
+            var resName = string.Format("{0}.{1}", platform, IgniteUtils.FileIgniteJniDll);
 
             var path = IgniteUtils.UnpackEmbeddedResource(resName, IgniteUtils.FileIgniteJniDll);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
index 9711087..b03d5f0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
@@ -20,6 +20,7 @@
     <DefineConstants>DEBUG;TRACE</DefineConstants>
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>pdbonly</DebugType>
@@ -28,6 +29,7 @@
     <DefineConstants>TRACE</DefineConstants>
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -35,6 +37,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.EntityFramework.Tests.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="EntityFramework">
       <HintPath>..\packages\EntityFramework.6.1.3\lib\net40\EntityFramework.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
index 7082182..372c2ed 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
@@ -21,9 +21,10 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <RunCodeAnalysis>true</RunCodeAnalysis>
-    <CodeAnalysisRuleSet>AllRules.ruleset</CodeAnalysisRuleSet>
+    <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
     <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
     <DocumentationFile>bin\Debug\Apache.Ignite.EntityFramework.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>pdbonly</DebugType>
@@ -32,7 +33,8 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <DebugType>none</DebugType>
-	<DocumentationFile>bin\Release\Apache.Ignite.EntityFramework.XML</DocumentationFile>
+    <DocumentationFile>bin\Release\Apache.Ignite.EntityFramework.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -40,6 +42,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.EntityFramework.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="EntityFramework, Version=6.0.0.0, Culture=neutral, PublicKeyToken=b77a5c561934e089, processorArchitecture=MSIL">
       <HintPath>..\packages\EntityFramework.6.1.3\lib\net40\EntityFramework.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
index 89523f4..0cf31e5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
@@ -117,6 +117,7 @@ namespace Apache.Ignite.EntityFramework.Impl
         }
 
         /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public override long GetBytes(int ordinal, long dataOffset, byte[] buffer, int bufferOffset, int length)
         {
             Debug.Assert(buffer != null);
@@ -137,6 +138,7 @@ namespace Apache.Ignite.EntityFramework.Impl
         }
 
         /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public override long GetChars(int ordinal, long dataOffset, char[] buffer, int bufferOffset, int length)
         {
             Debug.Assert(buffer != null);
@@ -193,8 +195,11 @@ namespace Apache.Ignite.EntityFramework.Impl
         }
 
         /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public override int GetValues(object[] values)
         {
+            Debug.Assert(values != null);
+
             var row = GetRow();
 
             var size = Math.Min(row.Length, values.Length);

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
index 48f763c..b683184 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.EntityFramework.Impl
     using System.Collections.Generic;
     using System.Data;
     using System.Data.Common;
+    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
 
     /// <summary>
@@ -38,6 +39,7 @@ namespace Apache.Ignite.EntityFramework.Impl
         /// <summary>
         /// Initializes a new instance of the <see cref="DataReaderResult"/> class.
         /// </summary>
+        [SuppressMessage("Microsoft.Usage", "CA2202: Do not call Dispose more than one time on an object")]
         public DataReaderResult(IDataReader reader)
         {
             try

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
index 6b5db05..b80764c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.EntityFramework.Impl
     using System.Data.Entity.Core.Common.CommandTrees;
     using System.Data.Entity.Core.Metadata.Edm;
     using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
 
     /// <summary>
@@ -166,8 +167,18 @@ namespace Apache.Ignite.EntityFramework.Impl
             public static readonly EntitySetComparer Instance = new EntitySetComparer();
 
             /** <inheritdoc /> */
+            [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
             public int Compare(EntitySetBase x, EntitySetBase y)
             {
+                if (x == null && y == null)
+                    return 0;
+
+                if (x == null)
+                    return -1;
+
+                if (y == null)
+                    return 1;
+
                 return string.CompareOrdinal(x.Name, y.Name);
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
index e3353d5..0eb90ae 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
@@ -73,6 +73,7 @@ namespace Apache.Ignite.EntityFramework.Impl
         }
 
         /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Security", "CA2100:Review SQL queries for security vulnerabilities", Justification = "This class is just a proxy.")]
         public override string CommandText
         {
             get { return _command.CommandText; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
index 72e050c..04b4916 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
@@ -21,6 +21,7 @@
     <ErrorReport>prompt</ErrorReport>
     <CodeAnalysisRuleSet>..\Apache.Ignite.Core\Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
     <RunCodeAnalysis>false</RunCodeAnalysis>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
@@ -30,6 +31,7 @@
     <ErrorReport>prompt</ErrorReport>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
     <DocumentationFile>bin\Release\Apache.Ignite.Linq.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -37,6 +39,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.Linq.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="Remotion.Linq, Version=2.0.0.0, Culture=neutral, PublicKeyToken=fee00910d6e5f53b, processorArchitecture=MSIL">
       <HintPath>..\packages\Remotion.Linq.2.0.1\lib\net40\Remotion.Linq.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj
index af5b7e4..142ce73 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj
@@ -22,6 +22,7 @@
     <WarningLevel>4</WarningLevel>
     <RunCodeAnalysis>true</RunCodeAnalysis>
     <CodeAnalysisRuleSet>AllRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>none</DebugType>
@@ -30,6 +31,7 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <DocumentationFile>bin\Release\Apache.Ignite.Log4Net.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -37,6 +39,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.Log4Net.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="log4net, Version=1.2.15.0, Culture=neutral, PublicKeyToken=669e0ddf0bb1aa2a, processorArchitecture=MSIL">
       <HintPath>..\packages\log4net.2.0.5\lib\net40-full\log4net.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs b/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs
index 058176f..1270600 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs
@@ -55,7 +55,7 @@ namespace Apache.Ignite.Log4Net
         /// <summary>
         /// Logs the specified message.
         /// </summary>
-        /// <param name="logLevel">The level.</param>
+        /// <param name="level">The level.</param>
         /// <param name="message">The message.</param>
         /// <param name="args">The arguments to format <paramref name="message" />.
         /// Can be null (formatting will not occur).</param>
@@ -63,10 +63,10 @@ namespace Apache.Ignite.Log4Net
         /// <param name="category">The logging category name.</param>
         /// <param name="nativeErrorInfo">The native error information.</param>
         /// <param name="ex">The exception. Can be null.</param>
-        public void Log(LogLevel logLevel, string message, object[] args, IFormatProvider formatProvider, 
+        public void Log(LogLevel level, string message, object[] args, IFormatProvider formatProvider, 
             string category, string nativeErrorInfo, Exception ex)
         {
-            var level = ConvertLogLevel(logLevel);
+            var logLevel = ConvertLogLevel(level);
 
             var repo = _log.Logger.Repository;
 
@@ -74,7 +74,7 @@ namespace Apache.Ignite.Log4Net
                 ? (object) message 
                 : new SystemStringFormat(formatProvider, message, args);
 
-            var evt = new LoggingEvent(GetType(), repo, category, level, messageObject, ex);
+            var evt = new LoggingEvent(GetType(), repo, category, logLevel, messageObject, ex);
 
             if (nativeErrorInfo != null)
                 evt.Properties["nativeErrorInfo"] = nativeErrorInfo;
@@ -85,15 +85,15 @@ namespace Apache.Ignite.Log4Net
         /// <summary>
         /// Determines whether the specified log level is enabled.
         /// </summary>
-        /// <param name="logLevel">The level.</param>
+        /// <param name="level">The level.</param>
         /// <returns>
         /// Value indicating whether the specified log level is enabled
         /// </returns>
-        public bool IsEnabled(LogLevel logLevel)
+        public bool IsEnabled(LogLevel level)
         {
-            var level = ConvertLogLevel(logLevel);
+            var logLevel = ConvertLogLevel(level);
 
-            return _log.Logger.IsEnabledFor(level);
+            return _log.Logger.IsEnabledFor(logLevel);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj
index 9fc6ffc..520763b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj
@@ -20,6 +20,7 @@
     <DefineConstants>DEBUG;TRACE</DefineConstants>
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>none</DebugType>
@@ -28,6 +29,7 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <DocumentationFile>bin\Release\Apache.Ignite.NLog.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -35,6 +37,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.NLog.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="NLog, Version=4.0.0.0, Culture=neutral, PublicKeyToken=5120e14c03d0593c, processorArchitecture=MSIL">
       <HintPath>..\packages\NLog.4.3.7\lib\net40\NLog.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.sln
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.sln b/modules/platforms/dotnet/Apache.Ignite.sln
index bcaa753..fe83ccd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.sln
+++ b/modules/platforms/dotnet/Apache.Ignite.sln
@@ -2,6 +2,9 @@
 Microsoft Visual Studio Solution File, Format Version 11.00
 # Visual Studio 2010
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Core", "Apache.Ignite.Core\Apache.Ignite.Core.csproj", "{4CD2F726-7E2B-46C4-A5BA-057BB82EECB6}"
+	ProjectSection(ProjectDependencies) = postProject
+		{4F7E4917-4612-4B96-9838-025711ADE391} = {4F7E4917-4612-4B96-9838-025711ADE391}
+	EndProjectSection
 EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Core.Tests", "Apache.Ignite.Core.Tests\Apache.Ignite.Core.Tests.csproj", "{6A62F66C-DA5B-4FBB-8CE7-A95F740FDC7A}"
 EndProject

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj b/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
index e98ddd1..13379f5 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
@@ -23,12 +23,17 @@
     <OutputPath>bin\Debug\</OutputPath>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
     <Optimize>true</Optimize>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
index 7d0b438..3f7e1dc 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
@@ -35,6 +35,9 @@
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="Apache.Ignite.Core">
       <HintPath Condition="Exists('..\..\Apache.Ignite')">..\..\Apache.Ignite\bin\$(Configuration)\Apache.Ignite.Core.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
index 8515bf6..6ea6227 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
@@ -11,6 +11,7 @@
     <AssemblyName>Apache.Ignite.ExamplesDll</AssemblyName>
     <TargetFrameworkVersion>v4.0</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
+    <NoWarn>649</NoWarn>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -32,6 +33,9 @@
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="Apache.Ignite.Core">
       <HintPath Condition="Exists('..\..\Apache.Ignite')">..\..\Apache.Ignite\bin\$(Configuration)\Apache.Ignite.Core.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
index 958d91d..7071cd4 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
@@ -30,9 +30,7 @@ namespace Apache.Ignite.ExamplesDll.Services
     public class MapService<TK, TV> : IService
     {
         /** Injected Ignite instance. */
-#pragma warning disable 649
         [InstanceResource] private readonly IIgnite _ignite;
-#pragma warning restore 649
 
         /** Cache. */
         private ICache<TK, TV> _cache;


[16/18] ignite git commit: IGNITE-4545 Added cache for router hostnames. - Fixes #1428.

Posted by vo...@apache.org.
IGNITE-4545 Added cache for router hostnames. - Fixes #1428.

Signed-off-by: Andrey Novikov <an...@gridgain.com>


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

Branch: refs/heads/ignite-2.0
Commit: 27ba69b55f645a022f2e84cc4db76309b02850e3
Parents: 79401b2
Author: Andrey Novikov <an...@gridgain.com>
Authored: Mon Jan 16 11:22:34 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Mon Jan 16 11:22:34 2017 +0700

----------------------------------------------------------------------
 .../GridClientConnectionManagerAdapter.java     |  7 ++-
 .../impl/connection/GridClientTopology.java     | 53 ++++++++++++++++----
 2 files changed, 49 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/27ba69b5/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
index 12baee0..f714e7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
@@ -85,6 +85,9 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
     /** Class logger. */
     private final Logger log;
 
+    /** All local enabled MACs. */
+    private final Collection<String> macs;
+
     /** NIO server. */
     private GridNioServer srv;
 
@@ -166,6 +169,8 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
         if (marshId == null && cfg.getMarshaller() == null)
             throw new GridClientException("Failed to start client (marshaller is not configured).");
 
+        macs = U.allLocalMACs();
+
         if (cfg.getProtocol() == GridClientProtocol.TCP) {
             try {
                 IgniteLogger gridLog = new JavaLogger(false);
@@ -316,7 +321,7 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
         }
 
         boolean sameHost = node.attributes().isEmpty() ||
-            F.containsAny(U.allLocalMACs(), node.attribute(ATTR_MACS).toString().split(", "));
+            F.containsAny(macs, node.attribute(ATTR_MACS).toString().split(", "));
 
         Collection<InetSocketAddress> srvs = new LinkedHashSet<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/27ba69b5/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java
index effd5b3..97aa586 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java
@@ -21,7 +21,6 @@ import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Set;
@@ -61,12 +60,18 @@ public class GridClientTopology {
     /** Cached last error prevented topology from update. */
     private GridClientException lastError;
 
+    /** Router addresses from configuration.  */
+    private final String routers;
+
     /**
      * Set of router addresses to infer direct connectivity
      * when client is working in router connection mode.
      * {@code null} when client is working in direct connection node.
      */
-    private final Set<String> routerAddrs;
+    private final Set<InetSocketAddress> routerAddrs;
+
+    /** List of all known local MACs */
+    private final Collection<String> macsCache;
 
     /** Protocol. */
     private final GridClientProtocol prot;
@@ -96,8 +101,38 @@ public class GridClientTopology {
         metricsCache = cfg.isEnableMetricsCache();
         attrCache = cfg.isEnableAttributesCache();
         prot = cfg.getProtocol();
-        routerAddrs = (!cfg.getRouters().isEmpty() && cfg.getServers().isEmpty()) ?
-            new HashSet<>(cfg.getRouters()) : null;
+
+        if (!cfg.getRouters().isEmpty() && cfg.getServers().isEmpty()) {
+            routers = cfg.getRouters().toString();
+
+            routerAddrs = U.newHashSet(cfg.getRouters().size());
+
+            for (String router : cfg.getRouters()) {
+                int portIdx = router.lastIndexOf(":");
+
+                if (portIdx > 0) {
+                    String hostName = router.substring(0, portIdx);
+
+                    try {
+                        int port = Integer.parseInt(router.substring(portIdx + 1));
+
+                        InetSocketAddress inetSockAddr = new InetSocketAddress(hostName, port);
+
+                        routerAddrs.add(inetSockAddr);
+                    }
+                    catch (Exception ignore) {
+                        // No-op.
+                    }
+                }
+            }
+        }
+        else {
+            routers = null;
+
+            routerAddrs = Collections.emptySet();
+        }
+
+        macsCache = U.allLocalMACs();
     }
 
     /**
@@ -279,7 +314,7 @@ public class GridClientTopology {
         try {
             if (lastError != null)
                 throw new GridClientDisconnectedException(
-                    "Topology is failed [protocol=" + prot + ", routers=" + routerAddrs + ']', lastError);
+                    "Topology is failed [protocol=" + prot + ", routers=" + routers + ']', lastError);
             else
                 return nodes.get(id);
         }
@@ -376,19 +411,17 @@ public class GridClientTopology {
             (metricsCache && attrCache) || (node.attributes().isEmpty() && node.metrics() == null);
 
         // Try to bypass object copying.
-        if (noAttrsAndMetrics && routerAddrs == null && node.connectable())
+        if (noAttrsAndMetrics && routerAddrs.isEmpty() && node.connectable())
             return node;
 
         // Return a new node instance based on the original one.
         GridClientNodeImpl.Builder nodeBuilder = GridClientNodeImpl.builder(node, !attrCache, !metricsCache);
 
         for (InetSocketAddress addr : node.availableAddresses(prot, true)) {
-            boolean router = routerAddrs == null ||
-                routerAddrs.contains(addr.getHostName() + ":" + addr.getPort()) ||
-                routerAddrs.contains(addr.getAddress().getHostAddress() + ":" + addr.getPort());
+            boolean router = routerAddrs.isEmpty() || routerAddrs.contains(addr);
 
             boolean reachable = noAttrsAndMetrics || !addr.getAddress().isLoopbackAddress() ||
-                F.containsAny(U.allLocalMACs(), node.attribute(ATTR_MACS).toString().split(", "));
+                F.containsAny(macsCache, node.<String>attribute(ATTR_MACS).split(", "));
 
             if (router && reachable) {
                 nodeBuilder.connectable(true);


[18/18] ignite git commit: Merge branch 'master' into ignite-2.0

Posted by vo...@apache.org.
Merge branch 'master' into ignite-2.0


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

Branch: refs/heads/ignite-2.0
Commit: 77ca2e636c73e464f833f227c4894df0785ae9e2
Parents: 6090ebd 82dd912
Author: devozerov <vo...@gridgain.com>
Authored: Mon Jan 16 16:07:49 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Jan 16 16:07:49 2017 +0300

----------------------------------------------------------------------
 .../ignite/cache/query/SqlFieldsQuery.java      |   6 +
 .../store/jdbc/dialect/BasicJdbcDialect.java    |  31 +-
 .../GridClientConnectionManagerAdapter.java     |   7 +-
 .../impl/connection/GridClientTopology.java     |  53 +++-
 .../GridNearAtomicAbstractUpdateFuture.java     |  34 ++-
 .../GridNearAtomicSingleUpdateFuture.java       |  44 +--
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  57 ++--
 .../IgniteCacheObjectProcessorImpl.java         |   5 +-
 .../processors/query/GridQueryProcessor.java    |   2 +-
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  48 +++-
 .../ignite/cache/store/jdbc/model/Person.java   |   2 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  |   6 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |   2 +-
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |   5 +-
 .../AtomicPutAllChangingTopologyTest.java       | 212 ++++++++++++++
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 .../apache/ignite/stream/flume/IgniteSink.java  |   7 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |  64 +++--
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  14 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |   6 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  18 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  14 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |   4 +-
 .../cpp/common/project/vs/common.vcxproj        |   1 +
 .../platforms/cpp/jni/project/vs/jni.vcxproj    |   1 +
 .../cpp/odbc-test/src/api_robustness_test.cpp   | 113 +++++++-
 modules/platforms/cpp/odbc/Makefile.am          |   3 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |   3 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |  29 +-
 .../ignite/odbc/config/connection_info.h        |   2 -
 .../cpp/odbc/include/ignite/odbc/log.h          | 123 ++++++++
 .../odbc/include/ignite/odbc/meta/column_meta.h |   2 -
 .../cpp/odbc/include/ignite/odbc/statement.h    | 143 +++++++---
 .../cpp/odbc/include/ignite/odbc/utility.h      |  19 +-
 .../odbc/os/linux/src/system/socket_client.cpp  |   9 +-
 .../odbc/os/win/src/system/socket_client.cpp    |   9 +-
 .../src/system/ui/dsn_configuration_window.cpp  |  18 +-
 .../cpp/odbc/os/win/src/system_dsn.cpp          |  17 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |   2 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   6 +
 .../cpp/odbc/src/config/connection_info.cpp     |   3 -
 modules/platforms/cpp/odbc/src/connection.cpp   |  21 +-
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |   3 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |  38 +++
 modules/platforms/cpp/odbc/src/entry_points.cpp |  52 ++--
 modules/platforms/cpp/odbc/src/log.cpp          |  83 ++++++
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |   4 -
 modules/platforms/cpp/odbc/src/odbc.cpp         | 283 ++++++++-----------
 .../odbc/src/query/column_metadata_query.cpp    |  13 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |  23 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |  12 +-
 modules/platforms/cpp/odbc/src/statement.cpp    | 187 ++++++++++--
 modules/platforms/cpp/odbc/src/utility.cpp      |  18 +-
 .../Apache.Ignite.AspNet.Tests.csproj           |   7 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Apache.Ignite.AspNet.csproj                 |   5 +
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Apache.Ignite.Benchmarks.csproj             |   5 +
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   5 +
 .../Apache.Ignite.Core.Tests.csproj             |   6 +
 .../Cache/CacheAbstractTest.cs                  |   2 +
 .../Log/ConcurrentMemoryTarget.cs               |  73 +++++
 .../Log/NLogLoggerTest.cs                       |   5 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |   6 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   1 -
 .../Apache.Ignite.Core.csproj                   |  16 +-
 .../Impl/Binary/BinaryReader.cs                 |  16 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   4 +-
 .../Apache.Ignite.EntityFramework.Tests.csproj  |   5 +
 .../Apache.Ignite.EntityFramework.csproj        |   9 +-
 .../Impl/ArrayDbDataReader.cs                   |   5 +
 .../Impl/DataReaderResult.cs                    |   2 +
 .../Impl/DbCommandInfo.cs                       |  11 +
 .../Impl/DbCommandProxy.cs                      |   1 +
 .../Apache.Ignite.Linq.csproj                   |   5 +
 .../Apache.Ignite.Log4Net.csproj                |   5 +
 .../IgniteLog4NetLogger.cs                      |  16 +-
 .../Apache.Ignite.NLog.csproj                   |   5 +
 modules/platforms/dotnet/Apache.Ignite.sln      |   3 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |   5 +
 modules/platforms/dotnet/README.md              |   6 +-
 modules/platforms/dotnet/build.ps1              |   2 +-
 .../Apache.Ignite.Examples.csproj               |   3 +
 .../Apache.Ignite.ExamplesDll.csproj            |   4 +
 .../Services/MapService.cs                      |   2 -
 86 files changed, 1602 insertions(+), 528 deletions(-)
----------------------------------------------------------------------



[05/18] ignite git commit: IGNITE-4470 Added support for log file configuration via environmental variable IGNITE_ODBC_LOG_PATH

Posted by vo...@apache.org.
IGNITE-4470 Added support for log file configuration via environmental variable IGNITE_ODBC_LOG_PATH

This closes #1384


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

Branch: refs/heads/ignite-2.0
Commit: 825fe9cd64ded64068c573d7e3c3901efae6283f
Parents: 9e8e979
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Tue Jan 10 15:57:05 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 15:57:05 2017 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/odbc/Makefile.am          |   3 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |   3 +-
 .../ignite/odbc/config/connection_info.h        |   2 -
 .../cpp/odbc/include/ignite/odbc/log.h          | 123 ++++++++++++++
 .../odbc/include/ignite/odbc/meta/column_meta.h |   2 -
 .../cpp/odbc/include/ignite/odbc/utility.h      |  19 +--
 .../odbc/os/linux/src/system/socket_client.cpp  |   9 +-
 .../odbc/os/win/src/system/socket_client.cpp    |   9 +-
 .../src/system/ui/dsn_configuration_window.cpp  |  18 +-
 .../cpp/odbc/os/win/src/system_dsn.cpp          |  17 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |   2 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   6 +
 .../cpp/odbc/src/config/connection_info.cpp     |   3 -
 modules/platforms/cpp/odbc/src/connection.cpp   |  19 ++-
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |   3 +-
 modules/platforms/cpp/odbc/src/entry_points.cpp |  52 +++---
 modules/platforms/cpp/odbc/src/log.cpp          |  83 +++++++++
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |   4 -
 modules/platforms/cpp/odbc/src/odbc.cpp         | 167 ++++++++++---------
 .../odbc/src/query/column_metadata_query.cpp    |  13 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |  23 ++-
 .../cpp/odbc/src/query/table_metadata_query.cpp |  12 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |  11 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |  18 +-
 24 files changed, 412 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am
index 1eb42d1..de6f75a 100644
--- a/modules/platforms/cpp/odbc/Makefile.am
+++ b/modules/platforms/cpp/odbc/Makefile.am
@@ -79,7 +79,8 @@ libignite_odbc_la_SOURCES = \
     src/column.cpp \
     src/statement.cpp \
     src/type_traits.cpp \
-    src/utility.cpp
+    src/utility.cpp \
+    src/log.cpp
 
 clean-local:
 	$(RM) *.gcno *.gcda

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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 073dcaa..36d8e25 100644
--- a/modules/platforms/cpp/odbc/include/Makefile.am
+++ b/modules/platforms/cpp/odbc/include/Makefile.am
@@ -53,7 +53,8 @@ noinst_HEADERS = \
     ignite/odbc/cursor.h \
     ignite/odbc/common_types.h \
     ignite/odbc/result_page.h \
-    ignite/odbc/type_traits.h
+    ignite/odbc/type_traits.h \
+    ignite/odbc/log.h
 
 uninstall-hook:
 	if [ -d ${includedir}/ignite ]; then find ${includedir}/ignite -type d -empty -delete; fi

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
index e0ed152..d421952 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
@@ -40,7 +40,6 @@ namespace ignite
                 /** Info type. */
                 typedef unsigned short InfoType;
 
-#ifdef ODBC_DEBUG
                 /**
                  * Convert type to string containing its name.
                  * Debug function.
@@ -48,7 +47,6 @@ namespace ignite
                  * @return Null-terminated string containing types name.
                  */
                 static const char* InfoTypeToString(InfoType type);
-#endif
 
                 /**
                  * Constructor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/include/ignite/odbc/log.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/log.h b/modules/platforms/cpp/odbc/include/ignite/odbc/log.h
new file mode 100644
index 0000000..5a6dafc
--- /dev/null
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/log.h
@@ -0,0 +1,123 @@
+/*
+ * 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_LOG
+#define _IGNITE_ODBC_LOG
+
+#include <string>
+#include <sstream>
+#include <fstream>
+
+#include "ignite/common/common.h"
+#include "ignite/common/concurrent.h"
+
+#   define LOG_MSG(param)                                      \
+    if (ignite::odbc::Logger* p = ignite::odbc::Logger::Get()) \
+    {                                                          \
+        ignite::odbc::LogStream lstream(p);                    \
+        lstream << __FUNCTION__ << ": " << param;              \
+    }
+
+namespace ignite
+{
+    namespace odbc
+    {
+        /* Forward declaration */
+        class Logger;
+
+        /**
+         * Helper object providing stream operations for single log line.
+         * Writes resulting string to Logger object upon destruction.
+         */
+        class LogStream: public std::basic_ostream<char>
+        {
+        public:
+            /**
+             * Constructor.
+             * @param parent pointer to Logger.
+             */
+            LogStream(Logger* parent);
+
+            /**
+             * Conversion operator helpful to determine if log is enabled
+             * @return True if logger is enabled
+             */
+            bool operator()();
+
+            /**
+             * Destructor.
+             */
+            virtual ~LogStream();
+
+        private:
+            IGNITE_NO_COPY_ASSIGNMENT(LogStream);
+
+            /** String buffer. */
+            std::basic_stringbuf<char> strbuf;
+
+            /** Parent logger object */
+            Logger* logger;
+        };
+
+        /**
+         * Logging facility.
+         */
+        class Logger
+        {
+        public:
+            /**
+             * Get instance of Logger, if enabled.
+             * @return Logger instance if logging is enabled. Null otherwise.
+             */
+            static Logger* Get();
+
+            /**
+             * Checks if logging is enabled.
+             * @return True, if logging is enabled.
+             */
+            bool IsEnabled() const;
+
+            /**
+             * Outputs the message to log file
+             * @param message The message to write
+             */
+            void WriteMessage(std::string const& message);
+
+        private:
+            /**
+             * Constructor.
+             * @param path to log file.
+             */
+            Logger(const char* path);
+
+            /**
+             * Destructor.
+             */
+            ~Logger();
+
+            IGNITE_NO_COPY_ASSIGNMENT(Logger);
+
+            /** Mutex for writes synchronization. */
+            ignite::common::concurrent::CriticalSection mutex;
+
+            /** File stream. */
+            std::ofstream stream;
+        };
+    }
+}
+
+#endif //_IGNITE_ODBC_LOG

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h b/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h
index 5ffff03..6a94b04 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h
@@ -38,7 +38,6 @@ namespace ignite
             class ColumnMeta
             {
             public:
-#ifdef ODBC_DEBUG
                 /**
                  * Convert attribute ID to string containing its name.
                  * Debug function.
@@ -46,7 +45,6 @@ namespace ignite
                  * @return Null-terminated string containing attribute name.
                  */
                 static const char* AttrIdToString(uint16_t id);
-#endif
 
                 /**
                  * Default constructor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
index 590e97c..7ff5b23 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
@@ -27,6 +27,7 @@
 
 #include <string>
 #include <algorithm>
+#include <sstream>
 
 #include <ignite/common/utils.h>
 #include <ignite/common/decimal.h>
@@ -34,22 +35,6 @@
 #include "ignite/impl/binary/binary_reader_impl.h"
 #include "ignite/impl/binary/binary_writer_impl.h"
 
-
-#ifdef ODBC_DEBUG
-
-extern FILE* log_file;
-void logInit(const char*);
-
-#   define LOG_MSG(fmt, ...)                                        \
-    do {                                                            \
-        logInit(ODBC_LOG_PATH);                                     \
-        fprintf(log_file, "%s: " fmt, __FUNCTION__, ##__VA_ARGS__);   \
-        fflush(log_file);                                           \
-    } while (false)
-#else
-#   define LOG_MSG(...)
-#endif
-
 namespace ignite
 {
     namespace utility
@@ -187,7 +172,7 @@ namespace ignite
          * @param count data length
          * @return standard string containing the formated hex dump
          */
-        std::string HexDump(const char* data, size_t count);
+        std::string HexDump(const void* data, size_t count);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp b/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp
index a355625..9bdf1d7 100644
--- a/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp
+++ b/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp
@@ -27,6 +27,7 @@
 
 #include "ignite/odbc/system/socket_client.h"
 #include "ignite/odbc/utility.h"
+#include "ignite/odbc/log.h"
 
 #define SOCKET_ERROR (-1)
 
@@ -49,7 +50,7 @@ namespace ignite
 
             bool SocketClient::Connect(const char* hostname, uint16_t port)
             {
-                LOG_MSG("Host: %s, port: %d\n", hostname, port);
+                LOG_MSG("Host: " << hostname << ", port: " << port);
 
                 addrinfo hints;
                 memset(&hints, 0, sizeof(hints));
@@ -70,8 +71,10 @@ namespace ignite
                 // Attempt to connect to an address until one succeeds
                 for (addrinfo *it = result; it != NULL; it = it->ai_next) 
                 {
-                    LOG_MSG("Addr: %u.%u.%u.%u\n", it->ai_addr->sa_data[2], it->ai_addr->sa_data[3],
-                                                   it->ai_addr->sa_data[4], it->ai_addr->sa_data[5]);
+                    LOG_MSG("Addr: " << it->ai_addr->sa_data[2] << "."
+                                     << it->ai_addr->sa_data[3] << "."
+                                     << it->ai_addr->sa_data[4] << "."
+                                     << it->ai_addr->sa_data[5]);
 
                     // Create a SOCKET for connecting to server
                     socketHandle = socket(it->ai_family, it->ai_socktype, it->ai_protocol);

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp b/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp
index e248323..4c440e2 100644
--- a/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp
+++ b/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp
@@ -28,6 +28,7 @@
 
 #include "ignite/odbc/system/socket_client.h"
 #include "ignite/odbc/utility.h"
+#include "ignite/odbc/log.h"
 
 namespace ignite
 {
@@ -64,7 +65,7 @@ namespace ignite
                 addrinfo *result = NULL;
                 addrinfo hints;
 
-                LOG_MSG("Host: %s, port: %d\n", hostname, port);
+                LOG_MSG("Host: " << hostname << " port: " << port);
 
                 memset(&hints, 0, sizeof(hints));
                 hints.ai_family = AF_UNSPEC;
@@ -83,8 +84,10 @@ namespace ignite
                 // Attempt to connect to an address until one succeeds
                 for (addrinfo *it = result; it != NULL; it = it->ai_next)
                 {
-                    LOG_MSG("Addr: %u.%u.%u.%u\n", it->ai_addr->sa_data[2] & 0xFF, it->ai_addr->sa_data[3] & 0xFF,
-                                                   it->ai_addr->sa_data[4] & 0xFF, it->ai_addr->sa_data[5] & 0xFF);
+                    LOG_MSG("Addr: " << (it->ai_addr->sa_data[2] & 0xFF) << "."
+                                     << (it->ai_addr->sa_data[3] & 0xFF) << "."
+                                     << (it->ai_addr->sa_data[4] & 0xFF) << "."
+                                     << (it->ai_addr->sa_data[5] & 0xFF));
 
                     // Create a SOCKET for connecting to server
                     socketHandle = socket(it->ai_family, it->ai_socktype, it->ai_protocol);

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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
index a758bd9..864ca85 100644
--- 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
@@ -17,6 +17,8 @@
 
 #include <Windowsx.h>
 
+#include "ignite/odbc/log.h"
+
 #include "ignite/odbc/system/ui/dsn_configuration_window.h"
 
 namespace ignite
@@ -300,14 +302,14 @@ namespace ignite
                     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("Page size:          %d\n", pageSize);
-                    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");
+                    LOG_MSG("Retriving arguments:");
+                    LOG_MSG("DSN:                " << dsn);
+                    LOG_MSG("Address:            " << address);
+                    LOG_MSG("Cache:              " << cache);
+                    LOG_MSG("Page size:          " << pageSize);
+                    LOG_MSG("Protocol version:   " << version);
+                    LOG_MSG("Distributed Joins:  " << (distributedJoins ? "true" : "false"));
+                    LOG_MSG("Enforce Join Order: " << (enforceJoinOrder ? "true" : "false"));
 
                     if (dsn.empty())
                         throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "DSN name can not be empty.");

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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
index f432a40..3d41e3e 100644
--- a/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp
+++ b/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp
@@ -16,6 +16,7 @@
  */
 
 #include "ignite/odbc/utility.h"
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/system/odbc_constants.h"
 
 #include "ignite/odbc/dsn_config.h"
@@ -147,27 +148,27 @@ BOOL INSTAPI ConfigDSN(HWND hwndParent, WORD req, LPCSTR driver, LPCSTR attribut
 {
     using namespace ignite::odbc;
 
-    LOG_MSG("ConfigDSN called\n");
+    LOG_MSG("ConfigDSN called");
 
     Configuration config;
 
-    LOG_MSG("Attributes: %s\n", attributes);
+    LOG_MSG("Attributes: " << 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("Driver: " << driver);
+    LOG_MSG("Attributes: " << attributes);
 
-    LOG_MSG("DSN: %s\n", config.GetDsn().c_str());
+    LOG_MSG("DSN: " << config.GetDsn());
 
     switch (req)
     {
         case ODBC_ADD_DSN:
         {
-            LOG_MSG("ODBC_ADD_DSN\n");
+            LOG_MSG("ODBC_ADD_DSN");
 
             if (!DisplayConfigureDsnWindow(hwndParent, config))
                 return FALSE;
@@ -180,7 +181,7 @@ BOOL INSTAPI ConfigDSN(HWND hwndParent, WORD req, LPCSTR driver, LPCSTR attribut
 
         case ODBC_CONFIG_DSN:
         {
-            LOG_MSG("ODBC_CONFIG_DSN\n");
+            LOG_MSG("ODBC_CONFIG_DSN");
 
             std::string dsn = config.GetDsn();
 
@@ -202,7 +203,7 @@ BOOL INSTAPI ConfigDSN(HWND hwndParent, WORD req, LPCSTR driver, LPCSTR attribut
 
         case ODBC_REMOVE_DSN:
         {
-            LOG_MSG("ODBC_REMOVE_DSN\n");
+            LOG_MSG("ODBC_REMOVE_DSN");
 
             if (!UnregisterDsn(config.GetDsn().c_str()))
                 return FALSE;

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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 56358c5..b00e432 100644
--- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
+++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
@@ -188,6 +188,7 @@
     <ClCompile Include="..\..\src\statement.cpp" />
     <ClCompile Include="..\..\src\type_traits.cpp" />
     <ClCompile Include="..\..\src\utility.cpp" />
+    <ClCompile Include="..\..\src\log.cpp" />
   </ItemGroup>
   <ItemGroup>
     <None Include="module.def" />
@@ -230,6 +231,7 @@
     <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="..\..\include\ignite\odbc\log.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>

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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 58764e4..1828be1 100644
--- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
+++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
@@ -52,6 +52,9 @@
     <ClCompile Include="..\..\src\utility.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\log.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\cursor.cpp">
       <Filter>Code</Filter>
     </ClCompile>
@@ -176,6 +179,9 @@
     <ClInclude Include="..\..\include\ignite\odbc\utility.h">
       <Filter>Code</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\odbc\log.h">
+      <Filter>Code</Filter>
+    </ClInclude>
     <ClInclude Include="..\..\include\ignite\odbc\query\query.h">
       <Filter>Code\query</Filter>
     </ClInclude>

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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 7c612bc..ed76aab 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -42,8 +42,6 @@ namespace ignite
         namespace config
         {
 
-#ifdef ODBC_DEBUG
-
 #define DBG_STR_CASE(x) case x: return #x
 
             const char * ConnectionInfo::InfoTypeToString(InfoType type)
@@ -102,7 +100,6 @@ namespace ignite
             }
 
 #undef DBG_STR_CASE
-#endif
 
             ConnectionInfo::ConnectionInfo() : strParams(), intParams(),
                 shortParams()

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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 4a6de5e..960a507 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -21,6 +21,7 @@
 
 #include <ignite/common/fixed_size_array.h>
 
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/utility.h"
 #include "ignite/odbc/statement.h"
 #include "ignite/odbc/connection.h"
@@ -65,6 +66,13 @@ namespace ignite
 
         void Connection::GetInfo(config::ConnectionInfo::InfoType type, void* buf, short buflen, short* reslen)
         {
+            LOG_MSG("SQLGetInfo called: "
+                << type << " ("
+                << config::ConnectionInfo::InfoTypeToString(type) << "), "
+                << std::hex << reinterpret_cast<size_t>(buf) << ", "
+                << buflen << ", "
+                << std::hex << reinterpret_cast<size_t>(reslen));
+
             IGNITE_ODBC_API_CALL(InternalGetInfo(type, buf, buflen, reslen));
         }
 
@@ -195,7 +203,7 @@ namespace ignite
             if (sent != len + sizeof(OdbcProtocolHeader))
                 IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message");
 
-            LOG_MSG("message sent: (%d bytes)%s\n", msg.GetSize(), utility::HexDump((char*)msg.GetData(), msg.GetSize()).c_str());
+            LOG_MSG("message sent: (" <<  msg.GetSize() << " bytes)" << utility::HexDump(msg.GetData(), msg.GetSize()));
         }
 
         size_t Connection::SendAll(const int8_t* data, size_t len)
@@ -206,7 +214,7 @@ namespace ignite
             {
                 int res = socket.Send(data + sent, len - sent);
 
-                LOG_MSG("Sent: %d\n", res);
+                LOG_MSG("Sent: " << res);
 
                 if (res <= 0)
                     return sent;
@@ -259,8 +267,7 @@ namespace ignite
                 size_t received = len - remain;
 
                 int res = socket.Receive(buffer + received, remain);
-                LOG_MSG("Receive res: %d\n", res);
-                LOG_MSG("remain: %d\n", remain);
+                LOG_MSG("Receive res: " << res << " remain: " << remain);
 
                 if (res <= 0)
                     return received;
@@ -345,7 +352,7 @@ namespace ignite
 
             if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
             {
-                LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                LOG_MSG("Error: " << rsp.GetError().c_str());
 
                 AddStatusRecord(SQL_STATE_08001_CANNOT_CONNECT, rsp.GetError());
 
@@ -356,7 +363,7 @@ namespace ignite
 
             if (!rsp.IsAccepted())
             {
-                LOG_MSG("Hanshake message has been rejected.\n");
+                LOG_MSG("Hanshake message has been rejected.");
 
                 std::stringstream constructor;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
index 260e3e4..94ea53e 100644
--- a/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
+++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/diagnostic/diagnosable_adapter.h"
 
@@ -41,7 +42,7 @@ namespace ignite
 
             void DiagnosableAdapter::AddStatusRecord(SqlState sqlState, const std::string& message)
             {
-                LOG_MSG("Adding new record: %s\n", message.c_str());
+                LOG_MSG("Adding new record: " << message);
 
                 AddStatusRecord(sqlState, message, 0, 0);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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 850fa10..e241a9f 100644
--- a/modules/platforms/cpp/odbc/src/entry_points.cpp
+++ b/modules/platforms/cpp/odbc/src/entry_points.cpp
@@ -17,6 +17,8 @@
 
 #include "ignite/odbc.h"
 
+#include "ignite/odbc/log.h"
+
 #include "ignite/odbc/utility.h"
 
 SQLRETURN SQL_API SQLGetInfo(SQLHDBC        conn,
@@ -440,7 +442,7 @@ SQLRETURN SQL_API SQLError(SQLHENV      env,
 
 SQLRETURN SQL_API SQLCancel(SQLHSTMT stmt)
 {
-    LOG_MSG("SQLCancel called\n");
+    LOG_MSG("SQLCancel called");
     return SQL_SUCCESS;
 }
 
@@ -452,7 +454,7 @@ SQLRETURN SQL_API SQLColAttributes(SQLHSTMT     stmt,
                                    SQLSMALLINT* strAttrResLen,
                                    SQLLEN*      numAttrBuf)
 {
-    LOG_MSG("SQLColAttributes called\n");
+    LOG_MSG("SQLColAttributes called");
     return SQL_SUCCESS;
 }
 
@@ -461,7 +463,7 @@ SQLRETURN SQL_API SQLGetCursorName(SQLHSTMT     stmt,
                                    SQLSMALLINT  nameBufLen,
                                    SQLSMALLINT* nameResLen)
 {
-    LOG_MSG("SQLGetCursorName called\n");
+    LOG_MSG("SQLGetCursorName called");
     return SQL_SUCCESS;
 }
 
@@ -469,7 +471,7 @@ SQLRETURN SQL_API SQLSetCursorName(SQLHSTMT     stmt,
                                    SQLCHAR*     name,
                                    SQLSMALLINT  nameLen)
 {
-    LOG_MSG("SQLSetCursorName called\n");
+    LOG_MSG("SQLSetCursorName called");
     return SQL_SUCCESS;
 }
 
@@ -477,7 +479,7 @@ SQLRETURN SQL_API SQLGetConnectOption(SQLHDBC       conn,
                                       SQLUSMALLINT  option,
                                       SQLPOINTER    value)
 {
-    LOG_MSG("SQLGetConnectOption called\n");
+    LOG_MSG("SQLGetConnectOption called");
     return SQL_SUCCESS;
 }
 
@@ -485,7 +487,7 @@ SQLRETURN SQL_API SQLGetStmtOption(SQLHSTMT     stmt,
                                    SQLUSMALLINT option,
                                    SQLPOINTER   value)
 {
-    LOG_MSG("SQLGetStmtOption called\n");
+    LOG_MSG("SQLGetStmtOption called");
     return SQL_SUCCESS;
 }
 
@@ -493,7 +495,7 @@ SQLRETURN SQL_API SQLSetConnectOption(SQLHDBC       conn,
                                       SQLUSMALLINT  option,
                                       SQLULEN       value)
 {
-    LOG_MSG("SQLSetConnectOption called\n");
+    LOG_MSG("SQLSetConnectOption called");
     return SQL_SUCCESS;
 }
 
@@ -501,7 +503,7 @@ SQLRETURN SQL_API SQLSetStmtOption(SQLHSTMT     stmt,
                                    SQLUSMALLINT option,
                                    SQLULEN      value)
 {
-    LOG_MSG("SQLSetStmtOption called\n");
+    LOG_MSG("SQLSetStmtOption called");
     return SQL_SUCCESS;
 }
 
@@ -515,7 +517,7 @@ SQLRETURN SQL_API SQLStatistics(SQLHSTMT        stmt,
                                 SQLUSMALLINT    unique,
                                 SQLUSMALLINT    reserved)
 {
-    LOG_MSG("SQLStatistics called\n");
+    LOG_MSG("SQLStatistics called");
     return SQL_SUCCESS;
 }
 
@@ -526,7 +528,7 @@ SQLRETURN SQL_API SQLBrowseConnect(SQLHDBC      conn,
                                    SQLSMALLINT  outConnectionStrBufLen,
                                    SQLSMALLINT* outConnectionStrResLen)
 {
-    LOG_MSG("SQLBrowseConnect called\n");
+    LOG_MSG("SQLBrowseConnect called");
     return SQL_SUCCESS;
 }
 
@@ -540,7 +542,7 @@ SQLRETURN SQL_API SQLProcedureColumns(SQLHSTMT      stmt,
                                       SQLCHAR *     columnName,
                                       SQLSMALLINT   columnNameLen)
 {
-    LOG_MSG("SQLProcedureColumns called\n");
+    LOG_MSG("SQLProcedureColumns called");
     return SQL_SUCCESS;
 }
 
@@ -549,7 +551,7 @@ SQLRETURN SQL_API SQLSetPos(SQLHSTMT        stmt,
                             SQLUSMALLINT    operation,
                             SQLUSMALLINT    lockType)
 {
-    LOG_MSG("SQLSetPos called\n");
+    LOG_MSG("SQLSetPos called");
     return SQL_SUCCESS;
 }
 
@@ -558,7 +560,7 @@ SQLRETURN SQL_API SQLSetScrollOptions(SQLHSTMT      stmt,
                                       SQLLEN        crowKeyset,
                                       SQLUSMALLINT  crowRowset)
 {
-    LOG_MSG("SQLSetScrollOptions called\n");
+    LOG_MSG("SQLSetScrollOptions called");
     return SQL_SUCCESS;
 }
 
@@ -568,7 +570,7 @@ SQLRETURN SQL_API SQLGetConnectAttr(SQLHDBC     conn,
                                     SQLINTEGER  valueBufLen,
                                     SQLINTEGER* valueResLen)
 {
-    LOG_MSG("SQLGetConnectAttr called\n");
+    LOG_MSG("SQLGetConnectAttr called");
     return SQL_SUCCESS;
 }
 
@@ -577,14 +579,14 @@ SQLRETURN SQL_API SQLSetConnectAttr(SQLHDBC     conn,
                                     SQLPOINTER  value,
                                     SQLINTEGER  valueLen)
 {
-    LOG_MSG("SQLSetConnectAttr called\n");
+    LOG_MSG("SQLSetConnectAttr called");
     return SQL_SUCCESS;
 }
 
 SQLRETURN SQL_API SQLBulkOperations(SQLHSTMT       stmt,
                                     SQLUSMALLINT   operation)
 {
-    LOG_MSG("SQLBulkOperations called\n");
+    LOG_MSG("SQLBulkOperations called");
     return SQL_SUCCESS;
 }
 
@@ -596,13 +598,13 @@ SQLRETURN SQL_API SQLTablePrivileges(SQLHSTMT      stmt,
                                      SQLCHAR*      tableName,
                                      SQLSMALLINT   tableNameLen)
 {
-    LOG_MSG("SQLTablePrivileges called\n");
+    LOG_MSG("SQLTablePrivileges called");
     return SQL_SUCCESS;
 }
 
 SQLRETURN SQL_API SQLCopyDesc(SQLHDESC src, SQLHDESC dst)
 {
-    LOG_MSG("SQLCopyDesc called\n");
+    LOG_MSG("SQLCopyDesc called");
     return SQL_SUCCESS;
 }
 
@@ -613,7 +615,7 @@ SQLRETURN SQL_API SQLGetDescField(SQLHDESC      descr,
                                   SQLINTEGER    bufferLen,
                                   SQLINTEGER*   resLen)
 {
-    LOG_MSG("SQLGetDescField called\n");
+    LOG_MSG("SQLGetDescField called");
     return SQL_SUCCESS;
 }
 
@@ -629,7 +631,7 @@ SQLRETURN SQL_API SQLGetDescRec(SQLHDESC        DescriptorHandle,
                                 SQLSMALLINT*    scale,
                                 SQLSMALLINT*    nullable)
 {
-    LOG_MSG("SQLGetDescRec called\n");
+    LOG_MSG("SQLGetDescRec called");
     return SQL_SUCCESS;
 }
 
@@ -639,7 +641,7 @@ SQLRETURN SQL_API SQLSetDescField(SQLHDESC      descr,
                                   SQLPOINTER    buffer,
                                   SQLINTEGER    bufferLen)
 {
-    LOG_MSG("SQLSetDescField called\n");
+    LOG_MSG("SQLSetDescField called");
     return SQL_SUCCESS;
 }
 
@@ -654,7 +656,7 @@ SQLRETURN SQL_API SQLSetDescRec(SQLHDESC      descr,
                                 SQLLEN*       resLen,
                                 SQLLEN*       id)
 {
-    LOG_MSG("SQLSetDescRec called\n");
+    LOG_MSG("SQLSetDescRec called");
     return SQL_SUCCESS;
 }
 
@@ -668,7 +670,7 @@ SQLRETURN SQL_API SQLColumnPrivileges(SQLHSTMT      stmt,
                                       SQLCHAR*      columnName,
                                       SQLSMALLINT   columnNameLen)
 {
-    LOG_MSG("SQLColumnPrivileges called\n");
+    LOG_MSG("SQLColumnPrivileges called");
     return SQL_SUCCESS;
 }
 
@@ -676,7 +678,7 @@ SQLRETURN SQL_API SQLParamOptions(SQLHSTMT  stmt,
                                   SQLULEN   paramSetSize,
                                   SQLULEN*  paramsProcessed)
 {
-    LOG_MSG("SQLParamOptions called\n");
+    LOG_MSG("SQLParamOptions called");
     return SQL_SUCCESS;
 }
 
@@ -688,6 +690,6 @@ SQLRETURN SQL_API SQLProcedures(SQLHSTMT        stmt,
                                 SQLCHAR*        tableName,
                                 SQLSMALLINT     tableNameLen)
 {
-    LOG_MSG("SQLProcedures called\n");
+    LOG_MSG("SQLProcedures called");
     return SQL_SUCCESS;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/log.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/log.cpp b/modules/platforms/cpp/odbc/src/log.cpp
new file mode 100644
index 0000000..755798f
--- /dev/null
+++ b/modules/platforms/cpp/odbc/src/log.cpp
@@ -0,0 +1,83 @@
+/*
+ * 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 <cstdlib>
+
+#include "ignite/odbc/log.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        LogStream::LogStream(Logger* parent) :
+            std::basic_ostream<char>(0),
+            strbuf(),
+            logger(parent)
+        {
+            init(&strbuf);
+        }
+
+        bool LogStream::operator()()
+        {
+            return logger != 0;
+        }
+
+        LogStream::~LogStream()
+        {
+            if (logger)
+            {
+                logger->WriteMessage(strbuf.str());
+            }
+        }
+
+        Logger::Logger(const char* path) :
+            mutex(),
+            stream()
+        {
+            if (path)
+            {
+                stream.open(path);
+            }
+        }
+
+        Logger::~Logger()
+        {
+        }
+
+        bool Logger::IsEnabled() const
+        {
+            return stream.is_open();
+        }
+
+        void Logger::WriteMessage(std::string const& message)
+        {
+            if (IsEnabled())
+            {
+                ignite::common::concurrent::CsLockGuard guard(mutex);
+                stream << message << std::endl;
+            }
+        }
+
+        Logger* Logger::Get()
+        {
+            const char* envVarName = "IGNITE_ODBC_LOG_PATH";
+            static Logger logger(getenv(envVarName));
+            return logger.IsEnabled() ? &logger : 0;
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/meta/column_meta.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/meta/column_meta.cpp b/modules/platforms/cpp/odbc/src/meta/column_meta.cpp
index d397618..12dbfc1 100644
--- a/modules/platforms/cpp/odbc/src/meta/column_meta.cpp
+++ b/modules/platforms/cpp/odbc/src/meta/column_meta.cpp
@@ -27,8 +27,6 @@ namespace ignite
         namespace meta
         {
 
-#ifdef ODBC_DEBUG
-
 #define DBG_STR_CASE(x) case x: return #x
 
             const char* ColumnMeta::AttrIdToString(uint16_t id)
@@ -70,8 +68,6 @@ namespace ignite
 
 #undef DBG_STR_CASE
 
-#endif
-
             void ColumnMeta::Read(ignite::impl::binary::BinaryReaderImpl& reader)
             {
                 utility::ReadString(reader, schemaName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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 fd69c0d..542e64c 100644
--- a/modules/platforms/cpp/odbc/src/odbc.cpp
+++ b/modules/platforms/cpp/odbc/src/odbc.cpp
@@ -20,6 +20,7 @@
 #include <cstring>
 #include <algorithm>
 
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/utility.h"
 #include "ignite/odbc/system/odbc_constants.h"
 
@@ -43,9 +44,10 @@ namespace ignite
         using odbc::Connection;
         using odbc::config::ConnectionInfo;
 
-        LOG_MSG("SQLGetInfo called: %d (%s), %p, %d, %p\n",
-                infoType, ConnectionInfo::InfoTypeToString(infoType),
-                infoValue, infoValueMax, length);
+        LOG_MSG("SQLGetInfo called: "
+            << infoType << " (" << ConnectionInfo::InfoTypeToString(infoType) << "), "
+            << std::hex << reinterpret_cast<size_t>(infoValue) << ", " << infoValueMax << ", "
+            << std::hex << reinterpret_cast<size_t>(length));
 
         Connection *connection = reinterpret_cast<Connection*>(conn);
 
@@ -59,7 +61,7 @@ namespace ignite
 
     SQLRETURN SQLAllocHandle(SQLSMALLINT type, SQLHANDLE parent, SQLHANDLE* result)
     {
-        //LOG_MSG("SQLAllocHandle called\n");
+        //LOG_MSG("SQLAllocHandle called");
         switch (type)
         {
             case SQL_HANDLE_ENV:
@@ -101,7 +103,7 @@ namespace ignite
     {
         using odbc::Environment;
 
-        LOG_MSG("SQLAllocEnv called\n");
+        LOG_MSG("SQLAllocEnv called");
 
         *env = reinterpret_cast<SQLHENV>(new Environment());
 
@@ -113,7 +115,7 @@ namespace ignite
         using odbc::Environment;
         using odbc::Connection;
 
-        LOG_MSG("SQLAllocConnect called\n");
+        LOG_MSG("SQLAllocConnect called");
 
         *conn = SQL_NULL_HDBC;
 
@@ -137,7 +139,7 @@ namespace ignite
         using odbc::Connection;
         using odbc::Statement;
 
-        LOG_MSG("SQLAllocStmt called\n");
+        LOG_MSG("SQLAllocStmt called");
 
         *stmt = SQL_NULL_HDBC;
 
@@ -178,7 +180,7 @@ namespace ignite
     {
         using odbc::Environment;
 
-        LOG_MSG("SQLFreeEnv called\n");
+        LOG_MSG("SQLFreeEnv called");
 
         Environment *environment = reinterpret_cast<Environment*>(env);
 
@@ -194,7 +196,7 @@ namespace ignite
     {
         using odbc::Connection;
 
-        LOG_MSG("SQLFreeConnect called\n");
+        LOG_MSG("SQLFreeConnect called");
 
         Connection *connection = reinterpret_cast<Connection*>(conn);
 
@@ -210,7 +212,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLFreeStmt called\n");
+        LOG_MSG("SQLFreeStmt called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -232,7 +234,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLCloseCursor called\n");
+        LOG_MSG("SQLCloseCursor called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -257,8 +259,9 @@ namespace ignite
 
         UNREFERENCED_PARAMETER(windowHandle);
 
-        LOG_MSG("SQLDriverConnect called\n");
-        LOG_MSG("Connection String: [%s]\n", inConnectionString);
+        LOG_MSG("SQLDriverConnect called");
+        if (inConnectionString)
+            LOG_MSG("Connection String: [" << inConnectionString << "]");
 
         Connection *connection = reinterpret_cast<Connection*>(conn);
 
@@ -290,7 +293,8 @@ namespace ignite
         if (outConnectionStringLen)
             *outConnectionStringLen = static_cast<SQLSMALLINT>(reslen);
 
-        LOG_MSG("%s\n", outConnectionString);
+        if (outConnectionString)
+            LOG_MSG(outConnectionString);
 
         return diag.GetReturnCode();
     }
@@ -318,7 +322,7 @@ namespace ignite
 
         std::string dsn = SqlStringToString(serverName, serverNameLen);
 
-        LOG_MSG("DSN: %s\n", dsn.c_str());
+        LOG_MSG("DSN: " << dsn);
 
         odbc::ReadDsnConfiguration(dsn.c_str(), config);
 
@@ -331,7 +335,7 @@ namespace ignite
     {
         using odbc::Connection;
 
-        LOG_MSG("SQLDisconnect called\n");
+        LOG_MSG("SQLDisconnect called");
 
         Connection *connection = reinterpret_cast<Connection*>(conn);
 
@@ -348,7 +352,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLPrepare called\n");
+        LOG_MSG("SQLPrepare called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -357,7 +361,7 @@ namespace ignite
 
         std::string sql = SqlStringToString(query, queryLen);
 
-        LOG_MSG("SQL: %s\n", sql.c_str());
+        LOG_MSG("SQL: " << sql);
 
         statement->PrepareSqlQuery(sql);
 
@@ -368,7 +372,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLExecute called\n");
+        LOG_MSG("SQLExecute called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -385,7 +389,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLExecDirect called\n");
+        LOG_MSG("SQLExecDirect called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -394,7 +398,7 @@ namespace ignite
 
         std::string sql = SqlStringToString(query, queryLen);
 
-        LOG_MSG("SQL: %s\n", sql.c_str());
+        LOG_MSG("SQL: " << sql);
 
         statement->ExecuteSqlQuery(sql);
 
@@ -413,7 +417,10 @@ namespace ignite
         using odbc::Statement;
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLBindCol called: index=%d, type=%d targetValue=%p bufferLength=%d\n", colNum, targetType, targetValue, bufferLength);
+        LOG_MSG("SQLBindCol called: index=" << colNum << ", type=" << targetType << 
+                ", targetValue=" << reinterpret_cast<size_t>(targetValue) << 
+                ", bufferLength=" << bufferLength << 
+                ", lengthInd=" << reinterpret_cast<size_t>(strLengthOrIndicator));
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -429,7 +436,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLFetch called\n");
+        LOG_MSG("SQLFetch called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -445,8 +452,8 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLFetchScroll called\n");
-        LOG_MSG("Orientation: %d, Offset: %d\n", orientation, offset);
+        LOG_MSG("SQLFetchScroll called");
+        LOG_MSG("Orientation: " << orientation << " Offset: " << offset);
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -464,7 +471,7 @@ namespace ignite
                                SQLULEN*         rowCount,
                                SQLUSMALLINT*    rowStatusArray)
     {
-        LOG_MSG("SQLExtendedFetch called\n");
+        LOG_MSG("SQLExtendedFetch called");
 
         SQLRETURN res = SQLFetchScroll(stmt, orientation, offset);
 
@@ -487,7 +494,7 @@ namespace ignite
         using odbc::Statement;
         using odbc::meta::ColumnMetaVector;
 
-        LOG_MSG("SQLNumResultCols called\n");
+        LOG_MSG("SQLNumResultCols called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -499,7 +506,7 @@ namespace ignite
         if (columnNum)
         {
             *columnNum = static_cast<SQLSMALLINT>(res);
-            LOG_MSG("columnNum: %d\n", *columnNum);
+            LOG_MSG("columnNum: " << *columnNum);
         }
 
         return statement->GetDiagnosticRecords().GetReturnCode();
@@ -518,7 +525,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLTables called\n");
+        LOG_MSG("SQLTables called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -530,10 +537,10 @@ namespace ignite
         std::string table = SqlStringToString(tableName, tableNameLen);
         std::string tableTypeStr = SqlStringToString(tableType, tableTypeLen);
 
-        LOG_MSG("catalog: %s\n", catalog.c_str());
-        LOG_MSG("schema: %s\n", schema.c_str());
-        LOG_MSG("table: %s\n", table.c_str());
-        LOG_MSG("tableType: %s\n", tableTypeStr.c_str());
+        LOG_MSG("catalog: " << catalog);
+        LOG_MSG("schema: " << schema);
+        LOG_MSG("table: " << table);
+        LOG_MSG("tableType: " << tableTypeStr);
 
         statement->ExecuteGetTablesMetaQuery(catalog, schema, table, tableTypeStr);
 
@@ -553,7 +560,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLColumns called\n");
+        LOG_MSG("SQLColumns called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -565,10 +572,10 @@ namespace ignite
         std::string table = SqlStringToString(tableName, tableNameLen);
         std::string column = SqlStringToString(columnName, columnNameLen);
 
-        LOG_MSG("catalog: %s\n", catalog.c_str());
-        LOG_MSG("schema: %s\n", schema.c_str());
-        LOG_MSG("table: %s\n", table.c_str());
-        LOG_MSG("column: %s\n", column.c_str());
+        LOG_MSG("catalog: " << catalog);
+        LOG_MSG("schema: " << schema);
+        LOG_MSG("table: " << table);
+        LOG_MSG("column: " << column);
 
         statement->ExecuteGetColumnsMetaQuery(schema, table, column);
 
@@ -579,7 +586,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLMoreResults called\n");
+        LOG_MSG("SQLMoreResults called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -604,7 +611,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLBindParameter called: %d, %d, %d\n", paramIdx, bufferType, paramSqlType);
+        LOG_MSG("SQLBindParameter called: " << paramIdx << ", " << bufferType << ", " << paramSqlType);
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -625,7 +632,7 @@ namespace ignite
     {
         using namespace utility;
 
-        LOG_MSG("SQLNativeSql called\n");
+        LOG_MSG("SQLNativeSql called");
 
         std::string in = SqlStringToString(inQuery, inQueryLen);
 
@@ -650,7 +657,7 @@ namespace ignite
         using odbc::meta::ColumnMetaVector;
         using odbc::meta::ColumnMeta;
 
-        LOG_MSG("SQLColAttribute called: %d (%s)\n", fieldId, ColumnMeta::AttrIdToString(fieldId));
+        LOG_MSG("SQLColAttribute called: " << fieldId << " (" << ColumnMeta::AttrIdToString(fieldId) << ")");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -689,7 +696,7 @@ namespace ignite
         using odbc::Statement;
         using odbc::SqlLen;
 
-        LOG_MSG("SQLDescribeCol called\n");
+        LOG_MSG("SQLDescribeCol called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -709,13 +716,13 @@ namespace ignite
         statement->GetColumnAttribute(columnNum, SQL_DESC_SCALE, 0, 0, 0, &decimalDigitsRes);
         statement->GetColumnAttribute(columnNum, SQL_DESC_NULLABLE, 0, 0, 0, &nullableRes);
 
-        LOG_MSG("columnNum: %lld\n", columnNum);
-        LOG_MSG("dataTypeRes: %lld\n", dataTypeRes);
-        LOG_MSG("columnSizeRes: %lld\n", columnSizeRes);
-        LOG_MSG("decimalDigitsRes: %lld\n", decimalDigitsRes);
-        LOG_MSG("nullableRes: %lld\n", nullableRes);
-        LOG_MSG("columnNameBuf: %s\n", columnNameBuf ? reinterpret_cast<const char*>(columnNameBuf) : "<null>");
-        LOG_MSG("columnNameLen: %d\n", columnNameLen ? *columnNameLen : -1);
+        LOG_MSG("columnNum: " << columnNum);
+        LOG_MSG("dataTypeRes: " << dataTypeRes);
+        LOG_MSG("columnSizeRes: " << columnSizeRes);
+        LOG_MSG("decimalDigitsRes: " << decimalDigitsRes);
+        LOG_MSG("nullableRes: " << nullableRes);
+        LOG_MSG("columnNameBuf: " << (columnNameBuf ? reinterpret_cast<const char*>(columnNameBuf) : "<null>"));
+        LOG_MSG("columnNameLen: " << (columnNameLen ? *columnNameLen : -1));
 
         if (dataType)
             *dataType = static_cast<SQLSMALLINT>(dataTypeRes);
@@ -737,7 +744,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLRowCount called\n");
+        LOG_MSG("SQLRowCount called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -769,7 +776,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLForeignKeys called\n");
+        LOG_MSG("SQLForeignKeys called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -783,12 +790,12 @@ namespace ignite
         std::string foreignSchema = SqlStringToString(foreignSchemaName, foreignSchemaNameLen);
         std::string foreignTable = SqlStringToString(foreignTableName, foreignTableNameLen);
 
-        LOG_MSG("primaryCatalog: %s\n", primaryCatalog.c_str());
-        LOG_MSG("primarySchema: %s\n", primarySchema.c_str());
-        LOG_MSG("primaryTable: %s\n", primaryTable.c_str());
-        LOG_MSG("foreignCatalog: %s\n", foreignCatalog.c_str());
-        LOG_MSG("foreignSchema: %s\n", foreignSchema.c_str());
-        LOG_MSG("foreignTable: %s\n", foreignTable.c_str());
+        LOG_MSG("primaryCatalog: " << primaryCatalog);
+        LOG_MSG("primarySchema: " << primarySchema);
+        LOG_MSG("primaryTable: " << primaryTable);
+        LOG_MSG("foreignCatalog: " << foreignCatalog);
+        LOG_MSG("foreignSchema: " << foreignSchema);
+        LOG_MSG("foreignTable: " << foreignTable);
 
         statement->ExecuteGetForeignKeysQuery(primaryCatalog, primarySchema,
             primaryTable, foreignCatalog, foreignSchema, foreignTable);
@@ -809,7 +816,7 @@ namespace ignite
 #ifdef ODBC_DEBUG
         using odbc::type_traits::StatementAttrIdToString;
 
-        LOG_MSG("Attr: %s (%d)\n", StatementAttrIdToString(attr), attr);
+        LOG_MSG("Attr: " << StatementAttrIdToString(attr) << " (" << attr << ")");
 #endif //ODBC_DEBUG
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
@@ -834,7 +841,7 @@ namespace ignite
 #ifdef ODBC_DEBUG
         using odbc::type_traits::StatementAttrIdToString;
 
-        LOG_MSG("Attr: %s (%d)\n", StatementAttrIdToString(attr), attr);
+        LOG_MSG("Attr: " << StatementAttrIdToString(attr) << " (" << attr << ")");
 #endif //ODBC_DEBUG
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
@@ -858,7 +865,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLPrimaryKeys called\n");
+        LOG_MSG("SQLPrimaryKeys called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -869,9 +876,9 @@ namespace ignite
         std::string schema = SqlStringToString(schemaName, schemaNameLen);
         std::string table = SqlStringToString(tableName, tableNameLen);
 
-        LOG_MSG("catalog: %s\n", catalog.c_str());
-        LOG_MSG("schema: %s\n", schema.c_str());
-        LOG_MSG("table: %s\n", table.c_str());
+        LOG_MSG("catalog: " << catalog);
+        LOG_MSG("schema: " << schema);
+        LOG_MSG("table: " << table);
 
         statement->ExecuteGetPrimaryKeysQuery(catalog, schema, table);
 
@@ -882,7 +889,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLNumParams called\n");
+        LOG_MSG("SQLNumParams called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -909,7 +916,7 @@ namespace ignite
 
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLGetDiagField called: %d\n", recNum);
+        LOG_MSG("SQLGetDiagField called: " << recNum);
 
         SqlLen outResLen;
         ApplicationDataBuffer outBuffer(IGNITE_ODBC_C_TYPE_DEFAULT, buffer, bufferLen, &outResLen);
@@ -960,7 +967,7 @@ namespace ignite
 
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLGetDiagRec called\n");
+        LOG_MSG("SQLGetDiagRec called");
 
         const DiagnosticRecordStorage* records = 0;
 
@@ -1007,7 +1014,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLGetTypeInfo called\n");
+        LOG_MSG("SQLGetTypeInfo called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1023,7 +1030,7 @@ namespace ignite
     {
         using namespace odbc;
 
-        LOG_MSG("SQLEndTran called\n");
+        LOG_MSG("SQLEndTran called");
 
         SQLRETURN result;
 
@@ -1086,7 +1093,7 @@ namespace ignite
         using odbc::Statement;
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLGetData called\n");
+        LOG_MSG("SQLGetData called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1109,7 +1116,7 @@ namespace ignite
     {
         using odbc::Environment;
 
-        LOG_MSG("SQLSetEnvAttr called\n");
+        LOG_MSG("SQLSetEnvAttr called");
 
         Environment *environment = reinterpret_cast<Environment*>(env);
 
@@ -1132,7 +1139,7 @@ namespace ignite
 
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLGetEnvAttr called\n");
+        LOG_MSG("SQLGetEnvAttr called");
 
         Environment *environment = reinterpret_cast<Environment*>(env);
 
@@ -1166,7 +1173,7 @@ namespace ignite
 
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLSpecialColumns called\n");
+        LOG_MSG("SQLSpecialColumns called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1177,9 +1184,9 @@ namespace ignite
         std::string schema = SqlStringToString(schemaName, schemaNameLen);
         std::string table = SqlStringToString(tableName, tableNameLen);
 
-        LOG_MSG("catalog: %s\n", catalog.c_str());
-        LOG_MSG("schema: %s\n", schema.c_str());
-        LOG_MSG("table: %s\n", table.c_str());
+        LOG_MSG("catalog: " << catalog);
+        LOG_MSG("schema: " << schema);
+        LOG_MSG("table: " << table);
 
         statement->ExecuteSpecialColumnsQuery(idType, catalog, schema, table, scope, nullable);
 
@@ -1190,7 +1197,7 @@ namespace ignite
     {
         using namespace ignite::odbc;
 
-        LOG_MSG("SQLParamData called\n");
+        LOG_MSG("SQLParamData called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1206,7 +1213,7 @@ namespace ignite
     {
         using namespace ignite::odbc;
 
-        LOG_MSG("SQLPutData called\n");
+        LOG_MSG("SQLPutData called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1227,7 +1234,7 @@ namespace ignite
     {
         using namespace ignite::odbc;
 
-        LOG_MSG("SQLDescribeParam called\n");
+        LOG_MSG("SQLDescribeParam called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1255,7 +1262,7 @@ namespace ignite
 
         using ignite::odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLError called\n");
+        LOG_MSG("SQLError called");
 
         SQLHANDLE handle = 0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp b/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
index 1cd2a9f..9aa5f7d 100644
--- a/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
@@ -20,6 +20,7 @@
 #include "ignite/odbc/type_traits.h"
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/query/column_metadata_query.h"
 
 namespace
@@ -292,8 +293,7 @@ namespace ignite
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
-
+                    LOG_MSG("Error: " << rsp.GetError());
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
                     return SQL_RESULT_ERROR;
@@ -303,11 +303,10 @@ namespace ignite
 
                 for (size_t i = 0; i < meta.size(); ++i)
                 {
-                    LOG_MSG("[%d] SchemaName:     %s\n", i, meta[i].GetSchemaName().c_str());
-                    LOG_MSG("[%d] TableName:      %s\n", i, meta[i].GetTableName().c_str());
-                    LOG_MSG("[%d] ColumnName:     %s\n", i, meta[i].GetColumnName().c_str());
-                    LOG_MSG("[%d] ColumnType:     %d\n", i, meta[i].GetDataType());
-                    LOG_MSG("\n");
+                    LOG_MSG("\n[" << i << "] SchemaName:     " << meta[i].GetSchemaName()
+                         << "\n[" << i << "] TableName:      " << meta[i].GetTableName()
+                         << "\n[" << i << "] ColumnName:     " << meta[i].GetColumnName()
+                         << "\n[" << i << "] ColumnType:     " << meta[i].GetDataType());
                 }
 
                 return SQL_RESULT_SUCCESS;

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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 5b1b758..c9762ad 100644
--- a/modules/platforms/cpp/odbc/src/query/data_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp
@@ -17,6 +17,7 @@
 
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/query/data_query.h"
 
 namespace ignite
@@ -40,7 +41,7 @@ namespace ignite
             {
                 Close();
             }
-            
+
             SqlResult DataQuery::Execute()
             {
                 if (cursor.get())
@@ -186,7 +187,7 @@ namespace ignite
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                    LOG_MSG("Error: " << rsp.GetError());
 
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
@@ -197,15 +198,13 @@ namespace ignite
 
                 resultMeta.assign(rsp.GetMeta().begin(), rsp.GetMeta().end());
 
-                LOG_MSG("Query id: %lld\n", cursor->GetQueryId());
-
+                LOG_MSG("Query id: " << cursor->GetQueryId());
                 for (size_t i = 0; i < rsp.GetMeta().size(); ++i)
                 {
-                    LOG_MSG("[%d] SchemaName:     %s\n", i, rsp.GetMeta()[i].GetSchemaName().c_str());
-                    LOG_MSG("[%d] TypeName:       %s\n", i, rsp.GetMeta()[i].GetTableName().c_str());
-                    LOG_MSG("[%d] ColumnName:     %s\n", i, rsp.GetMeta()[i].GetColumnName().c_str());
-                    LOG_MSG("[%d] ColumnType:     %d\n", i, rsp.GetMeta()[i].GetDataType());
-                    LOG_MSG("\n");
+                    LOG_MSG("\n[" << i << "] SchemaName:     " << rsp.GetMeta()[i].GetSchemaName()
+                        <<  "\n[" << i << "] TypeName:       " << rsp.GetMeta()[i].GetTableName()
+                        <<  "\n[" << i << "] ColumnName:     " << rsp.GetMeta()[i].GetColumnName()
+                        <<  "\n[" << i << "] ColumnType:     " << rsp.GetMeta()[i].GetDataType());
                 }
 
                 return SQL_RESULT_SUCCESS;
@@ -227,11 +226,11 @@ namespace ignite
                     return SQL_RESULT_ERROR;
                 }
 
-                LOG_MSG("Query id: %lld\n", rsp.GetQueryId());
+                LOG_MSG("Query id: " << rsp.GetQueryId());
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                    LOG_MSG("Error: " << rsp.GetError());
 
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
@@ -261,7 +260,7 @@ namespace ignite
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                    LOG_MSG("Error: " << rsp.GetError());
 
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp b/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
index 3cddd1b..4fd5f7b 100644
--- a/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
@@ -20,6 +20,7 @@
 #include "ignite/odbc/type_traits.h"
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/query/table_metadata_query.h"
 
 namespace
@@ -218,7 +219,7 @@ namespace ignite
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                    LOG_MSG("Error: " << rsp.GetError());
 
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
@@ -229,11 +230,10 @@ namespace ignite
 
                 for (size_t i = 0; i < meta.size(); ++i)
                 {
-                    LOG_MSG("[%d] CatalogName: %s\n", i, meta[i].GetCatalogName().c_str());
-                    LOG_MSG("[%d] SchemaName:  %s\n", i, meta[i].GetSchemaName().c_str());
-                    LOG_MSG("[%d] TableName:   %s\n", i, meta[i].GetTableName().c_str());
-                    LOG_MSG("[%d] TableType:   %s\n", i, meta[i].GetTableType().c_str());
-                    LOG_MSG("\n");
+                    LOG_MSG("\n[" << i << "] CatalogName: " << meta[i].GetCatalogName()
+                         << "\n[" << i << "] SchemaName:  " << meta[i].GetSchemaName()
+                         << "\n[" << i << "] TableName:   " << meta[i].GetTableName()
+                         << "\n[" << i << "] TableType:   " << meta[i].GetTableType());
                 }
 
                 return SQL_RESULT_SUCCESS;

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/statement.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp
index 09ad81c..2395d66 100644
--- a/modules/platforms/cpp/odbc/src/statement.cpp
+++ b/modules/platforms/cpp/odbc/src/statement.cpp
@@ -27,6 +27,7 @@
 #include "ignite/odbc/utility.h"
 #include "ignite/odbc/message.h"
 #include "ignite/odbc/statement.h"
+#include "ignite/odbc/log.h"
 
 namespace ignite
 {
@@ -232,7 +233,7 @@ namespace ignite
                 {
                     SQLULEN val = reinterpret_cast<SQLULEN>(value);
 
-                    LOG_MSG("SQL_ATTR_ROW_ARRAY_SIZE: %d\n", val);
+                    LOG_MSG("SQL_ATTR_ROW_ARRAY_SIZE: " << val);
 
                     if (val != 1)
                     {
@@ -986,7 +987,7 @@ namespace ignite
             if (paramNum > 0 && static_cast<size_t>(paramNum) <= paramTypes.size())
                 type = paramTypes[paramNum - 1];
 
-            LOG_MSG("Type: %d\n", type);
+            LOG_MSG("Type: " << type);
 
             if (!type)
             {
@@ -1044,7 +1045,7 @@ namespace ignite
 
             if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
             {
-                LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                LOG_MSG("Error: " << rsp.GetError());
 
                 AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
@@ -1054,7 +1055,9 @@ namespace ignite
             paramTypes = rsp.GetTypeIds();
 
             for (size_t i = 0; i < paramTypes.size(); ++i)
-                LOG_MSG("[%zu] Parameter type: %u\n", i, paramTypes[i]);
+            {
+                LOG_MSG("[" << i << "] Parameter type: " << paramTypes[i]);
+            }
 
             return SQL_RESULT_SUCCESS;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/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 c15c23c..63454dc 100644
--- a/modules/platforms/cpp/odbc/src/utility.cpp
+++ b/modules/platforms/cpp/odbc/src/utility.cpp
@@ -22,20 +22,6 @@
 #include "ignite/odbc/utility.h"
 #include "ignite/odbc/system/odbc_constants.h"
 
-#ifdef ODBC_DEBUG
-
-FILE* log_file = NULL;
-
-void logInit(const char* path)
-{
-    if (!log_file)
-    {
-        log_file = fopen(path, "w");
-    }
-}
-
-#endif //ODBC_DEBUG
-
 namespace ignite
 {
     namespace utility
@@ -155,11 +141,11 @@ namespace ignite
                 res.clear();
         }
 
-        std::string HexDump(const char* data, size_t count)
+        std::string HexDump(const void* data, size_t count)
         {
             std::stringstream  dump;
             size_t cnt = 0;
-            for(const char* p = data, *e = data + count; p != e; ++p)
+            for(const uint8_t* p = (const uint8_t*)data, *e = (const uint8_t*)data + count; p != e; ++p)
             {
                 if (cnt++ % 16 == 0)
                 {


[02/18] ignite git commit: .NET: Fix non-ascii chars in AssemblyInfo

Posted by vo...@apache.org.
.NET: Fix non-ascii chars in AssemblyInfo


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

Branch: refs/heads/ignite-2.0
Commit: d2e6007b905b6c19cd87786a039229156d10c013
Parents: f406887
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Jan 9 12:40:42 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Jan 9 12:40:42 2017 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs   | 2 +-
 .../dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs         | 2 +-
 .../dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs     | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6007b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
index f5fa618..1bca0e8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
@@ -1,4 +1,4 @@
-\ufeff\ufeff/*
+\ufeff/*
 * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6007b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
index d72c9db..0926a46 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
@@ -1,4 +1,4 @@
-\ufeff\ufeff/*
+\ufeff/*
 * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6007b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index cc833ea..1fc6c59 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@ -1,4 +1,4 @@
-\ufeff\ufeff/*
+\ufeff/*
 * 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.