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

[1/7] ignite git commit: IGNITE-6115: Ignore page eviction mode if Ignite persistence is enabled. - Fixes #2523.

Repository: ignite
Updated Branches:
  refs/heads/ignite-5896 085cc80e5 -> 4241f5d2c


IGNITE-6115: Ignore page eviction mode if Ignite persistence is enabled. - Fixes #2523.

Signed-off-by: shroman <rs...@yahoo.com>


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

Branch: refs/heads/ignite-5896
Commit: f9be391856ceff2268249fccbfd3c5ca0132378e
Parents: eae6e3b
Author: shroman <rs...@yahoo.com>
Authored: Thu Aug 31 12:15:09 2017 +0900
Committer: shroman <rs...@yahoo.com>
Committed: Thu Aug 31 12:15:09 2017 +0900

----------------------------------------------------------------------
 .../GridCacheDatabaseSharedManager.java         | 31 ++++++++++++--------
 .../IgniteCacheDatabaseSharedManager.java       |  2 +-
 .../db/file/IgnitePdsEvictionTest.java          |  3 +-
 3 files changed, 21 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f9be3918/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index b4502d2..c59cccc 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -478,6 +478,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /**
      * Try to register Metrics MBean.
+     *
      * @throws IgniteCheckedException If failed.
      */
     private void registrateMetricsMBean() throws IgniteCheckedException {
@@ -535,15 +536,16 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 }
 
                 if (totalSize == 0)
-                    return (float) 0;
+                    return (float)0;
 
-                return (float) loadSize / totalSize;
+                return (float)loadSize / totalSize;
             }
         };
     }
 
     /** {@inheritDoc} */
-    @Override public void readCheckpointAndRestoreMemory(List<DynamicCacheDescriptor> cachesToStart) throws IgniteCheckedException {
+    @Override public void readCheckpointAndRestoreMemory(
+        List<DynamicCacheDescriptor> cachesToStart) throws IgniteCheckedException {
         checkpointReadLock();
 
         try {
@@ -688,7 +690,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     @Override protected void checkPolicyEvictionProperties(MemoryPolicyConfiguration plcCfg, MemoryConfiguration dbCfg)
         throws IgniteCheckedException {
         if (plcCfg.getPageEvictionMode() != DataPageEvictionMode.DISABLED)
-            throw new IgniteCheckedException("Page eviction is not compatible with persistence: " + plcCfg.getName());
+            U.warn(log, "Page eviction mode for [" + plcCfg.getName() + "] memory region is ignored " +
+                "because Ignite Native Persistence is enabled");
     }
 
     /**
@@ -858,7 +861,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
-     * Gets the checkpoint read lock. While this lock is held, checkpoint thread will not acquiSnapshotWorkerre memory state.
+     * Gets the checkpoint read lock. While this lock is held, checkpoint thread will not acquiSnapshotWorkerre memory
+     * state.
      */
     @SuppressWarnings("LockAcquiredButNotSafelyReleased")
     @Override public void checkpointReadLock() {
@@ -909,7 +913,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             return true;
 
         for (MemoryPolicy memPlc : memPlcs) {
-            PageMemoryEx pageMemEx = (PageMemoryEx) memPlc.pageMemory();
+            PageMemoryEx pageMemEx = (PageMemoryEx)memPlc.pageMemory();
 
             if (!pageMemEx.safeToUpdate())
                 return false;
@@ -1302,6 +1306,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /**
      * Loads WAL pointer from CP file
+     *
      * @param cpMarkerFile Checkpoint mark file.
      * @return WAL pointer.
      * @throws IgniteCheckedException If failed to read mark file.
@@ -1706,7 +1711,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         List<IgniteBiTuple<PageMemory, Collection<FullPageId>>> cpEntities = new ArrayList<>(memPolicies.size());
 
         for (MemoryPolicy memPlc : memPolicies) {
-            PageMemoryEx pageMem = (PageMemoryEx) memPlc.pageMemory();
+            PageMemoryEx pageMem = (PageMemoryEx)memPlc.pageMemory();
             cpEntities.add(new IgniteBiTuple<PageMemory, Collection<FullPageId>>(pageMem,
                 (pageMem).beginCheckpoint()));
         }
@@ -1719,7 +1724,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         int cpPagesCnt = 0;
 
         for (IgniteBiTuple<PageMemory, Collection<FullPageId>> e : cpEntities) {
-            PageMemoryEx pageMem = (PageMemoryEx) e.get1();
+            PageMemoryEx pageMem = (PageMemoryEx)e.get1();
 
             Collection<FullPageId> cpPages = e.get2();
 
@@ -2251,7 +2256,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 if (printCheckpointStats) {
                     if (log.isInfoEnabled())
                         LT.info(log, String.format("Skipping checkpoint (no pages were modified) [" +
-                            "checkpointLockWait=%dms, checkpointLockHoldTime=%dms, reason='%s']",
+                                "checkpointLockWait=%dms, checkpointLockHoldTime=%dms, reason='%s']",
                             tracker.lockWaitDuration(),
                             tracker.lockHoldDuration(),
                             curr.reason));
@@ -2290,7 +2295,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             int pagesNum = 0;
 
             for (MemoryPolicy memPlc : memoryPolicies()) {
-                GridMultiCollectionWrapper<FullPageId> nextCpPagesCol = ((PageMemoryEx) memPlc.pageMemory()).beginCheckpoint();
+                GridMultiCollectionWrapper<FullPageId> nextCpPagesCol = ((PageMemoryEx)memPlc.pageMemory()).beginCheckpoint();
 
                 pagesNum += nextCpPagesCol.size();
 
@@ -2403,7 +2408,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         /** */
         private CheckpointMetricsTracker tracker;
 
-        /** Collection of page IDs to write under this task. Overall pages to write may be greater than this collection*/
+        /** Collection of page IDs to write under this task. Overall pages to write may be greater than this collection */
         private Collection<FullPageId> writePageIds;
 
         /** */
@@ -2415,11 +2420,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         /** Counter for all written pages. May be shared between several workers */
         private AtomicInteger writtenPagesCntr;
 
-        /** Total pages to write, counter may be greater than {@link #writePageIds} size*/
+        /** Total pages to write, counter may be greater than {@link #writePageIds} size */
         private final int totalPagesToWrite;
 
         /**
          * Creates task for write pages
+         *
          * @param tracker
          * @param writePageIds Collection of page IDs to write.
          * @param updStores
@@ -2810,7 +2816,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
 
         /**
-         *
          * @param cacheId Cache ID.
          * @param partId Partition ID.
          * @return Reserved counter or null if couldn't reserve.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9be3918/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index 69feac8..37e0108 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -901,7 +901,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @param pageMem Page memory.
      */
     private PageEvictionTracker createPageEvictionTracker(MemoryPolicyConfiguration plc, PageMemory pageMem) {
-        if (plc.getPageEvictionMode() == DataPageEvictionMode.DISABLED)
+        if (plc.getPageEvictionMode() == DataPageEvictionMode.DISABLED || cctx.gridConfig().isPersistentStoreEnabled())
             return new NoOpPageEvictionTracker();
 
         assert pageMem instanceof PageMemoryNoStoreImpl : pageMem.getClass();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9be3918/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
index d7790c3..8544cf1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataPageEvictionMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
@@ -84,6 +85,7 @@ public class IgnitePdsEvictionTest extends GridCommonAbstractTest {
         MemoryPolicyConfiguration memPlcCfg = new MemoryPolicyConfiguration();
         memPlcCfg.setInitialSize(MEMORY_LIMIT);
         memPlcCfg.setMaxSize(MEMORY_LIMIT);
+        memPlcCfg.setPageEvictionMode(DataPageEvictionMode.RANDOM_LRU);
         memPlcCfg.setName("dfltMemPlc");
 
         memCfg.setPageSize(PAGE_SIZE);
@@ -94,7 +96,6 @@ public class IgnitePdsEvictionTest extends GridCommonAbstractTest {
         return memCfg;
     }
 
-
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();


[2/7] ignite git commit: IGNITE-5409: JDBC thin: added schema to connection URL. This closes #2179.

Posted by pt...@apache.org.
IGNITE-5409: JDBC thin: added schema to connection URL. This closes #2179.


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

Branch: refs/heads/ignite-5896
Commit: bc8e64567eebfb4bfa30e989ee63ddae65b20693
Parents: f9be391
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Aug 31 11:03:18 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 31 11:03:18 2017 +0300

----------------------------------------------------------------------
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   | 22 ++++++
 .../jdbc/thin/JdbcThinNoDefaultSchemaTest.java  | 82 +++++++++++++++++++-
 .../org/apache/ignite/IgniteJdbcThinDriver.java | 20 +++--
 .../internal/jdbc/thin/JdbcThinConnection.java  | 11 ++-
 .../internal/jdbc/thin/JdbcThinUtils.java       |  2 +-
 5 files changed, 123 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bc8e6456/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
index a2e7be1..2306a2b 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
@@ -287,6 +287,28 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
     }
 
     /**
+     * Test schema property in URL.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSchema() throws Exception {
+        assertInvalid("jdbc:ignite:thin://127.0.0.1/qwe/qwe",
+            "Invalid URL format (only schema name is allowed in URL path parameter 'host:port[/schemaName]')" );
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/public")) {
+            assertEquals("Invalid schema", "public", conn.getSchema());
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/" + DEFAULT_CACHE_NAME)) {
+            assertEquals("Invalid schema", DEFAULT_CACHE_NAME, conn.getSchema());
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/_not_exist_schema_")) {
+            assertEquals("Invalid schema", "_not_exist_schema_", conn.getSchema());
+        }
+    }
+
+    /**
      * Get client socket for connection.
      *
      * @param conn Connection.

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc8e6456/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
index ff49742..7b226e5 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
@@ -20,7 +20,9 @@ package org.apache.ignite.jdbc.thin;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -28,6 +30,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -119,11 +122,9 @@ public class JdbcThinNoDefaultSchemaTest extends JdbcThinAbstractSelfTest {
     /**
      * @throws Exception If failed.
      */
-    public void testNoCacheNameQuery() throws Exception {
+    public void testSchemaNameInQuery() throws Exception {
         Connection conn = DriverManager.getConnection(URL);
 
-        conn.setSchema("cache1");
-
         Statement stmt = conn.createStatement();
 
         assertNotNull(stmt);
@@ -155,4 +156,79 @@ public class JdbcThinNoDefaultSchemaTest extends JdbcThinAbstractSelfTest {
 
         stmt.close();
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSchemaInUrl() throws Exception {
+        try(Connection conn = DriverManager.getConnection(URL + "/cache1")) {
+            Statement stmt = conn.createStatement();
+
+            stmt.execute("select t._key, t._val from Integer t");
+
+            ResultSet rs = stmt.getResultSet();
+
+            while (rs.next())
+                assertEquals(rs.getInt(2), rs.getInt(1) * 2);
+        }
+
+        try(Connection conn = DriverManager.getConnection(URL + "/cache2")) {
+            Statement stmt = conn.createStatement();
+
+            stmt.execute("select t._key, t._val from Integer t");
+
+            ResultSet rs = stmt.getResultSet();
+
+            while (rs.next())
+                assertEquals(rs.getInt(2), rs.getInt(1) * 3);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSchemaInUrlAndInQuery() throws Exception {
+        try(Connection conn = DriverManager.getConnection(URL + "/cache2")) {
+            Statement stmt = conn.createStatement();
+
+            stmt.execute("select t._key, t._val, v._val " +
+                "from \"cache1\".Integer t join Integer v on t._key = v._key");
+
+            ResultSet rs = stmt.getResultSet();
+
+            while (rs.next()) {
+                assertEquals(rs.getInt(2), rs.getInt(1) * 2);
+                assertEquals(rs.getInt(3), rs.getInt(1) * 3);
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetSchema() throws Exception {
+        try(Connection conn = DriverManager.getConnection(URL)) {
+            // Try to execute query without set schema
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Statement stmt = conn.createStatement();
+
+                    stmt.execute("select t._key, t._val from Integer t");
+
+                    return null;
+                }
+            }, SQLException.class, "Failed to query Ignite");
+
+            conn.setSchema("cache1");
+
+            Statement stmt = conn.createStatement();
+
+            stmt.execute("select t._key, t._val from Integer t");
+
+            ResultSet rs = stmt.getResultSet();
+
+            while (rs.next())
+                assertEquals(rs.getInt(2), rs.getInt(1) * 2);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc8e6456/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
index 7e23d7a..423223c 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
@@ -166,9 +166,9 @@ public class IgniteJdbcThinDriver implements Driver {
         if (!acceptsURL(url))
             return null;
 
-        parseUrl(url, props);
+        String schema = parseUrl(url, props);
 
-        return new JdbcThinConnection(url, props);
+        return new JdbcThinConnection(url, props, schema);
     }
 
     /** {@inheritDoc} */
@@ -218,9 +218,10 @@ public class IgniteJdbcThinDriver implements Driver {
      *
      * @param props Properties.
      * @param url URL.
+     * @return Scheme name. {@code null} in case the schema isn't specified in the url.
      * @throws SQLException On error.
      */
-    private void parseUrl(String url, Properties props) throws SQLException {
+    private String parseUrl(String url, Properties props) throws SQLException {
         if (F.isEmpty(url))
             throw new SQLException("URL cannot be null or empty.");
 
@@ -234,10 +235,9 @@ public class IgniteJdbcThinDriver implements Driver {
         if (nakedUrlParts.length > 2)
             throw new SQLException("Invalid URL format (only one ? character is allowed): " + url);
 
-        String endpoint = nakedUrlParts[0];
+        String[] pathParts = nakedUrlParts[0].split("/");
 
-        if (endpoint.endsWith("/"))
-            endpoint = endpoint.substring(0, endpoint.length() - 1);
+        String endpoint = pathParts[0];
 
         String[] endpointParts = endpoint.split(":");
 
@@ -251,6 +251,14 @@ public class IgniteJdbcThinDriver implements Driver {
 
         if (nakedUrlParts.length == 2)
             parseParameters(nakedUrlParts[1], props);
+
+        if (pathParts.length > 2) {
+            throw new SQLException("Invalid URL format (only schema name is allowed in URL path parameter " +
+                "'host:port[/schemaName]'): " + url);
+        }
+
+        // Gets schema from URL string & returns.
+        return pathParts.length == 2 ? pathParts[1] : null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc8e6456/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
index 1ed0844..370cfa4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
@@ -69,7 +69,7 @@ public class JdbcThinConnection implements Connection {
     private String url;
 
     /** Schema name. */
-    private String schemaName;
+    private String schema;
 
     /** Closed flag. */
     private boolean closed;
@@ -100,9 +100,10 @@ public class JdbcThinConnection implements Connection {
      *
      * @param url Connection URL.
      * @param props Additional properties.
+     * @param schema Schema name.
      * @throws SQLException In case Ignite client failed to start.
      */
-    public JdbcThinConnection(String url, Properties props) throws SQLException {
+    public JdbcThinConnection(String url, Properties props, String schema) throws SQLException {
         assert url != null;
         assert props != null;
 
@@ -112,6 +113,8 @@ public class JdbcThinConnection implements Connection {
         autoCommit = true;
         txIsolation = Connection.TRANSACTION_NONE;
 
+        this.schema = schema;
+
         String host = extractHost(props);
         int port = extractPort(props);
 
@@ -522,12 +525,12 @@ public class JdbcThinConnection implements Connection {
 
     /** {@inheritDoc} */
     @Override public void setSchema(String schema) throws SQLException {
-        schemaName = schema;
+        this.schema = schema;
     }
 
     /** {@inheritDoc} */
     @Override public String getSchema() throws SQLException {
-        return schemaName;
+        return schema;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc8e6456/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinUtils.java
index 675cf2d..bb6eb14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinUtils.java
@@ -46,7 +46,7 @@ public class JdbcThinUtils {
     public static final String URL_PREFIX = "jdbc:ignite:thin://";
 
     /** Prefix for property names. */
-    public static final String PROP_PREFIX = "ignite.jdbc";
+    public static final String PROP_PREFIX = "ignite.jdbc.";
 
     /** Port number property name. */
     public static final String PROP_PORT = PROP_PREFIX + "port";


[7/7] ignite git commit: Merge branch 'master' into ignite-5896

Posted by pt...@apache.org.
Merge branch 'master' into ignite-5896

# Conflicts:
#	modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj


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

Branch: refs/heads/ignite-5896
Commit: 4241f5d2cd5a724c3f781de86d472480a2ce1e85
Parents: 085cc80 5738d7e
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Aug 31 18:13:08 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Aug 31 18:13:08 2017 +0300

----------------------------------------------------------------------
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   4 +
 .../thin/JdbcThinComplexDmlDdlSelfTest.java     | 486 +++++++++++++++++++
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   |  22 +
 .../jdbc/thin/JdbcThinNoDefaultSchemaTest.java  |  82 +++-
 .../org/apache/ignite/IgniteJdbcThinDriver.java |  20 +-
 .../internal/jdbc/thin/JdbcThinConnection.java  |  17 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     |   1 -
 .../internal/jdbc/thin/JdbcThinStatement.java   |   2 +-
 .../internal/jdbc/thin/JdbcThinUtils.java       |   2 +-
 .../GridCacheDatabaseSharedManager.java         |  31 +-
 .../IgniteCacheDatabaseSharedManager.java       |   2 +-
 .../db/file/IgnitePdsEvictionTest.java          |   3 +-
 .../Binary/BinarySelfTest.cs                    |   7 +-
 .../Cache/CacheAbstractTest.cs                  |  50 +-
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |  10 +-
 .../Impl/Binary/BinaryWriter.cs                 |   9 +-
 .../Impl/Binary/BinaryWriterExtensions.cs       |   4 +-
 .../Impl/Binary/Marshaller.cs                   |   4 +-
 .../Impl/Cache/CacheAffinityImpl.cs             |   2 +-
 .../Impl/Cache/CacheEntryFilterHolder.cs        |   2 +-
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |   4 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   2 +-
 .../Impl/Cache/Store/CacheStoreInternal.cs      |  14 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |   2 +-
 .../Impl/Compute/Closure/ComputeActionJob.cs    |   4 +-
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |   6 +-
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |   4 +-
 .../Closure/ComputeReducingClosureTask.cs       |   2 +-
 .../Compute/Closure/IComputeResourceInjector.cs |   2 +-
 .../Impl/Compute/ComputeFunc.cs                 |   2 +-
 .../Impl/Compute/ComputeJob.cs                  |   2 +-
 .../Impl/Compute/ComputeJobHolder.cs            |  21 +-
 .../Impl/Compute/ComputeOutFunc.cs              |   2 +-
 .../Impl/Deployment/PeerAssemblyResolver.cs     |  13 +-
 .../Impl/Deployment/PeerLoadingExtensions.cs    |   4 +-
 .../Impl/Deployment/PeerLoadingObjectHolder.cs  |   4 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   8 +-
 .../Apache.Ignite.Core/Impl/IIgniteInternal.cs  |  69 +++
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  13 +-
 .../Impl/Messaging/MessageListenerHolder.cs     |   4 +-
 .../Impl/PlatformTargetAdapter.cs               |  13 +-
 .../Impl/Resource/ResourceProcessor.cs          |   2 +-
 .../Impl/Resource/ResourceTypeDescriptor.cs     |   2 +-
 44 files changed, 823 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4241f5d2/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 8a384fd,ee301be..6666533
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@@ -97,14 -97,9 +97,15 @@@
      <Compile Include="Cache\Configuration\MemoryPolicyConfiguration.cs" />
      <Compile Include="Cache\Configuration\PartitionLossPolicy.cs" />
      <Compile Include="Cache\IMemoryMetrics.cs" />
 +    <Compile Include="Client\IgniteClientConfiguration.cs" />
      <Compile Include="Common\ExceptionFactory.cs" />
      <Compile Include="Configuration\Package-Info.cs" />
+     <Compile Include="Impl\IIgniteInternal.cs" />
 +    <Compile Include="Impl\Cache\CacheClient.cs" />
 +    <Compile Include="Impl\Client\ClientOp.cs" />
 +    <Compile Include="Impl\Client\ClientProtocolVersion.cs" />
 +    <Compile Include="Impl\Client\ClientSocket.cs" />
 +    <Compile Include="Impl\Client\IgniteClient.cs" />
      <Compile Include="Impl\IPlatformTargetInternal.cs" />
      <Compile Include="Impl\PersistentStore\PersistentStoreMetrics.cs" />
      <Compile Include="Impl\PlatformDisposableTargetAdapter.cs" />


[5/7] ignite git commit: IGNITE-5425 JDBC thin driver: don't throw unsupported exception on ClientInfo methods. This closes #2184. This closes #2525.

Posted by pt...@apache.org.
IGNITE-5425 JDBC thin driver: don't throw unsupported exception on ClientInfo methods. This closes #2184. This closes #2525.


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

Branch: refs/heads/ignite-5896
Commit: fd9ab957911ee218544ded288dd42d77abdbb5ef
Parents: 8629b50
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Aug 31 17:03:44 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 31 17:03:44 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/jdbc/thin/JdbcThinConnection.java   | 6 ++++--
 .../ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java    | 1 -
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fd9ab957/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
index 370cfa4..ff76011 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
@@ -473,12 +473,14 @@ public class JdbcThinConnection implements Connection {
 
     /** {@inheritDoc} */
     @Override public void setClientInfo(String name, String val) throws SQLClientInfoException {
-        throw new UnsupportedOperationException("Client info is not supported.");
+        if (closed)
+            throw new SQLClientInfoException("Connection is closed.", null);
     }
 
     /** {@inheritDoc} */
     @Override public void setClientInfo(Properties props) throws SQLClientInfoException {
-        throw new UnsupportedOperationException("Client info is not supported.");
+        if (closed)
+            throw new SQLClientInfoException("Connection is closed.", null);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd9ab957/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
index 583bcec..32e1858 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
@@ -1490,7 +1490,6 @@ public class JdbcThinDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public ResultSet getClientInfoProperties() throws SQLException {
-        // TODO: IGNITE-5425.
         return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
             new JdbcColumnMeta(null, null, "NAME", String.class),
             new JdbcColumnMeta(null, null, "MAX_LEN", Integer.class),


[3/7] ignite git commit: IGNITE-6081 .NET: Fix PutAll for dependent objects

Posted by pt...@apache.org.
IGNITE-6081 .NET: Fix PutAll for dependent objects

This closes #2555


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

Branch: refs/heads/ignite-5896
Commit: f104976fe197d592c8265b3fd30bb0324f624e38
Parents: bc8e645
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Aug 31 11:38:37 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Aug 31 11:38:37 2017 +0300

----------------------------------------------------------------------
 .../Binary/BinarySelfTest.cs                    |  7 +--
 .../Cache/CacheAbstractTest.cs                  | 50 ++++++++++++++++----
 .../Impl/Binary/BinaryWriter.cs                 |  9 ++--
 .../Impl/Binary/BinaryWriterExtensions.cs       |  4 +-
 .../Impl/Cache/CacheEntryFilterHolder.cs        |  2 +-
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |  4 +-
 .../Impl/Cache/Store/CacheStoreInternal.cs      | 14 ++----
 .../Impl/Compute/Closure/ComputeActionJob.cs    |  2 +-
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |  4 +-
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |  2 +-
 .../Impl/Compute/ComputeFunc.cs                 |  2 +-
 .../Impl/Compute/ComputeJob.cs                  |  2 +-
 .../Impl/Compute/ComputeJobHolder.cs            |  2 +-
 .../Impl/Compute/ComputeOutFunc.cs              |  2 +-
 .../Impl/Deployment/PeerLoadingExtensions.cs    |  4 +-
 .../Impl/Deployment/PeerLoadingObjectHolder.cs  |  2 +-
 .../Impl/Messaging/MessageListenerHolder.cs     |  2 +-
 17 files changed, 68 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
index 4237eda..3ec1e8c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
@@ -1302,8 +1302,6 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             HandleOuter newOuter = outerObj.Deserialize<HandleOuter>();
 
-            Assert.IsFalse(newOuter == newOuter.Inner.Outer);
-            Assert.IsFalse(newOuter == newOuter.Inner.RawOuter);
             Assert.IsFalse(newOuter == newOuter.RawInner.RawOuter);
             Assert.IsFalse(newOuter == newOuter.RawInner.RawOuter);
 
@@ -1313,7 +1311,6 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.IsTrue(newOuter.RawInner.Outer == newOuter.RawInner.RawOuter);
 
             Assert.IsTrue(newOuter.Inner == newOuter.Inner.Outer.Inner);
-            Assert.IsTrue(newOuter.Inner == newOuter.Inner.Outer.RawInner);
             Assert.IsTrue(newOuter.RawInner == newOuter.RawInner.Outer.Inner);
             Assert.IsTrue(newOuter.RawInner == newOuter.RawInner.Outer.RawInner);
         }
@@ -2341,7 +2338,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
                 writer.WriteString("before", Before);
 
-                writer0.WithDetach(w => w.WriteObject("inner", Inner));
+                writer0.WriteObject("inner", Inner);
                 
                 writer.WriteString("after", After);
 
@@ -2349,7 +2346,7 @@ namespace Apache.Ignite.Core.Tests.Binary
 
                 rawWriter.WriteString(RawBefore);
 
-                writer0.WithDetach(w => w.WriteObject(RawInner));
+                writer0.WriteObjectDetached(RawInner);
 
                 rawWriter.WriteString(RawAfter);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/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 9e44720..27ac3f4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -597,27 +597,52 @@ namespace Apache.Ignite.Core.Tests.Cache
         }
 
         [Test]
-        public void TestPutAll()
+        public void TestPutAll([Values(true, false)] bool async)
         {
             var cache = Cache();
 
+            if (async)
+            {
+                cache = cache.WrapAsync();
+            }
+
+            // Primitives.
             cache.PutAll(new Dictionary<int, int> { { 1, 1 }, { 2, 2 }, { 3, 3 } });
 
             Assert.AreEqual(1, cache.Get(1));
             Assert.AreEqual(2, cache.Get(2));
             Assert.AreEqual(3, cache.Get(3));
-        }
 
-        [Test]
-        public void TestPutAllAsync()
-        {
-            var cache = Cache().WrapAsync();
+            // Objects.
+            var cache2 = Cache<int, Container>();
 
-            cache.PutAll(new Dictionary<int, int> { { 1, 1 }, { 2, 2 }, { 3, 3 } });
+            if (async)
+            {
+                cache2 = cache2.WrapAsync();
+            }
 
-            Assert.AreEqual(1, cache.Get(1));
-            Assert.AreEqual(2, cache.Get(2));
-            Assert.AreEqual(3, cache.Get(3));
+            var obj1 = new Container();
+            var obj2 = new Container();
+            var obj3 = new Container();
+
+            obj1.Inner = obj2;
+            obj2.Inner = obj1;
+            obj3.Inner = obj2;
+
+            cache2.PutAll(new Dictionary<int, Container>
+            {
+                {1, obj1},
+                {2, obj2},
+                {3, obj3}
+            });
+
+            var res1 = cache2[1];
+            var res2 = cache2[2];
+            var res3 = cache2[3];
+
+            Assert.AreEqual(res1, res1.Inner.Inner);
+            Assert.AreEqual(res2, res2.Inner.Inner);
+            Assert.IsNotNull(res3.Inner.Inner.Inner);
         }
 
         /// <summary>
@@ -2537,5 +2562,10 @@ namespace Apache.Ignite.Core.Tests.Cache
                 }
             }
         }
+
+        private class Container
+        {
+            public Container Inner;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
index 3f35b7b..73b4136 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
@@ -1413,11 +1413,12 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Perform action with detached semantics.
         /// </summary>
-        /// <param name="a"></param>
-        internal void WithDetach(Action<BinaryWriter> a)
+        internal void WriteObjectDetached<T>(T o)
         {
             if (_detaching)
-                a(this);
+            {
+                Write(o);
+            }
             else
             {
                 _detaching = true;
@@ -1427,7 +1428,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                 try
                 {
-                    a(this);
+                    Write(o);
                 }
                 finally
                 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs
index 3dc8a96..02b5dec 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriterExtensions.cs
@@ -174,8 +174,8 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             foreach (var pair in vals)
             {
-                writer.Write(pair.Key);
-                writer.Write(pair.Value);
+                writer.WriteObjectDetached(pair.Key);
+                writer.WriteObjectDetached(pair.Value);
 
                 cnt++;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryFilterHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryFilterHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryFilterHolder.cs
index 779e350..3377f11 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryFilterHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryFilterHolder.cs
@@ -82,7 +82,7 @@ namespace Apache.Ignite.Core.Impl.Cache
         {
             var writer0 = (BinaryWriter)writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(_pred));
+            writer0.WriteObjectDetached(_pred);
             
             writer0.WriteBoolean(_keepBinary);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorHolder.cs
index 9fc7c7d..24a142a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheEntryProcessorHolder.cs
@@ -105,8 +105,8 @@ namespace Apache.Ignite.Core.Impl.Cache
         {
             var writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(_proc));
-            writer0.WithDetach(w => w.WriteObject(_arg));
+            writer0.WriteObjectDetached(_proc);
+            writer0.WriteObjectDetached(_arg);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs
index df4c1ae..12183f8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs
@@ -137,11 +137,8 @@ namespace Apache.Ignite.Core.Impl.Cache.Store
                         {
                             lock (writer) // User-defined store can be multithreaded.
                             {
-                                writer.WithDetach(w =>
-                                {
-                                    w.WriteObject(k);
-                                    w.WriteObject(v);
-                                });
+                                writer.WriteObjectDetached(k);
+                                writer.WriteObjectDetached(v);
 
                                 cnt++;
                             }
@@ -188,11 +185,8 @@ namespace Apache.Ignite.Core.Impl.Cache.Store
                         {
                             var entry0 = entry; // Copy modified closure.
 
-                            writer.WithDetach(w =>
-                            {
-                                w.WriteObject(entry0.Key);
-                                w.WriteObject(entry0.Value);
-                            });
+                            writer.WriteObjectDetached(entry0.Key);
+                            writer.WriteObjectDetached(entry0.Value);
 
                             cnt++;
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
index a1f84da..6db1055 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
@@ -66,7 +66,7 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         {
             var writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteWithPeerDeployment(_action));
+            writer0.WriteWithPeerDeployment(_action);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
index 8350818..ecb4b98 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
@@ -68,8 +68,8 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         {
             BinaryWriter writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteWithPeerDeployment(_clo));
-            writer0.WithDetach(w => w.WriteWithPeerDeployment(_arg));
+            writer0.WriteWithPeerDeployment(_clo);
+            writer0.WriteWithPeerDeployment(_arg);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
index abdf448..9652794 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
@@ -62,7 +62,7 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         {
             var writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(_clo));
+            writer0.WriteObjectDetached(_clo);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
index 4132347..44f2880 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeFunc.cs
@@ -79,7 +79,7 @@ namespace Apache.Ignite.Core.Impl.Compute
         {
             var writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteWithPeerDeployment(_func));
+            writer0.WriteWithPeerDeployment(_func);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
index 56e3708..57e234c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJob.cs
@@ -109,7 +109,7 @@ namespace Apache.Ignite.Core.Impl.Compute
         {
             var writer0 = (BinaryWriter)writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteWithPeerDeployment(Job));
+            writer0.WriteWithPeerDeployment(Job);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
index 120d074..5c009ac 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
@@ -223,7 +223,7 @@ namespace Apache.Ignite.Core.Impl.Compute
         {
             BinaryWriter writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(_job));
+            writer0.WriteObjectDetached(_job);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
index 4e5f523..3d59245 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeOutFunc.cs
@@ -83,7 +83,7 @@ namespace Apache.Ignite.Core.Impl.Compute
         {
             var writer0 = (BinaryWriter)writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteWithPeerDeployment(_func));
+            writer0.WriteWithPeerDeployment(_func);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingExtensions.cs
index 035a041..614f26c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingExtensions.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingExtensions.cs
@@ -40,7 +40,7 @@ namespace Apache.Ignite.Core.Impl.Deployment
                 try
                 {
                     writer.WrapperFunc = WrapperFunc;
-                    writer.WriteObject(o);
+                    writer.WriteObjectDetached(o);
                 }
                 finally
                 {
@@ -49,7 +49,7 @@ namespace Apache.Ignite.Core.Impl.Deployment
             }
             else
             {
-                writer.WriteObject(o);
+                writer.WriteObjectDetached(o);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
index 584750a..9457538 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
@@ -84,7 +84,7 @@ namespace Apache.Ignite.Core.Impl.Deployment
 
             writer0.WriteGuid(writer0.Marshaller.Ignite.GetLocalNode().Id);
             writer0.WriteString(_object.GetType().AssemblyQualifiedName);
-            writer0.WithDetach(w => w.WriteObject(_object));
+            writer0.WriteObjectDetached(_object);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f104976f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
index 8e794d1..6e4e9f7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
@@ -148,7 +148,7 @@ namespace Apache.Ignite.Core.Impl.Messaging
         {
             var writer0 = (BinaryWriter)writer.GetRawWriter();
 
-            writer0.WithDetach(w => w.WriteObject(Filter));
+            writer0.WriteObjectDetached(Filter);
         }
 
         /// <summary>


[4/7] ignite git commit: IGNITE-5924 .NET: Decouple Marshaller from Ignite

Posted by pt...@apache.org.
IGNITE-5924 .NET: Decouple Marshaller from Ignite

This closes #2559


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

Branch: refs/heads/ignite-5896
Commit: 8629b50d6ff490fa2a13f829f9dc35661a3fdbeb
Parents: f104976
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Aug 31 14:06:47 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Aug 31 14:06:47 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.csproj                   |  1 +
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs | 10 +--
 .../Impl/Binary/Marshaller.cs                   |  4 +-
 .../Impl/Cache/CacheAffinityImpl.cs             |  2 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  2 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  2 +-
 .../Impl/Compute/Closure/ComputeActionJob.cs    |  2 +-
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |  2 +-
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |  2 +-
 .../Closure/ComputeReducingClosureTask.cs       |  2 +-
 .../Compute/Closure/IComputeResourceInjector.cs |  2 +-
 .../Impl/Compute/ComputeJobHolder.cs            | 19 ++----
 .../Impl/Deployment/PeerAssemblyResolver.cs     | 13 ++--
 .../Impl/Deployment/PeerLoadingObjectHolder.cs  |  2 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  8 +--
 .../Apache.Ignite.Core/Impl/IIgniteInternal.cs  | 69 ++++++++++++++++++++
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    | 13 ++--
 .../Impl/Messaging/MessageListenerHolder.cs     |  2 +-
 .../Impl/PlatformTargetAdapter.cs               | 13 +---
 .../Impl/Resource/ResourceProcessor.cs          |  2 +-
 .../Impl/Resource/ResourceTypeDescriptor.cs     |  2 +-
 21 files changed, 116 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/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 c444ed0..ee301be 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -99,6 +99,7 @@
     <Compile Include="Cache\IMemoryMetrics.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
     <Compile Include="Configuration\Package-Info.cs" />
+    <Compile Include="Impl\IIgniteInternal.cs" />
     <Compile Include="Impl\IPlatformTargetInternal.cs" />
     <Compile Include="Impl\PersistentStore\PersistentStoreMetrics.cs" />
     <Compile Include="Impl\PlatformDisposableTargetAdapter.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
index 1d699c2..2f66f66 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
@@ -59,7 +59,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// Schema.
         /// </returns>
         public static BinaryObjectSchemaField[] ReadSchema(IBinaryStream stream, int position, BinaryObjectHeader hdr, 
-            BinaryObjectSchema schema, Ignite ignite)
+            BinaryObjectSchema schema, IIgniteInternal ignite)
         {
             Debug.Assert(stream != null);
             Debug.Assert(schema != null);
@@ -252,7 +252,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Gets the field ids.
         /// </summary>
-        private static int[] GetFieldIds(BinaryObjectHeader hdr, Ignite ignite)
+        private static int[] GetFieldIds(BinaryObjectHeader hdr, IIgniteInternal ignite)
         {
             Debug.Assert(hdr.TypeId != BinaryUtils.TypeUnregistered);
 
@@ -268,13 +268,15 @@ namespace Apache.Ignite.Core.Impl.Binary
                 throw new BinaryObjectException("Cannot find schema for object with compact footer [" +
                                                 "typeId=" + hdr.TypeId + ", schemaId=" + hdr.SchemaId + ']');
             }
+
             return fieldIds;
         }
 
         /// <summary>
         /// Reads the schema, maintains stream position.
         /// </summary>
-        public static int[] GetFieldIds(BinaryObjectHeader hdr, Ignite ignite, IBinaryStream stream, int objectPos)
+        public static int[] GetFieldIds(BinaryObjectHeader hdr, IIgniteInternal ignite, IBinaryStream stream, 
+            int objectPos)
         {
             Debug.Assert(stream != null);
 
@@ -309,7 +311,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Gets the field ids.
         /// </summary>
-        private static int[] GetFieldIds(BinaryObjectHeader hdr, BinaryObjectSchema schema, Ignite ignite)
+        private static int[] GetFieldIds(BinaryObjectHeader hdr, BinaryObjectSchema schema, IIgniteInternal ignite)
         {
             return schema.Get(hdr.SchemaId) ?? GetFieldIds(hdr, ignite);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
index a6d5517..d78276d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
@@ -61,7 +61,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         private volatile IDictionary<int, BinaryTypeHolder> _metas = new Dictionary<int, BinaryTypeHolder>();
 
         /** */
-        private volatile Ignite _ignite;
+        private volatile IIgniteInternal _ignite;
 
         /** */
         private readonly ILogger _log;
@@ -110,7 +110,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Gets or sets the backing grid.
         /// </summary>
-        public Ignite Ignite
+        public IIgniteInternal Ignite
         {
             get { return _ignite; }
             set

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs
index a2bba29..f8a2499 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheAffinityImpl.cs
@@ -79,7 +79,7 @@ namespace Apache.Ignite.Core.Impl.Cache
         private readonly bool _keepBinary;
         
         /** Grid. */
-        private readonly Ignite _ignite;
+        private readonly IIgniteInternal _ignite;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="CacheAffinityImpl" /> class.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
index 06d01c0..64eee8d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs
@@ -45,7 +45,7 @@ namespace Apache.Ignite.Core.Impl.Cache
     internal class CacheImpl<TK, TV> : PlatformTargetAdapter, ICache<TK, TV>, ICacheInternal, ICacheLockInternal
     {
         /** Ignite instance. */
-        private readonly Ignite _ignite;
+        private readonly IIgniteInternal _ignite;
         
         /** Flag: skip store. */
         private readonly bool _flagSkipStore;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
index 678fb03..6d6756f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
@@ -146,7 +146,7 @@ namespace Apache.Ignite.Core.Impl.Cluster
         private const int OpGetServices = 34;
 
         /** Initial Ignite instance. */
-        private readonly Ignite _ignite;
+        private readonly IIgniteInternal _ignite;
         
         /** Predicate. */
         private readonly Func<IClusterNode, bool> _pred;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
index 6db1055..e44018b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeActionJob.cs
@@ -56,7 +56,7 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         }
 
         /** <inheritDoc /> */
-        public void Inject(Ignite grid)
+        public void Inject(IIgniteInternal grid)
         {
             ResourceProcessor.Inject(_action, grid);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
index ecb4b98..3c7cec8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeFuncJob.cs
@@ -58,7 +58,7 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         }
 
         /** <inheritDoc /> */
-        public void Inject(Ignite grid)
+        public void Inject(IIgniteInternal grid)
         {
             ResourceProcessor.Inject(_clo, grid);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
index 9652794..af03e3d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeOutFuncJob.cs
@@ -52,7 +52,7 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         }
 
         /** <inheritDoc /> */
-        public void Inject(Ignite grid)
+        public void Inject(IIgniteInternal grid)
         {
             ResourceProcessor.Inject(_clo, grid);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeReducingClosureTask.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeReducingClosureTask.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeReducingClosureTask.cs
index d1dab05..4c81c9a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeReducingClosureTask.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/ComputeReducingClosureTask.cs
@@ -55,7 +55,7 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         }
 
         /** <inheritDoc /> */
-        public void Inject(Ignite grid)
+        public void Inject(IIgniteInternal grid)
         {
             ResourceProcessor.Inject(_rdc, grid);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/IComputeResourceInjector.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/IComputeResourceInjector.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/IComputeResourceInjector.cs
index 8d3e8d7..7ef3d92 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/IComputeResourceInjector.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/Closure/IComputeResourceInjector.cs
@@ -26,6 +26,6 @@ namespace Apache.Ignite.Core.Impl.Compute.Closure
         /// Inject resources.
         /// </summary>
         /// <param name="grid">Grid.</param>
-        void Inject(Ignite grid);
+        void Inject(IIgniteInternal grid);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
index 5c009ac..945f2ab 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
@@ -24,7 +24,6 @@ namespace Apache.Ignite.Core.Impl.Compute
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
-    using Apache.Ignite.Core.Impl.Cluster;
     using Apache.Ignite.Core.Impl.Compute.Closure;
     using Apache.Ignite.Core.Impl.Deployment;
     using Apache.Ignite.Core.Impl.Memory;
@@ -39,7 +38,7 @@ namespace Apache.Ignite.Core.Impl.Compute
         private readonly IComputeJob _job;
         
         /** Owning grid. */
-        private readonly Ignite _ignite;
+        private readonly IIgniteInternal _ignite;
 
         /** Result (set for local jobs only). */
         private volatile ComputeJobResultImpl _jobRes;
@@ -62,7 +61,7 @@ namespace Apache.Ignite.Core.Impl.Compute
         /// </summary>
         /// <param name="grid">Grid.</param>
         /// <param name="job">Job.</param>
-        public ComputeJobHolder(Ignite grid, IComputeJob job)
+        public ComputeJobHolder(IIgniteInternal grid, IComputeJob job)
         {
             Debug.Assert(grid != null);
             Debug.Assert(job != null);
@@ -87,7 +86,7 @@ namespace Apache.Ignite.Core.Impl.Compute
                 success ? null : new IgniteException("Compute job has failed on local node, " +
                                                      "examine InnerException for details.", (Exception) res), 
                 _job, 
-                _ignite.GetLocalNode().Id, 
+                _ignite.GetCluster().GetLocalNode().Id, 
                 cancel
             );
         }
@@ -109,9 +108,7 @@ namespace Apache.Ignite.Core.Impl.Compute
             }
 
             // 2. Try writing result to the stream.
-            ClusterGroupImpl prj = _ignite.ClusterGroup;
-
-            BinaryWriter writer = prj.Marshaller.StartMarshal(stream);
+            var writer = _ignite.Marshaller.StartMarshal(stream);
 
             try
             {
@@ -121,7 +118,7 @@ namespace Apache.Ignite.Core.Impl.Compute
             finally
             {
                 // 4. Process metadata.
-                prj.FinishMarshal(writer);
+                _ignite.Marshaller.FinishMarshal(writer);
             }
         }
 
@@ -142,9 +139,7 @@ namespace Apache.Ignite.Core.Impl.Compute
             Justification = "User job can throw any exception")]
         internal bool Serialize(IBinaryStream stream)
         {
-            ClusterGroupImpl prj = _ignite.ClusterGroup;
-
-            BinaryWriter writer = prj.Marshaller.StartMarshal(stream);
+            BinaryWriter writer = _ignite.Marshaller.StartMarshal(stream);
 
             try
             {
@@ -162,7 +157,7 @@ namespace Apache.Ignite.Core.Impl.Compute
             finally
             {
                 // 4. Process metadata.
-                prj.FinishMarshal(writer);
+                _ignite.Marshaller.FinishMarshal(writer);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerAssemblyResolver.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerAssemblyResolver.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerAssemblyResolver.cs
index b81bbbc..e22e325 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerAssemblyResolver.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerAssemblyResolver.cs
@@ -39,7 +39,7 @@ namespace Apache.Ignite.Core.Impl.Deployment
         /// <summary>
         /// Initializes a new instance of the <see cref="PeerAssemblyResolver"/> class.
         /// </summary>
-        public PeerAssemblyResolver(Ignite ignite, Guid originNodeId)
+        public PeerAssemblyResolver(IIgniteInternal ignite, Guid originNodeId)
         {
             Debug.Assert(ignite != null);
 
@@ -59,7 +59,7 @@ namespace Apache.Ignite.Core.Impl.Deployment
         /// <summary>
         /// Gets an instance of <see cref="PeerAssemblyResolver"/> when peer loading is enabled; otherwise null.
         /// </summary>
-        public static PeerAssemblyResolver GetInstance(Ignite ignite, Guid originNodeId)
+        public static PeerAssemblyResolver GetInstance(IIgniteInternal ignite, Guid originNodeId)
         {
             if (ignite == null || ignite.Configuration.PeerAssemblyLoadingMode == PeerAssemblyLoadingMode.Disabled)
             {
@@ -78,7 +78,7 @@ namespace Apache.Ignite.Core.Impl.Deployment
         /// <returns>
         /// Resulting type or null.
         /// </returns>
-        public static Type LoadAssemblyAndGetType(string typeName, Ignite ignite, Guid originNodeId)
+        public static Type LoadAssemblyAndGetType(string typeName, IIgniteInternal ignite, Guid originNodeId)
         {
             Debug.Assert(!string.IsNullOrEmpty(typeName));
 
@@ -102,7 +102,7 @@ namespace Apache.Ignite.Core.Impl.Deployment
         /// <summary>
         /// Gets the assembly.
         /// </summary>
-        private static Assembly GetAssembly(Ignite ignite, string assemblyName, Guid originNodeId)
+        private static Assembly GetAssembly(IIgniteInternal ignite, string assemblyName, Guid originNodeId)
         {
             return LoadedAssembliesResolver.Instance.GetAssembly(assemblyName)
                    ?? AssemblyLoader.GetAssembly(assemblyName)
@@ -112,7 +112,7 @@ namespace Apache.Ignite.Core.Impl.Deployment
         /// <summary>
         /// Loads the assembly.
         /// </summary>
-        private static Assembly LoadAssembly(Ignite ignite, string assemblyName, Guid originNodeId)
+        private static Assembly LoadAssembly(IIgniteInternal ignite, string assemblyName, Guid originNodeId)
         {
             var res = RequestAssembly(assemblyName, ignite, originNodeId);
 
@@ -132,7 +132,8 @@ namespace Apache.Ignite.Core.Impl.Deployment
         /// Successful result or null.
         /// </returns>
         /// <exception cref="IgniteException"></exception>
-        private static AssemblyRequestResult RequestAssembly(string assemblyName, Ignite ignite, Guid originNodeId)
+        private static AssemblyRequestResult RequestAssembly(string assemblyName, IIgniteInternal ignite, 
+            Guid originNodeId)
         {
             Debug.Assert(assemblyName != null);
             Debug.Assert(ignite != null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
index 9457538..c3546bc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Deployment/PeerLoadingObjectHolder.cs
@@ -82,7 +82,7 @@ namespace Apache.Ignite.Core.Impl.Deployment
         {
             var writer0 = (BinaryWriter) writer.GetRawWriter();
 
-            writer0.WriteGuid(writer0.Marshaller.Ignite.GetLocalNode().Id);
+            writer0.WriteGuid(writer0.Marshaller.Ignite.GetCluster().GetLocalNode().Id);
             writer0.WriteString(_object.GetType().AssemblyQualifiedName);
             writer0.WriteObjectDetached(_object);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
index ebfcc28..c0c261b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
@@ -118,7 +118,7 @@ namespace Apache.Ignite.Core.Impl
         /// <param name="reader">Error data reader.</param>
         /// <param name="innerException">Inner exception.</param>
         /// <returns>Exception.</returns>
-        public static Exception GetException(Ignite ignite, string clsName, string msg, string stackTrace,
+        public static Exception GetException(IIgniteInternal ignite, string clsName, string msg, string stackTrace,
             BinaryReader reader = null, Exception innerException = null)
         {
             // Set JavaException as immediate inner.
@@ -153,7 +153,7 @@ namespace Apache.Ignite.Core.Impl
                 return ProcessCachePartialUpdateException(ignite, msg, stackTrace, reader);
 
             // Predefined mapping not found - check plugins.
-            if (ignite != null)
+            if (ignite != null && ignite.PluginProcessor != null)
             {
                 ctor = ignite.PluginProcessor.GetExceptionMapping(clsName);
 
@@ -177,8 +177,8 @@ namespace Apache.Ignite.Core.Impl
         /// <param name="reader">Reader.</param>
         /// <returns>CachePartialUpdateException.</returns>
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]
-        private static Exception ProcessCachePartialUpdateException(Ignite ignite, string msg, string stackTrace,
-            BinaryReader reader)
+        private static Exception ProcessCachePartialUpdateException(IIgniteInternal ignite, string msg, 
+            string stackTrace, BinaryReader reader)
         {
             if (reader == null)
                 return new CachePartialUpdateException(msg, new IgniteException("Failed keys are not available."));

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IIgniteInternal.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IIgniteInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IIgniteInternal.cs
new file mode 100644
index 0000000..789633b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IIgniteInternal.cs
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl
+{
+    using System;
+    using Apache.Ignite.Core.Datastream;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Cluster;
+    using Apache.Ignite.Core.Impl.Handle;
+    using Apache.Ignite.Core.Impl.Plugin;
+
+    /// <summary>
+    /// Internal Ignite interface.
+    /// </summary>
+    internal interface IIgniteInternal : IIgnite
+    {
+        /// <summary>
+        /// Gets the binary processor.
+        /// </summary>
+        BinaryProcessor BinaryProcessor { get; }
+
+        /// <summary>
+        /// Configuration.
+        /// </summary>
+        IgniteConfiguration Configuration { get; }
+
+        /// <summary>
+        /// Handle registry.
+        /// </summary>
+        HandleRegistry HandleRegistry { get; }
+
+        /// <summary>
+        /// Gets the node from cache.
+        /// </summary>
+        /// <param name="id">Node id.</param>
+        /// <returns>Cached node.</returns>
+        ClusterNodeImpl GetNode(Guid? id);
+
+        /// <summary>
+        /// Gets the marshaller.
+        /// </summary>
+        Marshaller Marshaller { get; }
+
+        /// <summary>
+        /// Gets the plugin processor.
+        /// </summary>
+        PluginProcessor PluginProcessor { get; }
+
+        /// <summary>
+        /// Gets the data streamer.
+        /// </summary>
+        IDataStreamer<TK, TV> GetDataStreamer<TK, TV>(string cacheName, bool keepBinary);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index aae6ce7..630aa85 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -55,7 +55,7 @@ namespace Apache.Ignite.Core.Impl
     /// <summary>
     /// Native Ignite wrapper.
     /// </summary>
-    internal class Ignite : PlatformTargetAdapter, IIgnite, ICluster
+    internal class Ignite : PlatformTargetAdapter, ICluster, IIgniteInternal
     {
         /// <summary>
         /// Operation codes for PlatformProcessorImpl calls.
@@ -211,7 +211,6 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
-
         public ICluster GetCluster()
         {
             return this;
@@ -569,7 +568,7 @@ namespace Apache.Ignite.Core.Impl
         /// <summary>
         /// Gets the data streamer.
         /// </summary>
-        internal IDataStreamer<TK, TV> GetDataStreamer<TK, TV>(string cacheName, bool keepBinary)
+        public IDataStreamer<TK, TV> GetDataStreamer<TK, TV>(string cacheName, bool keepBinary)
         {
             var streamerTarget = DoOutOpObject((int) Op.GetDataStreamer, w =>
             {
@@ -797,7 +796,7 @@ namespace Apache.Ignite.Core.Impl
         /// Gets internal projection.
         /// </summary>
         /// <returns>Projection.</returns>
-        internal ClusterGroupImpl ClusterGroup
+        public ClusterGroupImpl ClusterGroup
         {
             get { return _prj; }
         }
@@ -805,7 +804,7 @@ namespace Apache.Ignite.Core.Impl
         /// <summary>
         /// Gets the binary processor.
         /// </summary>
-        internal BinaryProcessor BinaryProcessor
+        public BinaryProcessor BinaryProcessor
         {
             get { return _binaryProc; }
         }
@@ -813,7 +812,7 @@ namespace Apache.Ignite.Core.Impl
         /// <summary>
         /// Configuration.
         /// </summary>
-        internal IgniteConfiguration Configuration
+        public IgniteConfiguration Configuration
         {
             get { return _cfg; }
         }
@@ -889,7 +888,7 @@ namespace Apache.Ignite.Core.Impl
         /// <summary>
         /// Gets the plugin processor.
         /// </summary>
-        internal PluginProcessor PluginProcessor
+        public PluginProcessor PluginProcessor
         {
             get { return _pluginProcessor; }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
index 6e4e9f7..457533a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
@@ -36,7 +36,7 @@ namespace Apache.Ignite.Core.Impl.Messaging
         private readonly Func<Guid, object, bool> _invoker;
 
         /** Current Ignite instance. */
-        private readonly Ignite _ignite;
+        private readonly IIgniteInternal _ignite;
         
         /** Underlying filter. */
         private readonly object _filter;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTargetAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTargetAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTargetAdapter.cs
index 64b5f29..ec3a637 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTargetAdapter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTargetAdapter.cs
@@ -79,7 +79,7 @@ namespace Apache.Ignite.Core.Impl
         /// <summary>
         /// Unmanaged target.
         /// </summary>
-        internal IPlatformTargetInternal Target
+        public IPlatformTargetInternal Target
         {
             get { return _target; }
         }
@@ -87,7 +87,7 @@ namespace Apache.Ignite.Core.Impl
         /// <summary>
         /// Marshaller.
         /// </summary>
-        internal Marshaller Marshaller
+        public Marshaller Marshaller
         {
             get { return _marsh; }
         }
@@ -406,15 +406,6 @@ namespace Apache.Ignite.Core.Impl
         #region Miscelanneous
 
         /// <summary>
-        /// Finish marshaling.
-        /// </summary>
-        /// <param name="writer">Writer.</param>
-        internal void FinishMarshal(BinaryWriter writer)
-        {
-            _marsh.FinishMarshal(writer);
-        }
-
-        /// <summary>
         /// Unmarshal object using the given stream.
         /// </summary>
         /// <param name="stream">Stream.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceProcessor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceProcessor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceProcessor.cs
index fc677b0..6e37006 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceProcessor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceProcessor.cs
@@ -71,7 +71,7 @@ namespace Apache.Ignite.Core.Impl.Resource
         /// </summary>
         /// <param name="target">Target object.</param>
         /// <param name="grid">Grid.</param>
-        public static void Inject(object target, Ignite grid)
+        public static void Inject(object target, IIgniteInternal grid)
         {
             if (target != null) {
                 var desc = Descriptor(target.GetType());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8629b50d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceTypeDescriptor.cs
index 3a3f629..609ccfd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceTypeDescriptor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Resource/ResourceTypeDescriptor.cs
@@ -87,7 +87,7 @@ namespace Apache.Ignite.Core.Impl.Resource
         /// </summary>
         /// <param name="target">Target.</param>
         /// <param name="ignite">Grid.</param>
-        public void InjectIgnite(object target, Ignite ignite)
+        public void InjectIgnite(object target, IIgniteInternal ignite)
         {
             Inject0(target, ignite, _igniteInjectors);
         }


[6/7] ignite git commit: IGNITE-5462: JDBC thin driver: additional test for DDL/DML commands. This closes #2122.

Posted by pt...@apache.org.
IGNITE-5462: JDBC thin driver: additional test for DDL/DML commands. This closes #2122.


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

Branch: refs/heads/ignite-5896
Commit: 5738d7e7d3b1e59df155bcaa3803430a9d897cd1
Parents: fd9ab95
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Aug 31 17:23:31 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 31 17:23:31 2017 +0300

----------------------------------------------------------------------
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   4 +
 .../thin/JdbcThinComplexDmlDdlSelfTest.java     | 486 +++++++++++++++++++
 .../internal/jdbc/thin/JdbcThinStatement.java   |   2 +-
 3 files changed, 491 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5738d7e7/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index 691ddbf..d08528d 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -53,6 +53,7 @@ import org.apache.ignite.jdbc.thin.JdbcThinPreparedStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinResultSetSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSelfTest;
+import org.apache.ignite.jdbc.thin.JdbcThinComplexDmlDdlSelfTest;
 
 /**
  * JDBC driver test suite.
@@ -137,6 +138,9 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         suite.addTest(new TestSuite(JdbcThinDynamicIndexTransactionalPartitionedSelfTest.class));
         suite.addTest(new TestSuite(JdbcThinDynamicIndexTransactionalReplicatedSelfTest.class));
 
+        // New thin JDBC driver, full SQL tests
+        suite.addTest(new TestSuite(JdbcThinComplexDmlDdlSelfTest.class));
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5738d7e7/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinComplexDmlDdlSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinComplexDmlDdlSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinComplexDmlDdlSelfTest.java
new file mode 100644
index 0000000..0760107
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinComplexDmlDdlSelfTest.java
@@ -0,0 +1,486 @@
+/*
+ * 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.jdbc.thin;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgnitePredicate;
+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 org.jetbrains.annotations.NotNull;
+
+/**
+ * Base class for complex SQL tests based on JDBC driver.
+ */
+public class JdbcThinComplexDmlDdlSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Cache mode to test with. */
+    private final CacheMode cacheMode = CacheMode.PARTITIONED;
+
+    /** Cache atomicity mode to test with. */
+    private final CacheAtomicityMode atomicityMode = CacheAtomicityMode.ATOMIC;
+
+    /** Names of companies to use. */
+    private static final List<String> COMPANIES = Arrays.asList("ASF", "GNU", "BSD");
+
+    /** Cities to use. */
+    private static final List<String> CITIES = Arrays.asList("St. Petersburg", "Boston", "Berkeley", "London");
+
+    /** JDBC connection. */
+    private Connection conn;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME));
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /**
+     * @param name Cache name.
+     * @return Cache configuration.
+     * @throws Exception In case of error.
+     */
+    private CacheConfiguration cacheConfiguration(@NotNull String name) throws Exception {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setName(name);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        conn.close();
+
+        // Destroy all SQL caches after test.
+        for (String cacheName : grid(0).cacheNames()) {
+            DynamicCacheDescriptor cacheDesc = grid(0).context().cache().cacheDescriptor(cacheName);
+
+            if (cacheDesc != null && cacheDesc.sql())
+                grid(0).destroyCache0(cacheName, true);
+        }
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testCreateSelect() throws Exception {
+        GridTestUtils.assertThrows(null, new IgniteCallable<Object>() {
+            @Override public Object call() throws Exception {
+                sql(new ResultChecker(new Object[][] {}), "SELECT * from Person");
+
+                return null;
+            }
+        }, SQLException.class, "Failed to parse query: SELECT * from Person");
+
+        sql(new UpdateChecker(0),
+            "CREATE TABLE person (id int, name varchar, age int, company varchar, city varchar, " +
+                "primary key (id, name, city)) WITH \"template=" + cacheMode.name() + ",atomicity=" + atomicityMode.name()
+                + ",affinitykey=city\"");
+
+        sql(new UpdateChecker(0), "CREATE INDEX idx on person (city asc, name asc)");
+
+        sql(new UpdateChecker(0), "CREATE TABLE city (name varchar, population int, primary key (name)) WITH " +
+            "\"template=" + cacheMode.name() + ",atomicity=" + atomicityMode.name() + ",affinitykey=name\"");
+
+        sql(new UpdateChecker(3),
+            "INSERT INTO city (name, population) values(?, ?), (?, ?), (?, ?)",
+            "St. Petersburg", 6000000,
+            "Boston", 2000000,
+            "London", 8000000
+        );
+
+        sql(new ResultColumnChecker("id", "name", "age", "comp"),
+            "SELECT id, name, age, company as comp FROM person where id < 50");
+
+        for (int i = 0; i < 100; i++) {
+            sql(new UpdateChecker(1),
+                "INSERT INTO person (id, name, age, company, city) values (?, ?, ?, ?, ?)",
+                i,
+                "Person " + i,
+                20 + (i % 10),
+                COMPANIES.get(i % COMPANIES.size()),
+                CITIES.get(i % CITIES.size()));
+        }
+
+        final int[] cnt = {0};
+
+        sql(new ResultPredicateChecker(new IgnitePredicate<Object[]>() {
+            @Override public boolean apply(Object[] objs) {
+                int id = ((Integer)objs[0]);
+
+                if (id >= 50)
+                    return false;
+
+                if (20 + (id % 10) != ((Integer)objs[2]))
+                    return false;
+
+                if (!("Person " + id).equals(objs[1]))
+                    return false;
+
+                ++cnt[0];
+
+                return true;
+            }
+        }), "SELECT id, name, age FROM person where id < 50");
+
+        assert cnt[0] == 50 : "Invalid rows count";
+
+        // Berkeley is not present in City table, although 25 people have it specified as their city.
+        sql(new ResultChecker(new Object[][] {{75L}}),
+            "SELECT COUNT(*) from Person p inner join City c on p.city = c.name");
+
+        sql(new UpdateChecker(34),
+            "UPDATE Person SET company = 'New Company', age = CASE WHEN MOD(id, 2) <> 0 THEN age + 5 ELSE "
+                + "age + 1 END WHERE company = 'ASF'");
+
+        cnt[0] = 0;
+
+        sql(new ResultPredicateChecker(new IgnitePredicate<Object[]>() {
+            @Override public boolean apply(Object[] objs) {
+                int id = ((Integer)objs[0]);
+                int age = ((Integer)objs[2]);
+
+                if (id % 2 == 0) {
+                    if (age != 20 + (id % 10) + 1)
+                        return false;
+                }
+                else {
+                    if (age != 20 + (id % 10) + 5)
+                        return false;
+                }
+
+                ++cnt[0];
+
+                return true;
+            }
+        }), "SELECT * FROM person where company = 'New Company'");
+
+        assert cnt[0] == 34 : "Invalid rows count";
+
+        sql(new UpdateChecker(0), "DROP INDEX idx");
+    }
+
+    /**
+     * Run sql statement with arguments and check results.
+     *
+     * @param checker Query result's checker.
+     * @param sql SQL statement to execute.
+     * @param args Arguments.
+     *
+     * @throws SQLException On failed.
+     */
+    protected void sql(SingleStatementChecker checker, String sql, Object... args) throws SQLException {
+        Statement stmt;
+
+        if (args.length > 0) {
+            stmt = conn.prepareStatement(sql);
+
+            PreparedStatement pstmt = (PreparedStatement)stmt;
+
+            for (int i = 0; i < args.length; ++i)
+                pstmt.setObject(i + 1, args[i]);
+
+            pstmt.execute();
+        }
+        else {
+            stmt = conn.createStatement();
+
+            stmt.execute(sql);
+        }
+
+        checkResults(stmt, checker);
+    }
+
+    /**
+     * Check query results wwith provided checker.
+     *
+     * @param stmt Statement.
+     * @param checker Checker.
+     * @throws SQLException If failed.
+     */
+    private void checkResults(Statement stmt, SingleStatementChecker checker) throws SQLException {
+        ResultSet rs = stmt.getResultSet();
+
+        if (rs != null)
+            checker.check(rs);
+        else {
+            int updCnt = stmt.getUpdateCount();
+
+            assert updCnt != -1 : "Invalid results. Result set is null and update count is -1";
+
+            checker.check(updCnt);
+        }
+    }
+
+    /**
+     *
+     */
+    interface SingleStatementChecker {
+        /**
+         * Called when query produces results.
+         *
+         * @param rs Result set.
+         * @throws SQLException On error.
+         */
+        void check(ResultSet rs) throws SQLException;
+
+        /**
+         * Called when query produces any update.
+         *
+         * @param updateCount Update count.
+         */
+        void check(int updateCount);
+    }
+
+    /**
+     *
+     */
+    static class UpdateChecker implements SingleStatementChecker {
+        /** Expected update count. */
+        private final int expUpdCnt;
+
+        /**
+         * @param expUpdCnt Expected Update count.
+         */
+        UpdateChecker(int expUpdCnt) {
+            this.expUpdCnt = expUpdCnt;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void check(ResultSet rs) {
+            fail("Update results are expected. [rs=" + rs + ']');
+        }
+
+        /** {@inheritDoc} */
+        @Override public void check(int updateCount) {
+            assertEquals(expUpdCnt, updateCount);
+        }
+    }
+
+    /**
+     *
+     */
+    static class ResultChecker implements SingleStatementChecker {
+        /** Expected update count. */
+        private final Set<Row> expRs = new HashSet<>();
+
+        /**
+         * @param expRs Expected result set.
+         */
+        ResultChecker(Object[][] expRs) {
+            for (Object[] row : expRs)
+                this.expRs.add(new Row(row));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void check(ResultSet rs) throws SQLException {
+            int cols = rs.getMetaData().getColumnCount();
+
+            while (rs.next()) {
+                Object [] rowObjs = new Object[cols];
+
+                for (int i = 0; i < cols; ++i)
+                    rowObjs[i] = rs.getObject(i + 1);
+
+                Row row = new Row(rowObjs);
+
+                assert expRs.remove(row) : "Invalid row. [row=" + row + ", remainedRows="
+                    + printRemainedExpectedResult() + ']';
+            }
+
+            assert expRs.isEmpty() : "Expected results has rows that aren't contained at the result set. [remainedRows="
+                + printRemainedExpectedResult() + ']';
+        }
+
+        /** {@inheritDoc} */
+        @Override public void check(int updateCount) {
+            fail("Results set is expected. [updateCount=" + updateCount + ']');
+        }
+
+        /**
+         * @return Print remaining expected rows.
+         */
+        private String printRemainedExpectedResult() {
+            StringBuilder sb = new StringBuilder();
+
+            for (Row r : expRs)
+                sb.append('\n').append(r.toString());
+
+            return sb.toString();
+        }
+    }
+
+    /**
+     *
+     */
+    static class ResultColumnChecker extends ResultChecker {
+        /** Expected column names. */
+        private final String[] expColLabels;
+
+        /**
+         * Checker column names for rmpty results.
+         *
+         * @param expColLabels Expected column names.
+         */
+        ResultColumnChecker(String... expColLabels) {
+            super(new Object[][]{});
+
+            this.expColLabels = expColLabels;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void check(ResultSet rs) throws SQLException {
+            ResultSetMetaData meta = rs.getMetaData();
+
+            int cols = meta.getColumnCount();
+
+            assert cols == expColLabels.length : "Invalid columns count: [expected=" + expColLabels.length
+                + ", actual=" + cols + ']';
+
+            for (int i = 0; i < cols; ++i)
+                assert expColLabels[i].equalsIgnoreCase(meta.getColumnName(i + 1));
+
+            super.check(rs);
+        }
+    }
+
+    /**
+     *
+     */
+    static class ResultPredicateChecker implements SingleStatementChecker {
+        /** Row predicate. */
+        private IgnitePredicate<Object[]> rowPredicate;
+
+        /**
+         * @param rowPredicate Row predicate to check result set.
+         */
+        ResultPredicateChecker(IgnitePredicate<Object[]> rowPredicate) {
+            this.rowPredicate = rowPredicate;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void check(ResultSet rs) throws SQLException {
+            int cols = rs.getMetaData().getColumnCount();
+
+            while (rs.next()) {
+                Object [] rowObjs = new Object[cols];
+
+                for (int i = 0; i < cols; ++i)
+                    rowObjs[i] = rs.getObject(i + 1);
+
+                assert rowPredicate.apply(rowObjs) : "Invalid row. [row=" + Arrays.toString(rowObjs) + ']';
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void check(int updateCount) {
+            fail("Results set is expected. [updateCount=" + updateCount + ']');
+        }
+    }
+
+    /**
+     *
+     */
+    private static class Row {
+        /** Row. */
+        private final Object[] row;
+
+        /**
+         * @param row Data row.
+         */
+        private Row(Object[] row) {
+            this.row = row;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            Row row1 = (Row)o;
+
+            return Arrays.equals(row, row1.row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return Arrays.hashCode(row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return Arrays.toString(row);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5738d7e7/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
index 3772b83..59a9db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
@@ -128,7 +128,7 @@ public class JdbcThinStatement implements Statement {
             throw new SQLException("Failed to query Ignite.", e);
         }
         catch (IgniteCheckedException e) {
-            throw new SQLException("Failed to query Ignite.", e);
+            throw new SQLException("Failed to query Ignite [err=\"" + e.getMessage() + "\"]", e);
         }
     }