You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/09/08 00:50:13 UTC

[01/39] ignite git commit: ignite-1366 Start cache processor before query processor. Initialize topology version for GridCacheQueryRequest to do not miss messages before message handler is registered.

Repository: ignite
Updated Branches:
  refs/heads/ignite-843 aec68a4d9 -> 5625672bc


ignite-1366 Start cache processor before query processor. Initialize topology version for GridCacheQueryRequest to do not miss messages before message handler is registered.


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

Branch: refs/heads/ignite-843
Commit: 15f3edb546c9f08ed46e3baa51f41250d57b1d98
Parents: f1f6be8
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 10:30:18 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 10:30:18 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../dht/GridPartitionedGetFuture.java           |  14 +-
 .../distributed/near/GridNearGetFuture.java     |  13 ++
 .../query/GridCacheDistributedQueryFuture.java  |   5 +-
 .../query/GridCacheDistributedQueryManager.java |   9 +-
 .../cache/query/GridCacheQueryManager.java      | 177 ++++++-------------
 .../cache/query/GridCacheQueryRequest.java      |  59 ++++++-
 .../IgniteCacheNodeJoinAbstractTest.java        |  42 +++++
 8 files changed, 185 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/15f3edb5/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index ad4940a..7deede7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -876,8 +876,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             startProcessor(new GridAffinityProcessor(ctx));
             startProcessor(createComponent(GridSegmentationProcessor.class, ctx));
             startProcessor(createComponent(IgniteCacheObjectProcessor.class, ctx));
-            startProcessor(new GridQueryProcessor(ctx));
             startProcessor(new GridCacheProcessor(ctx));
+            startProcessor(new GridQueryProcessor(ctx));
             startProcessor(new GridTaskSessionProcessor(ctx));
             startProcessor(new GridJobProcessor(ctx));
             startProcessor(new GridTaskProcessor(ctx));

http://git-wip-us.apache.org/repos/asf/ignite/blob/15f3edb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 2f0de86..3ddf6d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -770,6 +770,18 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
                 if (log.isDebugEnabled())
                     log.debug("Remapping mini get future [invalidParts=" + invalidParts + ", fut=" + this + ']');
 
+                if (!canRemap) {
+                    map(F.view(keys.keySet(), new P1<KeyCacheObject>() {
+                        @Override public boolean apply(KeyCacheObject key) {
+                            return invalidParts.contains(cctx.affinity().partition(key));
+                        }
+                    }), F.t(node, keys), topVer);
+
+                    onDone(createResultMap(res.entries()));
+
+                    return;
+                }
+
                 // Need to wait for next topology version to remap.
                 IgniteInternalFuture<Long> topFut = cctx.discovery().topologyFuture(rmtTopVer.topologyVersion());
 
@@ -779,7 +791,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
                         AffinityTopologyVersion topVer = new AffinityTopologyVersion(fut.get());
 
                         // This will append new futures to compound list.
-                        map(F.view(keys.keySet(),  new P1<KeyCacheObject>() {
+                        map(F.view(keys.keySet(), new P1<KeyCacheObject>() {
                             @Override public boolean apply(KeyCacheObject key) {
                                 return invalidParts.contains(cctx.affinity().partition(key));
                             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/15f3edb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 9d2113e..a7875f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -904,6 +904,19 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                 if (log.isDebugEnabled())
                     log.debug("Remapping mini get future [invalidParts=" + invalidParts + ", fut=" + this + ']');
 
+                if (!canRemap) {
+                    map(F.view(keys.keySet(), new P1<KeyCacheObject>() {
+                        @Override public boolean apply(KeyCacheObject key) {
+                            return invalidParts.contains(cctx.affinity().partition(key));
+                        }
+                    }), F.t(node, keys), topVer);
+
+                    // It is critical to call onDone after adding futures to compound list.
+                    onDone(loadEntries(node.id(), keys.keySet(), res.entries(), savedVers, topVer));
+
+                    return;
+                }
+
                 // Need to wait for next topology version to remap.
                 IgniteInternalFuture<Long> topFut = cctx.discovery().topologyFuture(rmtTopVer.topologyVersion());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/15f3edb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
index 32a4599..1d547c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
@@ -98,7 +98,10 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
                 subgrid.clear();
             }
 
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(), reqId, fields());
+            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fields(),
+                qryMgr.queryTopologyVersion());
 
             // Process cancel query directly (without sending) for local node,
             cctx.closures().callLocalSafe(new Callable<Object>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/15f3edb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index d1fdfcf..4422952 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -566,7 +566,8 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
                 false,
                 qry.query().keepPortable(),
                 qry.query().subjectId(),
-                qry.query().taskHash());
+                qry.query().taskHash(),
+                queryTopologyVersion());
 
             addQueryFuture(req.id(), fut);
 
@@ -610,7 +611,8 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
                 all,
                 qry.keepPortable(),
                 qry.subjectId(),
-                qry.taskHash());
+                qry.taskHash(),
+                queryTopologyVersion());
 
             sendRequest(fut, req, nodes);
         }
@@ -675,7 +677,8 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
                 qry.query().includeMetadata(),
                 qry.query().keepPortable(),
                 qry.query().subjectId(),
-                qry.query().taskHash());
+                qry.query().taskHash(),
+                queryTopologyVersion());
 
             addQueryFuture(req.id(), fut);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/15f3edb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index b3f8720..2041464 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -168,6 +168,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /** */
     private boolean enabled;
 
+    /** */
+    private AffinityTopologyVersion qryTopVer;
+
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         qryProc = cctx.kernalContext().query();
@@ -182,12 +185,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 if (futs != null) {
                     for (Map.Entry<Long, GridFutureAdapter<QueryResult<K, V>>> entry : futs.entrySet()) {
-                        final Object recipient = recipient(nodeId, entry.getKey());
+                        final Object rcpt = recipient(nodeId, entry.getKey());
 
                         entry.getValue().listen(new CIX1<IgniteInternalFuture<QueryResult<K, V>>>() {
                             @Override public void applyx(IgniteInternalFuture<QueryResult<K, V>> f)
                                 throws IgniteCheckedException {
-                                f.get().closeIfNotShared(recipient);
+                                f.get().closeIfNotShared(rcpt);
                             }
                         });
                     }
@@ -197,12 +200,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 if (fieldsFuts != null) {
                     for (Map.Entry<Long, GridFutureAdapter<FieldsResult>> entry : fieldsFuts.entrySet()) {
-                        final Object recipient = recipient(nodeId, entry.getKey());
+                        final Object rcpt = recipient(nodeId, entry.getKey());
 
                         entry.getValue().listen(new CIX1<IgniteInternalFuture<FieldsResult>>() {
                             @Override public void applyx(IgniteInternalFuture<FieldsResult> f)
                                 throws IgniteCheckedException {
-                                f.get().closeIfNotShared(recipient);
+                                f.get().closeIfNotShared(rcpt);
                             }
                         });
                     }
@@ -213,6 +216,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         cctx.events().addListener(lsnr, EVT_NODE_LEFT, EVT_NODE_FAILED);
 
         enabled = GridQueryProcessor.isEnabled(cctx.config());
+
+        qryTopVer = cctx.startTopologyVersion();
+
+        if (qryTopVer == null)
+            qryTopVer = new AffinityTopologyVersion(cctx.localNode().order(), 0);
     }
 
     /**
@@ -281,16 +289,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * Rebuilds all search indexes of given value type.
      *
-     * @param valType Value type.
-     * @return Future that will be completed when rebuilding of all indexes is finished.
-     */
-    public IgniteInternalFuture<?> rebuildIndexes(Class<?> valType) {
-        return rebuildIndexes(valType.getName());
-    }
-
-    /**
-     * Rebuilds all search indexes of given value type.
-     *
      * @param typeName Value type name.
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
@@ -307,23 +305,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     }
 
     /**
-     * Rebuilds all search indexes of all types.
-     *
-     * @return Future that will be completed when rebuilding of all indexes is finished.
-     */
-    public IgniteInternalFuture<?> rebuildAllIndexes() {
-        if (!enterBusy())
-            throw new IllegalStateException("Failed to rebuild indexes (grid is stopping).");
-
-        try {
-            return qryProc.rebuildAllIndexes();
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /**
      * Marks this request as canceled.
      *
      * @param reqId Request id.
@@ -531,12 +512,13 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param loc Local query or not.
      * @param subjId Security subject ID.
      * @param taskName Task name.
-     * @param recipient ID of the recipient.
+     * @param rcpt ID of the recipient.
      * @return Collection of found keys.
      * @throws IgniteCheckedException In case of error.
      */
+    @SuppressWarnings("unchecked")
     private QueryResult<K, V> executeQuery(GridCacheQueryAdapter<?> qry,
-        @Nullable Object[] args, boolean loc, @Nullable UUID subjId, @Nullable String taskName, Object recipient)
+        @Nullable Object[] args, boolean loc, @Nullable UUID subjId, @Nullable String taskName, Object rcpt)
         throws IgniteCheckedException {
         if (qry.type() == null) {
             assert !loc;
@@ -555,16 +537,16 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
             res = (QueryResult<K, V>)qryResCache.get(resKey);
 
-            if (res != null && res.addRecipient(recipient))
+            if (res != null && res.addRecipient(rcpt))
                 return res;
 
-            res = new QueryResult<>(qry.type(), recipient);
+            res = new QueryResult<>(qry.type(), rcpt);
 
             if (qryResCache.putIfAbsent(resKey, res) != null)
                 resKey = null;
         }
         else
-            res = new QueryResult<>(qry.type(), recipient);
+            res = new QueryResult<>(qry.type(), rcpt);
 
         GridCloseableIterator<IgniteBiTuple<K, V>> iter;
 
@@ -667,12 +649,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param loc Local query or not.
      * @param subjId Security subject ID.
      * @param taskName Task name.
-     * @param recipient ID of the recipient.
+     * @param rcpt ID of the recipient.
      * @return Collection of found keys.
      * @throws IgniteCheckedException In case of error.
      */
     private FieldsResult executeFieldsQuery(GridCacheQueryAdapter<?> qry, @Nullable Object[] args,
-        boolean loc, @Nullable UUID subjId, @Nullable String taskName, Object recipient) throws IgniteCheckedException {
+        boolean loc, @Nullable UUID subjId, @Nullable String taskName, Object rcpt) throws IgniteCheckedException {
         assert qry != null;
 
         FieldsResult res;
@@ -709,10 +691,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
             res = (FieldsResult)qryResCache.get(resKey);
 
-            if (res != null && res.addRecipient(recipient))
+            if (res != null && res.addRecipient(rcpt))
                 return res; // Cached result found.
 
-            res = new FieldsResult(recipient);
+            res = new FieldsResult(rcpt);
 
             if (qryResCache.putIfAbsent(resKey, res) != null)
                 resKey = null; // Failed to cache result.
@@ -736,7 +718,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     taskName));
             }
 
-            res = new FieldsResult(recipient);
+            res = new FieldsResult(rcpt);
         }
 
         try {
@@ -1191,7 +1173,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 // If metadata needs to be returned to user and cleaned from internal fields - copy it.
                 List<GridQueryFieldMetadata> meta = qryInfo.includeMetaData() ?
-                    (res.metaData() != null ? new ArrayList<GridQueryFieldMetadata>(res.metaData()) : null) :
+                    (res.metaData() != null ? new ArrayList<>(res.metaData()) : null) :
                     res.metaData();
 
                 if (!qryInfo.includeMetaData())
@@ -1996,6 +1978,13 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     }
 
     /**
+     * @return Topology version for query requests.
+     */
+    public AffinityTopologyVersion queryTopologyVersion() {
+        return qryTopVer;
+    }
+
+    /**
      * @param qry Query.
      * @return Filter.
      */
@@ -2347,10 +2336,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         /**
          * @param type Query type.
-         * @param recipient ID of the recipient.
+         * @param rcpt ID of the recipient.
          */
-        private QueryResult(GridCacheQueryType type, Object recipient) {
-            super(recipient);
+        private QueryResult(GridCacheQueryType type, Object rcpt) {
+            super(rcpt);
 
             this.type = type;
         }
@@ -2374,10 +2363,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         private List<GridQueryFieldMetadata> meta;
 
         /**
-         * @param recipient ID of the recipient.
+         * @param rcpt ID of the recipient.
          */
-        FieldsResult(Object recipient) {
-            super(recipient);
+        FieldsResult(Object rcpt) {
+            super(rcpt);
         }
 
         /**
@@ -2674,39 +2663,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     }
 
     /**
-     *
-     */
-    private class GridCacheScanSwapEntry implements Cache.Entry<K, V> {
-        /** */
-        private final AbstractLazySwapEntry e;
-
-        /**
-         * @param e Entry.
-         */
-        private GridCacheScanSwapEntry(AbstractLazySwapEntry e) {
-            this.e = e;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public V getValue() {
-            return e.value();
-        }
-
-        /** {@inheritDoc} */
-        @Override public K getKey() {
-            return e.key();
-        }
-
-        /** {@inheritDoc} */
-        @Override public <T> T unwrap(Class<T> clazz) {
-            if (clazz.isAssignableFrom(getClass()))
-                return clazz.cast(this);
-
-            throw new IllegalArgumentException();
-        }
-    }
-
-    /**
      * Cached result.
      */
     private abstract static class CachedResult<R> extends GridFutureAdapter<IgniteSpiCloseableIterator<R>> {
@@ -2720,10 +2676,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         private final Map<Object, QueueIterator> recipients = new GridLeanMap<>(1);
 
         /**
-         * @param recipient ID of the recipient.
+         * @param rcpt ID of the recipient.
          */
-        protected CachedResult(Object recipient) {
-            boolean res = addRecipient(recipient);
+        protected CachedResult(Object rcpt) {
+            boolean res = addRecipient(rcpt);
 
             assert res;
         }
@@ -2731,17 +2687,17 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         /**
          * Close if this result does not have any other recipients.
          *
-         * @param recipient ID of the recipient.
+         * @param rcpt ID of the recipient.
          * @throws IgniteCheckedException If failed.
          */
-        public void closeIfNotShared(Object recipient) throws IgniteCheckedException {
+        public void closeIfNotShared(Object rcpt) throws IgniteCheckedException {
             assert isDone();
 
             synchronized (recipients) {
                 if (recipients.isEmpty())
                     return;
 
-                recipients.remove(recipient);
+                recipients.remove(rcpt);
 
                 if (recipients.isEmpty())
                     get().close();
@@ -2749,17 +2705,17 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /**
-         * @param recipient ID of the recipient.
+         * @param rcpt ID of the recipient.
          * @return {@code true} If the recipient successfully added.
          */
-        public boolean addRecipient(Object recipient) {
+        public boolean addRecipient(Object rcpt) {
             synchronized (recipients) {
                 if (isDone())
                     return false;
 
-                assert !recipients.containsKey(recipient) : recipient + " -> " + recipients;
+                assert !recipients.containsKey(rcpt) : rcpt + " -> " + recipients;
 
-                recipients.put(recipient, new QueueIterator(recipient));
+                recipients.put(rcpt, new QueueIterator(rcpt));
             }
 
             return true;
@@ -2798,18 +2754,18 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /**
-         * @param recipient ID of the recipient.
+         * @param rcpt ID of the recipient.
          * @throws IgniteCheckedException If failed.
          */
-        public IgniteSpiCloseableIterator<R> iterator(Object recipient) throws IgniteCheckedException {
-            assert recipient != null;
+        public IgniteSpiCloseableIterator<R> iterator(Object rcpt) throws IgniteCheckedException {
+            assert rcpt != null;
 
             IgniteSpiCloseableIterator<R> it = get();
 
             assert it != null;
 
             synchronized (recipients) {
-                return queue == null ? it : recipients.get(recipient);
+                return queue == null ? it : recipients.get(rcpt);
             }
         }
 
@@ -2825,7 +2781,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             private static final int NEXT_SIZE = 64;
 
             /** */
-            private final Object recipient;
+            private final Object rcpt;
 
             /** */
             private int pos;
@@ -2834,10 +2790,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             private Queue<R> next;
 
             /**
-             * @param recipient ID of the recipient.
+             * @param rcpt ID of the recipient.
              */
-            private QueueIterator(Object recipient) {
-                this.recipient = recipient;
+            private QueueIterator(Object rcpt) {
+                this.rcpt = rcpt;
             }
 
             /**
@@ -2850,7 +2806,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
             /** {@inheritDoc} */
             @Override public void close() throws IgniteCheckedException {
-                closeIfNotShared(recipient);
+                closeIfNotShared(rcpt);
             }
 
             /** {@inheritDoc} */
@@ -3101,25 +3057,4 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             false,
             keepPortable);
     }
-
-    /**
-     * Creates SQL fields query which will include results metadata if needed.
-     *
-     * @param qry SQL query.
-     * @param incMeta Whether to include results metadata.
-     * @param keepPortable Keep portable flag.
-     * @return Created query.
-     */
-    public CacheQuery<List<?>> createSqlFieldsQuery(String qry, boolean incMeta, boolean keepPortable) {
-        assert qry != null;
-
-        return new GridCacheQueryAdapter<>(cctx,
-            SQL_FIELDS,
-            null,
-            qry,
-            null,
-            null,
-            incMeta,
-            keepPortable);
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/15f3edb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
index c21ac66..f7ef76f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
@@ -121,6 +122,9 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
     /** Partition. */
     private int part;
 
+    /** */
+    private AffinityTopologyVersion topVer;
+
     /**
      * Required by {@link Externalizable}
      */
@@ -129,13 +133,21 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
     }
 
     /**
+     * Creates cancel query request.
+     *
+     * @param cacheId Cache ID.
      * @param id Request to cancel.
      * @param fields Fields query flag.
+     * @param topVer Topology version.
      */
-    public GridCacheQueryRequest(int cacheId, long id, boolean fields) {
+    public GridCacheQueryRequest(int cacheId,
+        long id,
+        boolean fields,
+        AffinityTopologyVersion topVer) {
         this.cacheId = cacheId;
         this.id = id;
         this.fields = fields;
+        this.topVer = topVer;
 
         cancel = true;
     }
@@ -151,6 +163,9 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
      * @param fields Fields query flag.
      * @param all Whether to load all pages.
      * @param keepPortable Whether to keep portables.
+     * @param subjId Subject ID.
+     * @param taskHash Task name hash code.
+     * @param topVer Topology version.
      */
     public GridCacheQueryRequest(
         int cacheId,
@@ -162,7 +177,8 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
         boolean all,
         boolean keepPortable,
         UUID subjId,
-        int taskHash
+        int taskHash,
+        AffinityTopologyVersion topVer
     ) {
         this.cacheId = cacheId;
         this.id = id;
@@ -174,6 +190,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
         this.keepPortable = keepPortable;
         this.subjId = subjId;
         this.taskHash = taskHash;
+        this.topVer = topVer;
     }
 
     /**
@@ -192,6 +209,10 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
      * @param incBackups {@code true} if need to include backups.
      * @param args Query arguments.
      * @param incMeta Include meta data or not.
+     * @param keepPortable Keep portable flag.
+     * @param subjId Subject ID.
+     * @param taskHash Task name hash code.
+     * @param topVer Topology version.
      */
     public GridCacheQueryRequest(
         int cacheId,
@@ -211,7 +232,8 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
         boolean incMeta,
         boolean keepPortable,
         UUID subjId,
-        int taskHash
+        int taskHash,
+        AffinityTopologyVersion topVer
     ) {
         assert type != null || fields;
         assert clause != null || (type == SCAN || type == SET || type == SPI);
@@ -235,10 +257,15 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
         this.keepPortable = keepPortable;
         this.subjId = subjId;
         this.taskHash = taskHash;
+        this.topVer = topVer;
     }
 
-    /** {@inheritDoc}
-     * @param ctx*/
+    /** {@inheritDoc} */
+    @Override public AffinityTopologyVersion topologyVersion() {
+        return topVer;
+    }
+
+    /** {@inheritDoc} */
     @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {
         super.prepareMarshal(ctx);
 
@@ -554,12 +581,18 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 writer.incrementState();
 
             case 20:
-                if (!writer.writeByteArray("transBytes", transBytes))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
             case 21:
+                if (!writer.writeByteArray("transBytes", transBytes))
+                    return false;
+
+                writer.incrementState();
+
+            case 22:
                 if (!writer.writeByte("type", type != null ? (byte)type.ordinal() : -1))
                     return false;
 
@@ -718,7 +751,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 reader.incrementState();
 
             case 20:
-                transBytes = reader.readByteArray("transBytes");
+                topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -726,6 +759,14 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 reader.incrementState();
 
             case 21:
+                transBytes = reader.readByteArray("transBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 22:
                 byte typeOrd;
 
                 typeOrd = reader.readByte("type");
@@ -749,11 +790,11 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 22;
+        return 23;
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheQueryRequest.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/15f3edb5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNodeJoinAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNodeJoinAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNodeJoinAbstractTest.java
index 58aa571..2e7f2ea 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNodeJoinAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheNodeJoinAbstractTest.java
@@ -23,6 +23,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractTest;
@@ -106,4 +107,45 @@ public abstract class IgniteCacheNodeJoinAbstractTest extends IgniteCacheAbstrac
             stopGrid(1);
         }
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testScanQuery() throws Exception {
+        final IgniteCache<Integer, Integer> cache = jcache(0);
+
+        for (int i = 0; i < 5; i++) {
+            log.info("Iteration: " + i);
+
+            final IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    startGrid(1);
+
+                    return null;
+                }
+            });
+
+            final AtomicBoolean stop = new AtomicBoolean();
+
+            GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    ScanQuery qry = new ScanQuery();
+
+                    while (!stop.get() && !fut.isDone())
+                        cache.query(qry).getAll();
+
+                    return null;
+                }
+            }, 10, "test-qry");
+
+            try {
+                fut.get(60_000);
+            }
+            finally {
+                stop.set(true);
+            }
+
+            stopGrid(1);
+        }
+    }
 }
\ No newline at end of file


[27/39] ignite git commit: ignite-gg-10720: fixing issue when IgniteNodeAttribute.MARSH is set to invalid value

Posted by ak...@apache.org.
ignite-gg-10720: fixing issue when IgniteNodeAttribute.MARSH is set to invalid value


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

Branch: refs/heads/ignite-843
Commit: 4c6b8ff5e971108c0e42e7e845c59fff10bde1d0
Parents: e35aa50
Author: Denis Magda <dm...@gridgain.com>
Authored: Mon Sep 7 09:32:51 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Mon Sep 7 09:32:51 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/internal/IgniteKernal.java     |  7 ++++---
 .../main/java/org/apache/ignite/internal/IgnitionEx.java  | 10 +++++++++-
 .../java/org/apache/ignite/plugin/PluginProvider.java     |  2 ++
 3 files changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4c6b8ff5/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 14d7c14..4209119 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -657,6 +657,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @param igfsExecSvc IGFS executor service.
      * @param restExecSvc Reset executor service.
      * @param errHnd Error handler to use for notification about startup problems.
+     * @param plugins Plugins.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @SuppressWarnings({"CatchGenericClass", "unchecked"})
@@ -669,7 +670,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ExecutorService mgmtExecSvc,
         ExecutorService igfsExecSvc,
         ExecutorService restExecSvc,
-        GridAbsClosure errHnd)
+        GridAbsClosure errHnd,
+        List<PluginProvider> plugins
+        )
         throws IgniteCheckedException
     {
         gw.compareAndSet(null, new GridKernalGatewayImpl(cfg.getGridName()));
@@ -758,8 +761,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         // Ack configuration.
         ackSpis();
 
-        List<PluginProvider> plugins = U.allPluginProviders();
-
         // Spin out SPIs & managers.
         try {
             ctx = new GridKernalContextImpl(log,

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c6b8ff5/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 02b28c5..b4c0dee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -78,6 +78,7 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.mxbean.IgnitionMXBean;
+import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.plugin.segmentation.SegmentationPolicy;
 import org.apache.ignite.resources.SpringApplicationContextResource;
 import org.apache.ignite.spi.IgniteSpi;
@@ -1509,6 +1510,11 @@ public class IgnitionEx {
 
             IgniteConfiguration myCfg = initializeConfiguration(cfg);
 
+            List<PluginProvider> plugins = U.allPluginProviders();
+
+            for (PluginProvider provider : plugins)
+                provider.initConfiguration(myCfg);
+
             // Set configuration URL, if any, into system property.
             if (startCtx.configUrl() != null)
                 System.setProperty(IGNITE_CONFIG_URL, startCtx.configUrl().toString());
@@ -1620,7 +1626,9 @@ public class IgnitionEx {
                         @Override public void apply() {
                             startLatch.countDown();
                         }
-                    });
+                    },
+                    plugins
+                );
 
                 state = STARTED;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4c6b8ff5/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java b/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
index 47acf0f..da67a35 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
@@ -57,6 +57,8 @@ public interface PluginProvider<C extends PluginConfiguration> {
      */
     public <T extends IgnitePlugin> T plugin();
 
+    public void initConfiguration(IgniteConfiguration cfg);
+
     /**
      * Registers extensions.
      *


[23/39] ignite git commit: 1.4.0-SNAPSHOT

Posted by ak...@apache.org.
1.4.0-SNAPSHOT


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

Branch: refs/heads/ignite-843
Commit: 78a4b7c2daa5b71d3c9d65e41036fbe86d623648
Parents: fc05689
Author: Ignite Teamcity <ig...@apache.org>
Authored: Fri Sep 4 17:39:01 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Fri Sep 4 17:39:01 2015 +0300

----------------------------------------------------------------------
 pom.xml | 45 ++++++++++++++++++++++-----------------------
 1 file changed, 22 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/78a4b7c2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8a143cd..d026b30 100644
--- a/pom.xml
+++ b/pom.xml
@@ -780,41 +780,40 @@
                                             setClientVersion('new.ignite.version.fixed', 'new.client.version');
                                         </script>
 
-                                        <echo message="Update ignite.version in dotnet client"/>
-                                        <echo message="${new.client.version}"/>
+                                        <echo message="Update ignite.version in dotnet client" />
+                                        <echo message="${new.client.version}" />
 
                                         <replaceregexp byline="true" encoding="UTF-8">
-                                            <regexp
-                                                pattern="(\[assembly:\s*Assembly\w*Version\w*\(&quot;)\d.\d.\d(.\d)?(&quot;\)\])"/>
-                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <regexp pattern="(\[assembly:\s*Assembly\w*Version\w*\(&quot;)\d.\d.\d(.\d)?(&quot;\)\])" />
+                                            <substitution expression="\1${new.client.version}\3" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/AssemblyInfo.cs"/>
-                                                <include name="**/AssemblyInfo.cpp"/>
+                                                <include name="**/AssemblyInfo.cs" />
+                                                <include name="**/AssemblyInfo.cpp" />
                                             </fileset>
                                         </replaceregexp>
 
-                                        <echo message="Update ignite.version in cpp client"/>
+                                        <echo message="Update ignite.version in cpp client" />
                                         <replaceregexp byline="true" encoding="UTF-8">
-                                            <regexp pattern="(AC_INIT.+\[)\d.\d.\d(.\d)?(\].+)"/>
-                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <regexp pattern="(AC_INIT.+\[)\d.\d.\d(.\d)?(\].+)" />
+                                            <substitution expression="\1${new.client.version}\3" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/configure.ac"/>
+                                                <include name="**/configure.ac" />
                                             </fileset>
                                         </replaceregexp>
 
                                         <replaceregexp byline="true" encoding="UTF-8">
-                                            <regexp pattern="(define GG_VERSION_STR_WIN &quot;)\d.\d.\d(.\d)?(&quot;)"/>
-                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <regexp pattern="(define GG_VERSION_STR_WIN &quot;)\d.\d.\d(.\d)?(&quot;)" />
+                                            <substitution expression="\1${new.client.version}\3" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/resource.h"/>
+                                                <include name="**/resource.h" />
                                             </fileset>
                                         </replaceregexp>
 
                                         <replaceregexp byline="true" encoding="UTF-16">
-                                            <regexp pattern="(Version&quot;, &quot;)\d.\d.\d.\d(&quot;)"/>
-                                            <substitution expression="\1${new.client.version}\2"/>
+                                            <regexp pattern="(Version&quot;, &quot;)\d.\d.\d.\d(&quot;)" />
+                                            <substitution expression="\1${new.client.version}\2" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/Resource.rc"/>
+                                                <include name="**/Resource.rc" />
                                             </fileset>
                                         </replaceregexp>
 
@@ -833,18 +832,18 @@
                                         </script>
 
                                         <replaceregexp byline="true" encoding="UTF-8">
-                                            <regexp pattern="(define GG_VERSION_BIN_WIN )\d,\d,\d,\d"/>
-                                            <substitution expression="\1${new.bin.version}"/>
+                                            <regexp pattern="(define GG_VERSION_BIN_WIN )\d,\d,\d,\d" />
+                                            <substitution expression="\1${new.bin.version}" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/resource.h"/>
+                                                <include name="**/resource.h" />
                                             </fileset>
                                         </replaceregexp>
 
                                         <replaceregexp byline="true" encoding="UTF-16">
-                                            <regexp pattern="(VERSION )\d,\d,\d,\d"/>
-                                            <substitution expression="\1${new.bin.version}"/>
+                                            <regexp pattern="(VERSION )\d,\d,\d,\d" />
+                                            <substitution expression="\1${new.bin.version}" />
                                             <fileset dir="${basedir}/">
-                                                <include name="**/Resource.rc"/>
+                                                <include name="**/Resource.rc" />
                                             </fileset>
                                         </replaceregexp>
                                     </target>


[09/39] ignite git commit: Fixed test.

Posted by ak...@apache.org.
Fixed test.


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

Branch: refs/heads/ignite-843
Commit: acfd99bc3949dde1439e34cfe26caf9f85f58ab5
Parents: 8a4b754
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 13:20:54 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 13:20:54 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheIncrementTransformTest.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/acfd99bc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java
index 812baa5..a3d2dde 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java
@@ -172,7 +172,7 @@ public class GridCacheIncrementTransformTest extends GridCommonAbstractTest {
                 ignite = restarts ? grids.getAndSet(idx, null) : grid(idx);
             }
 
-            IgniteCache<String, TestObject> cache = ignite.cache(null);
+            IgniteCache<String, TestObject> cache = ignite.<String, TestObject>cache(null).withNoRetries();
 
             assertNotNull(cache);
 


[24/39] ignite git commit: 1.4.0-SNAPSHOT

Posted by ak...@apache.org.
1.4.0-SNAPSHOT


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

Branch: refs/heads/ignite-843
Commit: 38baf596e5194a27c0f77ca017e54755210ca904
Parents: 78a4b7c
Author: Ignite Teamcity <ig...@apache.org>
Authored: Fri Sep 4 17:42:31 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Fri Sep 4 17:42:31 2015 +0300

----------------------------------------------------------------------
 modules/platform/src/main/cpp/common/configure.ac                | 2 +-
 .../main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs    | 4 ++--
 .../dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs   | 4 ++--
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/38baf596/modules/platform/src/main/cpp/common/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/configure.ac b/modules/platform/src/main/cpp/common/configure.ac
index 7706737..920a1b5 100644
--- a/modules/platform/src/main/cpp/common/configure.ac
+++ b/modules/platform/src/main/cpp/common/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Ignite JNI bridge for C++], [7.4.1], [dev@ignite.apache.org], [ignite-common], [ignite.apache.org])
+AC_INIT([Ignite JNI bridge for C++], [1.4.0], [dev@ignite.apache.org], [ignite-common], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/38baf596/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index 326df2a..7d2d4cc 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("97db45a8-f922-456a-a819-7b3c6e5e03ba")]
 
-[assembly: AssemblyVersion("1.4.0.0")]
-[assembly: AssemblyFileVersion("1.4.0.0")]
+[assembly: AssemblyVersion("1.4.0")]
+[assembly: AssemblyFileVersion("1.4.0")]
 
 [assembly: CLSCompliant(true)]
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/38baf596/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index 4600569..b8eb3b8 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("de8dd5cc-7c7f-4a09-80d5-7086d9416a7b")]
 
-[assembly: AssemblyVersion("1.4.1.0")]
-[assembly: AssemblyFileVersion("1.4.1.0")]
\ No newline at end of file
+[assembly: AssemblyVersion("1.4.0")]
+[assembly: AssemblyFileVersion("1.4.0")]
\ No newline at end of file


[17/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4


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

Branch: refs/heads/ignite-843
Commit: b18a844829ad7aed98d00c1b9f4e39ec2f8ed594
Parents: 83454cc 5859711
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Sep 4 14:35:02 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Sep 4 14:35:02 2015 +0300

----------------------------------------------------------------------
 .../ignite/examples/messaging/MessagingPingPongExample.java    | 6 +++++-
 .../portable/datagrid/CacheClientPortableQueryExample.java     | 3 ---
 .../examples/java8/messaging/MessagingPingPongExample.java     | 2 +-
 .../org/apache/ignite/examples/MessagingExamplesSelfTest.java  | 3 +--
 .../ignite/java8/examples/MessagingExamplesSelfTest.java       | 3 ++-
 5 files changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[33/39] ignite git commit: MInor changes in .Net.

Posted by ak...@apache.org.
MInor changes in .Net.


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

Branch: refs/heads/ignite-843
Commit: 27eb8606357368a7b01a059e896941cf88ed2914
Parents: 2b16b57
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Sep 7 16:33:53 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Sep 7 16:33:53 2015 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs    | 1 +
 .../Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs   | 1 -
 2 files changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/27eb8606/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
index 2b0277a..dd53281 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
@@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Impl.Memory
     /// <summary>
     /// Utility methods for platform memory management.
     /// </summary>
+    [CLSCompliant(false)]
     public static unsafe class PlatformMemoryUtils
     {
         #region CONSTANTS

http://git-wip-us.apache.org/repos/asf/ignite/blob/27eb8606/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
index 38a19ab..648d754 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
@@ -26,7 +26,6 @@ namespace Apache.Ignite.Core.Impl.Portable.IO
     /// <summary>
     /// Base class for managed and unmanaged data streams.
     /// </summary>
-    [CLSCompliant(false)]
     internal unsafe abstract class PortableAbstractStream : IPortableStream
     {
         /// <summary>


[30/39] ignite git commit: Fixed RAT.

Posted by ak...@apache.org.
Fixed RAT.


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

Branch: refs/heads/ignite-843
Commit: 3e09aa340e6b9d5dac5428305501e11f87af3aa5
Parents: 55027dc
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Sep 7 12:47:05 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Sep 7 12:47:05 2015 +0300

----------------------------------------------------------------------
 parent/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3e09aa34/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index eba7390..015d92f 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -748,6 +748,7 @@
                                         <exclude>src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.jar</exclude>
                                         <exclude>**/Makefile.am</exclude>
                                         <exclude>**/configure.ac</exclude>
+                                        <exclude>**/*.sln</exclude>
                                         <exclude>**/*.vcxproj</exclude>
                                         <exclude>**/*.vcxproj.filters</exclude>
                                         <exclude>**/module.def</exclude>


[28/39] ignite git commit: Revert "ignite-gg-10720: fixing issue when IgniteNodeAttribute.MARSH is set to invalid value"

Posted by ak...@apache.org.
Revert "ignite-gg-10720: fixing issue when IgniteNodeAttribute.MARSH is set to invalid value"

This reverts commit 4c6b8ff5e971108c0e42e7e845c59fff10bde1d0.


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

Branch: refs/heads/ignite-843
Commit: 55027dc77781ced5d435fae5fe803b1528eaa10c
Parents: 4c6b8ff
Author: sboikov <sb...@gridgain.com>
Authored: Mon Sep 7 10:08:50 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Sep 7 10:08:50 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/internal/IgniteKernal.java     |  7 +++----
 .../main/java/org/apache/ignite/internal/IgnitionEx.java  | 10 +---------
 .../java/org/apache/ignite/plugin/PluginProvider.java     |  2 --
 3 files changed, 4 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/55027dc7/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 4209119..14d7c14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -657,7 +657,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @param igfsExecSvc IGFS executor service.
      * @param restExecSvc Reset executor service.
      * @param errHnd Error handler to use for notification about startup problems.
-     * @param plugins Plugins.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @SuppressWarnings({"CatchGenericClass", "unchecked"})
@@ -670,9 +669,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ExecutorService mgmtExecSvc,
         ExecutorService igfsExecSvc,
         ExecutorService restExecSvc,
-        GridAbsClosure errHnd,
-        List<PluginProvider> plugins
-        )
+        GridAbsClosure errHnd)
         throws IgniteCheckedException
     {
         gw.compareAndSet(null, new GridKernalGatewayImpl(cfg.getGridName()));
@@ -761,6 +758,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         // Ack configuration.
         ackSpis();
 
+        List<PluginProvider> plugins = U.allPluginProviders();
+
         // Spin out SPIs & managers.
         try {
             ctx = new GridKernalContextImpl(log,

http://git-wip-us.apache.org/repos/asf/ignite/blob/55027dc7/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index b4c0dee..02b28c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -78,7 +78,6 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.mxbean.IgnitionMXBean;
-import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.plugin.segmentation.SegmentationPolicy;
 import org.apache.ignite.resources.SpringApplicationContextResource;
 import org.apache.ignite.spi.IgniteSpi;
@@ -1510,11 +1509,6 @@ public class IgnitionEx {
 
             IgniteConfiguration myCfg = initializeConfiguration(cfg);
 
-            List<PluginProvider> plugins = U.allPluginProviders();
-
-            for (PluginProvider provider : plugins)
-                provider.initConfiguration(myCfg);
-
             // Set configuration URL, if any, into system property.
             if (startCtx.configUrl() != null)
                 System.setProperty(IGNITE_CONFIG_URL, startCtx.configUrl().toString());
@@ -1626,9 +1620,7 @@ public class IgnitionEx {
                         @Override public void apply() {
                             startLatch.countDown();
                         }
-                    },
-                    plugins
-                );
+                    });
 
                 state = STARTED;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/55027dc7/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java b/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
index da67a35..47acf0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
@@ -57,8 +57,6 @@ public interface PluginProvider<C extends PluginConfiguration> {
      */
     public <T extends IgnitePlugin> T plugin();
 
-    public void initConfiguration(IgniteConfiguration cfg);
-
     /**
      * Registers extensions.
      *


[02/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4


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

Branch: refs/heads/ignite-843
Commit: d223a707de65daf3d0e8d49ef10f9f674840fe97
Parents: 15f3edb 28213a3
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 10:33:19 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 10:33:19 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/internal/IgniteKernal.java   | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d223a707/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------


[35/39] ignite git commit: Merge branch 'ignite-1.4'

Posted by ak...@apache.org.
Merge branch 'ignite-1.4'

Conflicts:
	parent/pom.xml


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

Branch: refs/heads/ignite-843
Commit: 99187cf8fa3499cd0e510481c4148626c7859622
Parents: 5f01d00 1fe7e95
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Sep 7 16:36:53 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Sep 7 16:36:53 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteDaemonNodeMarshallerCacheTest.java        | 10 ++++++++++
 parent/pom.xml                                            |  1 +
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/99187cf8/parent/pom.xml
----------------------------------------------------------------------
diff --cc parent/pom.xml
index 846211b,015d92f..f5a29cb
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@@ -749,23 -748,11 +749,24 @@@
                                          <exclude>src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.jar</exclude>
                                          <exclude>**/Makefile.am</exclude>
                                          <exclude>**/configure.ac</exclude>
 +                                        <exclude>**/*.pc.in</exclude>
+                                         <exclude>**/*.sln</exclude>
                                          <exclude>**/*.vcxproj</exclude>
 +                                        <exclude>**/*.vcxprojrel</exclude>
                                          <exclude>**/*.vcxproj.filters</exclude>
 +                                        <exclude>**/*.sln</exclude>
 +                                        <exclude>**/*.slnrel</exclude>
 +                                        <exclude>**/*.opensdf</exclude>
                                          <exclude>**/module.def</exclude>
                                          <exclude>**/ignite-common.pc.in</exclude>
 +                                        <exclude>**/*.csproj</exclude>
 +                                        <exclude>**/*.fxcop</exclude>
 +                                        <exclude>**/*.metaproj</exclude>
 +                                        <exclude>**/*.metaproj.tmp</exclude>
 +                                        <exclude>**/x64/Debug/**</exclude>
 +                                        <exclude>**/teamcity_boost.cpp</exclude>
 +                                        <exclude>**/teamcity_messages.h</exclude>
 +                                        <exclude>**/teamcity_messages.cpp</exclude>
                                      </excludes>
                                  </configuration>
                              </execution>


[32/39] ignite git commit: MInors.

Posted by ak...@apache.org.
MInors.


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

Branch: refs/heads/ignite-843
Commit: fb4631624f7fd124f63887bd6727f55ecda3988e
Parents: 8b214d5
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Sep 7 16:30:26 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Sep 7 16:30:26 2015 +0300

----------------------------------------------------------------------
 .../src/main/cpp/common/project/vs/common.vcxproj         | 10 +++++-----
 modules/platform/src/main/cpp/common/src/java.cpp         |  2 +-
 .../Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs |  1 +
 .../Impl/Portable/Io/PortableAbstractStream.cs            |  1 -
 4 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fb463162/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/project/vs/common.vcxproj b/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
index 478932f..ddc0019 100644
--- a/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
+++ b/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
@@ -1,5 +1,5 @@
 <?xml version="1.0" encoding="utf-8"?>
-<Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+<Project DefaultTargets="Build" ToolsVersion="12.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
   <ItemGroup Label="ProjectConfigurations">
     <ProjectConfiguration Include="Debug|Win32">
       <Configuration>Debug</Configuration>
@@ -27,26 +27,26 @@
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="Configuration">
     <ConfigurationType>DynamicLibrary</ConfigurationType>
     <UseDebugLibraries>true</UseDebugLibraries>
-    <PlatformToolset>v100</PlatformToolset>
+    <PlatformToolset>v120</PlatformToolset>
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="Configuration">
     <ConfigurationType>DynamicLibrary</ConfigurationType>
     <UseDebugLibraries>true</UseDebugLibraries>
-    <PlatformToolset>v100</PlatformToolset>
+    <PlatformToolset>v120</PlatformToolset>
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
     <ConfigurationType>DynamicLibrary</ConfigurationType>
     <UseDebugLibraries>false</UseDebugLibraries>
-    <PlatformToolset>v100</PlatformToolset>
+    <PlatformToolset>v120</PlatformToolset>
     <WholeProgramOptimization>true</WholeProgramOptimization>
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
     <ConfigurationType>DynamicLibrary</ConfigurationType>
     <UseDebugLibraries>false</UseDebugLibraries>
-    <PlatformToolset>v100</PlatformToolset>
+    <PlatformToolset>v120</PlatformToolset>
     <WholeProgramOptimization>true</WholeProgramOptimization>
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb463162/modules/platform/src/main/cpp/common/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/src/java.cpp b/modules/platform/src/main/cpp/common/src/java.cpp
index 5c5911a..bb4cc20 100644
--- a/modules/platform/src/main/cpp/common/src/java.cpp
+++ b/modules/platform/src/main/cpp/common/src/java.cpp
@@ -2060,7 +2060,7 @@ namespace ignite
             JNIEXPORT void JNICALL JniComputeJobDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong jobPtr) {
                 IGNITE_SAFE_PROC(env, envPtr, ComputeJobDestroyHandler, computeJobDestroy, jobPtr);
             }
-
+            
             JNIEXPORT void JNICALL JniContinuousQueryListenerApply(JNIEnv *env, jclass cls, jlong envPtr, jlong cbPtr, jlong memPtr) {
                 IGNITE_SAFE_PROC(env, envPtr, ContinuousQueryListenerApplyHandler, contQryLsnrApply, cbPtr, memPtr);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb463162/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
index 2b0277a..dd53281 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
@@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Impl.Memory
     /// <summary>
     /// Utility methods for platform memory management.
     /// </summary>
+    [CLSCompliant(false)]
     public static unsafe class PlatformMemoryUtils
     {
         #region CONSTANTS

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb463162/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
index 38a19ab..648d754 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Portable/Io/PortableAbstractStream.cs
@@ -26,7 +26,6 @@ namespace Apache.Ignite.Core.Impl.Portable.IO
     /// <summary>
     /// Base class for managed and unmanaged data streams.
     /// </summary>
-    [CLSCompliant(false)]
     internal unsafe abstract class PortableAbstractStream : IPortableStream
     {
         /// <summary>


[08/39] ignite git commit: Handle GridDhtInvalidPartitionException in GridDhtLocalPartition.clearAll.

Posted by ak...@apache.org.
Handle GridDhtInvalidPartitionException in GridDhtLocalPartition.clearAll.


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

Branch: refs/heads/ignite-843
Commit: 8a4b7541a050a29f3a3032554ffece1496c76145
Parents: bd8c607
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 12:15:09 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 12:15:09 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLocalPartition.java  | 51 ++++++++++++++++++--
 1 file changed, 46 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8a4b7541/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index d0e2b5b..6d22dc7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -628,22 +628,41 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
         try {
             while (it.hasNext()) {
-                GridDhtCacheEntry cached = it.next();
+                GridDhtCacheEntry cached = null;
 
                 try {
+                    cached = it.next();
+
                     if (cached.clearInternal(clearVer, swap)) {
                         map.remove(cached.key(), cached);
 
                         if (!cached.isInternal()) {
                             mapPubSize.decrement();
 
-                            if (rec)
-                                cctx.events().addEvent(cached.partition(), cached.key(), cctx.localNodeId(),
-                                    (IgniteUuid)null, null, EVT_CACHE_REBALANCE_OBJECT_UNLOADED, null, false,
-                                    cached.rawGet(), cached.hasValue(), null, null, null);
+                            if (rec) {
+                                cctx.events().addEvent(cached.partition(),
+                                    cached.key(),
+                                    cctx.localNodeId(),
+                                    (IgniteUuid)null,
+                                    null,
+                                    EVT_CACHE_REBALANCE_OBJECT_UNLOADED,
+                                    null,
+                                    false,
+                                    cached.rawGet(),
+                                    cached.hasValue(),
+                                    null,
+                                    null,
+                                    null);
+                            }
                         }
                     }
                 }
+                catch (GridDhtInvalidPartitionException e) {
+                    assert map.isEmpty() && state() == EVICTED: "Invalid error [e=" + e + ", part=" + this + ']';
+                    assert swapEmpty() : "Invalid error when swap is not cleared [e=" + e + ", part=" + this + ']';
+
+                    break; // Partition is already concurrently cleared and evicted.
+                }
                 catch (IgniteCheckedException e) {
                     U.error(log, "Failed to clear cache entry for evicted partition: " + cached, e);
                 }
@@ -655,6 +674,28 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     }
 
     /**
+     * @return {@code True} if there are no swap entries for this partition.
+     */
+    private boolean swapEmpty() {
+        GridCloseableIterator<?> it0 = null;
+
+        try {
+            it0 = cctx.swap().iterator(id);
+
+            return it0 == null || !it0.hasNext();
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to get partition swap iterator: " + this, e);
+
+            return true;
+        }
+        finally {
+            if (it0 != null)
+                U.closeQuiet(it0);
+        }
+    }
+
+    /**
      * @param it Swap iterator.
      * @return Unswapping iterator over swapped entries.
      */


[06/39] ignite git commit: Fixed GridCacheQueryRequest.

Posted by ak...@apache.org.
Fixed GridCacheQueryRequest.


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

Branch: refs/heads/ignite-843
Commit: 039c48d3fb1594deb9e1121afd06a7e107559f62
Parents: 1f6d2ff
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 11:33:34 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 11:33:34 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/query/GridCacheQueryRequest.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/039c48d3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
index f7ef76f..c9ce933 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
@@ -262,7 +262,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
 
     /** {@inheritDoc} */
     @Override public AffinityTopologyVersion topologyVersion() {
-        return topVer;
+        return topVer != null ? topVer : AffinityTopologyVersion.NONE;
     }
 
     /** {@inheritDoc} */


[12/39] ignite git commit: ignite-1361: Fixed.

Posted by ak...@apache.org.
ignite-1361: Fixed.


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

Branch: refs/heads/ignite-843
Commit: cd81fff65d6788e33a0065976ce4c3d3d1c68247
Parents: 83b6e5f
Author: ashutak <as...@gridgain.com>
Authored: Fri Sep 4 14:02:13 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Fri Sep 4 14:02:13 2015 +0300

----------------------------------------------------------------------
 .../ignite/examples/messaging/MessagingPingPongExample.java    | 6 +++++-
 .../examples/java8/messaging/MessagingPingPongExample.java     | 2 +-
 .../org/apache/ignite/examples/MessagingExamplesSelfTest.java  | 3 +--
 .../ignite/java8/examples/MessagingExamplesSelfTest.java       | 3 ++-
 4 files changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cd81fff6/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java b/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
index eca4a2c..a2ba227 100644
--- a/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/messaging/MessagingPingPongExample.java
@@ -26,6 +26,7 @@ import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.examples.ExampleNodeStartup;
 import org.apache.ignite.examples.ExamplesUtils;
 import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.resources.IgniteInstanceResource;
 
 /**
  * Demonstrates simple message exchange between local and remote nodes.
@@ -64,6 +65,9 @@ public class MessagingPingPongExample {
 
             // Set up remote player.
             ignite.message(nodeB).remoteListen(null, new IgniteBiPredicate<UUID, String>() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
                 @Override public boolean apply(UUID nodeId, String rcvMsg) {
                     System.out.println("Received message [msg=" + rcvMsg + ", sender=" + nodeId + ']');
 
@@ -116,4 +120,4 @@ public class MessagingPingPongExample {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd81fff6/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingPingPongExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingPingPongExample.java b/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingPingPongExample.java
index 9962748..133e735 100644
--- a/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingPingPongExample.java
+++ b/examples/src/main/java8/org/apache/ignite/examples/java8/messaging/MessagingPingPongExample.java
@@ -110,4 +110,4 @@ public class MessagingPingPongExample {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd81fff6/examples/src/test/java/org/apache/ignite/examples/MessagingExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/MessagingExamplesSelfTest.java b/examples/src/test/java/org/apache/ignite/examples/MessagingExamplesSelfTest.java
index a5b4329..66cddb4 100644
--- a/examples/src/test/java/org/apache/ignite/examples/MessagingExamplesSelfTest.java
+++ b/examples/src/test/java/org/apache/ignite/examples/MessagingExamplesSelfTest.java
@@ -39,7 +39,6 @@ public class MessagingExamplesSelfTest extends GridAbstractExamplesTest {
     }
 
     /**
-     * TODO IGNITE-533
      * @throws Exception If failed.
      */
     public void testMessagingPingPongExample() throws Exception {
@@ -52,4 +51,4 @@ public class MessagingExamplesSelfTest extends GridAbstractExamplesTest {
     public void testMessagingPingPongListenActorExample() throws Exception {
         MessagingPingPongListenActorExample.main(EMPTY_ARGS);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd81fff6/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java b/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
index 0948a2a..c77e4b6 100644
--- a/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
+++ b/examples/src/test/java8/org/apache/ignite/java8/examples/MessagingExamplesSelfTest.java
@@ -38,10 +38,11 @@ public class MessagingExamplesSelfTest extends GridAbstractExamplesTest {
     }
 
     /**
-     * TODO IGNITE-533
      * @throws Exception If failed.
      */
     public void testMessagingPingPongExample() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-10");
+
         MessagingPingPongExample.main(EMPTY_ARGS);
     }
 


[37/39] ignite git commit: IGNITE-1367: Moved .Net executable to Ignite.

Posted by ak...@apache.org.
IGNITE-1367: Moved .Net executable to Ignite.


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

Branch: refs/heads/ignite-843
Commit: c2d19b177c7ebde873f9eecd609aed33b4050a0c
Parents: 99187cf
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Mon Sep 7 16:45:31 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Sep 7 16:45:31 2015 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |   2 +-
 .../Impl/Common/DelegateTypeDescriptor.cs       |  17 +-
 .../Impl/Compute/ComputeImpl.cs                 |   2 +-
 .../Impl/Compute/ComputeJobHolder.cs            |   6 +
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |   2 +-
 .../Properties/AssemblyInfo.cs                  |   1 +
 .../platform/src/main/dotnet/Apache.Ignite.sln  |  10 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |  76 +++++++
 .../src/main/dotnet/Apache.Ignite/App.config    |  56 +++++
 .../Config/AppSettingsConfigurator.cs           | 113 ++++++++++
 .../Apache.Ignite/Config/ArgsConfigurator.cs    | 164 ++++++++++++++
 .../Apache.Ignite/Config/ConfigValueParser.cs   |  42 ++++
 .../Apache.Ignite/Config/IConfigurator.cs       |  34 +++
 .../main/dotnet/Apache.Ignite/IgniteRunner.cs   | 171 ++++++++++++++
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |  35 +++
 .../Apache.Ignite/Service/IgniteService.cs      | 219 ++++++++++++++++++
 .../Apache.Ignite/Service/NativeMethods.cs      |  57 +++++
 .../Apache.Ignite/Service/ServiceDescription.cs |  32 +++
 .../Apache.Ignite.Core.Tests.csproj             |   5 +-
 .../Cache/CacheForkedTest.cs                    |   1 -
 .../Forked/ForkedPortableClosureTaskTest.cs     |   3 -
 .../Compute/Forked/ForkedResourceTaskTest.cs    |   2 +-
 .../Forked/ForkedSerializableClosureTaskTest.cs |   2 +-
 .../Compute/Forked/ForkedTaskAdapterTest.cs     |   3 -
 .../Config/Apache.Ignite.exe.config.test        |  41 ++++
 .../Config/Ignite.exe.config.test               |  41 ----
 .../native-client-test-cache-portables.xml      | 226 -------------------
 .../Config/native-client-test-cache.xml         |  30 ---
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  39 ++--
 .../IgniteStartStopTest.cs                      |  28 ---
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |   2 +-
 .../Process/IgniteProcess.cs                    |  33 +--
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   2 +-
 33 files changed, 1098 insertions(+), 399 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs
index 1fa11b5..21d6931 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cluster/IClusterGroup.cs
@@ -174,7 +174,7 @@ namespace Apache.Ignite.Core.Cluster
         IClusterGroup ForYoungest();
 
         /// <summary>
-        /// Creates grid projection for nodes supporting .Net, i.e. for nodes started with Ignite.exe.
+        /// Creates grid projection for nodes supporting .Net, i.e. for nodes started with Apache.Ignite.exe.
         /// </summary>
         /// <returns>Grid projection for nodes supporting .Net.</returns>
         IClusterGroup ForDotNet();

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs
index 680228d..8d7cb3a 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateTypeDescriptor.cs
@@ -48,10 +48,7 @@ namespace Apache.Ignite.Core.Impl.Common
 
         /** */
         private readonly Func<object, object, object, bool> _cacheEntryFilter;
-
-        /** */
-        private readonly Func<object, object, object, byte, bool> _cacheDrEntryFilter;
-
+        
         /** */
         private readonly Tuple<Func<object, IMutableCacheEntryInternal, object, object>, Tuple<Type, Type>> 
             _cacheEntryProcessor;
@@ -110,17 +107,7 @@ namespace Apache.Ignite.Core.Impl.Common
         {
             return Get(type)._cacheEntryFilter;
         }
-
-        /// <summary>
-        /// Gets the <see cref="ICacheDrEntryFilter{K, V}" /> invocator.
-        /// </summary>
-        /// <param name="type">Type.</param>
-        /// <returns>Precompiled invocator delegate.</returns>
-        public static Func<object, object, object, byte, bool> GetCacheDrEntryFilter(Type type)
-        {
-            return Get(type)._cacheDrEntryFilter;
-        }
-
+        
         /// <summary>
         /// Gets the <see cref="ICacheEntryProcessor{K, V, A, R}" /> invocator.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
index 789e1c4..45c847a 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
@@ -573,7 +573,7 @@ namespace Apache.Ignite.Core.Impl.Compute
         /// <returns>Handle to the job holder</returns>
         private long WriteJob(IComputeJob job, PortableWriterImpl writer)
         {
-            var jobHolder = new ComputeJobHolder(_prj.Ignite as Ignite, job);
+            var jobHolder = new ComputeJobHolder((Ignite) _prj.Ignite, job);
 
             var jobHandle = Marshaller.Ignite.HandleRegistry.Allocate(jobHolder);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
index 9bdb5cf..4e63282 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeJobHolder.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Core.Impl.Compute
 {
     using System;
+    using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Cluster;
@@ -48,6 +49,8 @@ namespace Apache.Ignite.Core.Impl.Compute
         /// <param name="reader"></param>
         public ComputeJobHolder(IPortableReader reader)
         {
+            Debug.Assert(reader != null);
+
             var reader0 = (PortableReaderImpl) reader.RawReader();
 
             _ignite = reader0.Marshaller.Ignite;
@@ -62,6 +65,9 @@ namespace Apache.Ignite.Core.Impl.Compute
         /// <param name="job">Job.</param>
         public ComputeJobHolder(Ignite grid, IComputeJob job)
         {
+            Debug.Assert(grid != null);
+            Debug.Assert(job != null);
+
             _ignite = grid;
             _job = job;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs
index 6203b3c..0edaeab 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs
@@ -39,7 +39,7 @@ namespace Apache.Ignite.Core.Impl
         internal const string EnvIgniteHome = "IGNITE_HOME";
 
         /** Environment variable: whether to set test classpath or not. */
-        private const string EnvIgniteNativeTestClasspath = "IGNITE_NATIVE_TEST_CLASSPATH";
+        internal const string EnvIgniteNativeTestClasspath = "IGNITE_NATIVE_TEST_CLASSPATH";
         
         /** Classpath prefix. */
         private const string ClasspathPrefix = "-Djava.class.path=";

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index 5dbfce9..1876e24 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@ -37,6 +37,7 @@ using System.Runtime.InteropServices;
 [assembly: AssemblyFileVersion("1.5.0")]
 
 [assembly: CLSCompliant(true)]
+[assembly: InternalsVisibleTo("Apache.Ignite")]
 
 #if !EXCLUDE_TESTS
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite.sln
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.sln b/modules/platform/src/main/dotnet/Apache.Ignite.sln
index 11fc4ce..e9886ed 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.sln
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.sln
@@ -11,6 +11,8 @@ Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "common", "..\cpp\common\pro
 EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Core.Tests.TestDll", "..\..\test\dotnet\Apache.Ignite.Core.Tests.TestDll\Apache.Ignite.Core.Tests.TestDll.csproj", "{F4A69E2D-908E-4F0F-A794-84D508D60E5F}"
 EndProject
+Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite", "Apache.Ignite\Apache.Ignite.csproj", "{27F7F3C6-BDDE-43A9-B565-856F8395A04B}"
+EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
 		Debug|x64 = Debug|x64
@@ -51,6 +53,14 @@ Global
 		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Release|x64.Build.0 = Release|x64
 		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Release|x86.ActiveCfg = Release|x86
 		{F4A69E2D-908E-4F0F-A794-84D508D60E5F}.Release|x86.Build.0 = Release|x86
+		{27F7F3C6-BDDE-43A9-B565-856F8395A04B}.Debug|x64.ActiveCfg = Debug|x64
+		{27F7F3C6-BDDE-43A9-B565-856F8395A04B}.Debug|x64.Build.0 = Debug|x64
+		{27F7F3C6-BDDE-43A9-B565-856F8395A04B}.Debug|x86.ActiveCfg = Debug|x86
+		{27F7F3C6-BDDE-43A9-B565-856F8395A04B}.Debug|x86.Build.0 = Debug|x86
+		{27F7F3C6-BDDE-43A9-B565-856F8395A04B}.Release|x64.ActiveCfg = Release|x64
+		{27F7F3C6-BDDE-43A9-B565-856F8395A04B}.Release|x64.Build.0 = Release|x64
+		{27F7F3C6-BDDE-43A9-B565-856F8395A04B}.Release|x86.ActiveCfg = Release|x86
+		{27F7F3C6-BDDE-43A9-B565-856F8395A04B}.Release|x86.Build.0 = Release|x86
 	EndGlobalSection
 	GlobalSection(SolutionProperties) = preSolution
 		HideSolutionNode = FALSE

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/Apache.Ignite.csproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Apache.Ignite.csproj b/modules/platform/src/main/dotnet/Apache.Ignite/Apache.Ignite.csproj
new file mode 100644
index 0000000..7f6db3a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Apache.Ignite.csproj
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="12.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
+  <PropertyGroup>
+    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
+    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
+    <ProjectGuid>{27F7F3C6-BDDE-43A9-B565-856F8395A04B}</ProjectGuid>
+    <OutputType>Exe</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Apache.Ignite</RootNamespace>
+    <AssemblyName>Apache.Ignite</AssemblyName>
+    <TargetFrameworkVersion>v4.0</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x64'">
+    <PlatformTarget>x64</PlatformTarget>
+    <OutputPath>bin\x64\Debug\</OutputPath>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x64'">
+    <PlatformTarget>x64</PlatformTarget>
+    <OutputPath>bin\x64\Release\</OutputPath>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x86'">
+    <PlatformTarget>x86</PlatformTarget>
+    <OutputPath>bin\x86\Debug\</OutputPath>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x86'">
+    <PlatformTarget>x86</PlatformTarget>
+    <OutputPath>bin\x86\Release\</OutputPath>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="System" />
+    <Reference Include="System.Configuration" />
+    <Reference Include="System.Core" />
+    <Reference Include="System.ServiceProcess" />
+    <Reference Include="System.Xml.Linq" />
+    <Reference Include="System.Data.DataSetExtensions" />
+    <Reference Include="Microsoft.CSharp" />
+    <Reference Include="System.Data" />
+    <Reference Include="System.Xml" />
+  </ItemGroup>
+  <ItemGroup>
+    <Compile Include="Config\AppSettingsConfigurator.cs" />
+    <Compile Include="Config\ArgsConfigurator.cs" />
+    <Compile Include="Config\ConfigValueParser.cs" />
+    <Compile Include="Config\IConfigurator.cs" />
+    <Compile Include="IgniteRunner.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="Service\IgniteService.cs">
+      <SubType>Component</SubType>
+    </Compile>
+    <Compile Include="Service\NativeMethods.cs" />
+    <Compile Include="Service\ServiceDescription.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\Apache.Ignite.Core\Apache.Ignite.Core.csproj">
+      <Project>{4cd2f726-7e2b-46c4-a5ba-057bb82eecb6}</Project>
+      <Name>Apache.Ignite.Core</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="App.config" />
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
+  <PropertyGroup>
+    <PostBuildEvent>copy $(TargetPath) $(SolutionDir)..\..\test\dotnet\Apache.Ignite.Core.Tests\$(OutDir)
+copy $(TargetPath).config $(SolutionDir)..\..\test\dotnet\Apache.Ignite.Core.Tests\$(OutDir)</PostBuildEvent>
+  </PropertyGroup>
+  <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
+       Other similar extension points exist, see Microsoft.Common.targets.
+  <Target Name="BeforeBuild">
+  </Target>
+  <Target Name="AfterBuild">
+  </Target>
+  -->
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/App.config
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/App.config b/modules/platform/src/main/dotnet/Apache.Ignite/App.config
new file mode 100644
index 0000000..a9e8c39
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/App.config
@@ -0,0 +1,56 @@
+<?xml version="1.0" encoding="utf-8"?>
+
+<!--
+  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.
+-->
+
+
+<!--
+    Apache Ignite .Net startup application configuration file.
+-->
+
+<configuration>
+    <startup>
+        <supportedRuntime version="v4.0" sku=".NETFramework,Version=v4.0"/>
+    </startup>
+
+    <runtime>
+        <gcServer enabled="true" />
+    </runtime>
+
+    <appSettings>
+        <!-- Path to spring configuration file relative from IGNITE_HOME (if not provided "config/default-config.xml" is used) -->
+        <!-- <add key="Ignite.SpringConfigUrl" value="my-config.xml"/> -->
+
+        <!-- Absolute path to spring configuration file (if not provided "config/default-config.xml" is used) -->
+        <!-- <add key="Ignite.SpringConfigUrl" value="C:\my-dir\my-config.xml"/> -->
+
+        <!-- Path to Java library jvm.dll (if not provided JAVA_HOME environment variable is used to find jvm.dll) -->
+        <!-- <add key="Ignite.JvmDllPath" value="C:\Program Files\Java\jdk1.7.0_45\jre\bin\server\jvm.dll"/> -->
+
+        <!-- Additional classpath passed to JVM (enlist additional jar files here) -->
+        <!-- <add key="Ignite.JvmClasspath" value="c:\my-dir\my-lib1.jar;c:\my-dir\my-lib2.jar"/> -->
+
+        <!-- JVM Options passed to JVM -->
+        <!-- <add key="Ignite.JvmOption.1" value="-Xmx512m"/> -->
+        <!-- <add key="Ignite.JvmOption.2" value="-DIGNITE_QUIET=false"/> -->
+
+        <!-- Additional .Net assemblies to be loaded on startup. -->
+        <!-- <add key="Ignite.Assembly.1" value="System.Data.Linq,Culture=neutral,Version=1.0.0.0,PublicKeyToken=b77a5c561934e089"/> -->
+        <!-- <add key="Ignite.Assembly.2" value="my-assembly.dll"/> -->
+    </appSettings>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/Config/AppSettingsConfigurator.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Config/AppSettingsConfigurator.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Config/AppSettingsConfigurator.cs
new file mode 100644
index 0000000..b2e827e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Config/AppSettingsConfigurator.cs
@@ -0,0 +1,113 @@
+/*
+ * 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.Config
+{
+    using System.Collections.Generic;
+    using System.Collections.Specialized;
+    using Apache.Ignite.Core;
+
+    /// <summary>
+    /// Configurator which uses application configuration.
+    /// </summary>
+    internal class AppSettingsConfigurator : IConfigurator<NameValueCollection>
+    {
+        /** Common configuration property prefix. */
+        private static readonly string CfgPrefix = "Ignite.".ToLower();
+
+        /** Configuration property: Ignite home. */
+        private static readonly string CfgHome = "Home".ToLower();
+
+        /** Configuration property: Spring config URL. */
+        private static readonly string CfgSpringCfgUrl = "SpringConfigUrl".ToLower();
+
+        /** Configuration property: Path to JVM dll. */
+        private static readonly string CfgJvmDll = "JvmDll".ToLower();
+
+        /** Configuration property: JVM classpath. */
+        private static readonly string CfgJvmClasspath = "JvmClasspath".ToLower();
+
+        /** Configuration property: suppress warnings flag. */
+        private static readonly string CfgSuppressWarn = "SuppressWarnings".ToLower();
+
+        /** Configuration property: JVM option prefix. */
+        private static readonly string CfgJvmOptPrefix = "JvmOption".ToLower();
+
+        /** Configuration property: assembly prefix. */
+        private static readonly string CfgAssemblyPrefix = "Assembly".ToLower();
+
+        /** Configuration property: JVM min memory. */
+        private static readonly string CfgJvmMinMem = "JvmInitialMemoryMB".ToLower();
+
+        /** Configuration property: JVM max memory. */
+        private static readonly string CfgJvmMaxMem = "JvmMaxMemoryMB".ToLower();
+
+        /** <inheritDoc /> */
+        public void Configure(IgniteConfiguration cfg, NameValueCollection src)
+        {
+            var jvmOpts = new List<string>();
+            var assemblies = new List<string>();
+
+            foreach (string key in src.Keys)
+            {
+                var key0 = key.ToLower();
+
+                if (key0.StartsWith(CfgPrefix))
+                {
+                    key0 = key0.Substring(CfgPrefix.Length);
+
+                    var val = src[key];
+
+                    if (CfgHome.Equals(key0))
+                        cfg.IgniteHome = val;
+                    else if (CfgSpringCfgUrl.Equals(key0))
+                        cfg.SpringConfigUrl = val;
+                    else if (CfgJvmDll.Equals(key0))
+                        cfg.JvmDllPath = val;
+                    else if (CfgJvmClasspath.Equals(key0))
+                        cfg.JvmClasspath = val;
+                    else if (CfgSuppressWarn.Equals(key0))
+                        cfg.SuppressWarnings = val != null && bool.TrueString.ToLower().Equals(val.ToLower());
+                    else if (key0.StartsWith(CfgJvmOptPrefix))
+                        jvmOpts.Add(val);
+                    else if (key0.StartsWith(CfgAssemblyPrefix))
+                        assemblies.Add(val);
+                    else if (CfgJvmMinMem.Equals(key0))
+                        cfg.JvmInitialMemoryMb = ConfigValueParser.ParseInt(val, key);
+                    else if (CfgJvmMaxMem.Equals(key0))
+                        cfg.JvmMaxMemoryMb = ConfigValueParser.ParseInt(val, key);
+                }
+            }
+
+            if (jvmOpts.Count > 0)
+            {
+                if (cfg.JvmOptions == null)
+                    cfg.JvmOptions = jvmOpts;
+                else
+                    jvmOpts.ForEach(val => cfg.JvmOptions.Add(val));
+            }
+
+            if (assemblies.Count > 0)
+            {
+                if (cfg.Assemblies == null)
+                    cfg.Assemblies = assemblies;
+                else
+                    assemblies.ForEach(val => cfg.Assemblies.Add(val));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/Config/ArgsConfigurator.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Config/ArgsConfigurator.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Config/ArgsConfigurator.cs
new file mode 100644
index 0000000..b0651d7
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Config/ArgsConfigurator.cs
@@ -0,0 +1,164 @@
+/*
+ * 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.Config
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core;
+
+    /// <summary>
+    /// Configurator which uses arguments array.
+    /// </summary>
+    internal class ArgsConfigurator : IConfigurator<string[]>
+    {
+        /** Command line argument: Ignite home. */
+        private static readonly string CmdIgniteHome = "-IgniteHome=".ToLower();
+
+        /** Command line argument: Spring config URL. */
+        private static readonly string CmdSpringCfgUrl = "-SpringConfigUrl=".ToLower();
+
+        /** Command line argument: Path to JVM dll. */
+        private static readonly string CmdJvmDll = "-JvmDll=".ToLower();
+
+        /** Command line argument: JVM classpath. */
+        private static readonly string CmdJvmClasspath = "-JvmClasspath=".ToLower();
+
+        /** Command line argument: suppress warnings flag. */
+        private static readonly string CmdSuppressWarn = "-SuppressWarnings=".ToLower();
+
+        /** Command line argument: JVM option prefix. */
+        private static readonly string CmdJvmOpt = "-J".ToLower();
+
+        /** Command line argument: assembly. */
+        private static readonly string CmdAssembly = "-Assembly=".ToLower();
+
+        /** Command line argument: JvmInitialMemoryMB. */
+        private static readonly string CmdJvmMinMem = "-JvmInitialMemoryMB=".ToLower();
+
+        /** Command line argument: JvmMaxMemoryMB. */
+        private static readonly string CmdJvmMaxMem = "-JvmMaxMemoryMB=".ToLower();
+
+        /// <summary>
+        /// Convert configuration to arguments.
+        /// </summary>
+        /// <param name="cfg"></param>
+        /// <returns></returns>
+        internal static string[] ToArgs(IgniteConfiguration cfg)
+        {
+            var args = new List<string>();
+
+            if (cfg.IgniteHome != null)
+                args.Add(CmdIgniteHome + cfg.IgniteHome);
+
+            if (cfg.SpringConfigUrl != null)
+                args.Add(CmdSpringCfgUrl + cfg.SpringConfigUrl);
+
+            if (cfg.JvmDllPath != null)
+                args.Add(CmdJvmDll + cfg.JvmDllPath);
+
+            if (cfg.JvmClasspath != null)
+                args.Add(CmdJvmClasspath + cfg.JvmClasspath);
+            
+            if (cfg.SuppressWarnings)
+                args.Add(CmdSuppressWarn + bool.TrueString);
+
+            if (cfg.JvmOptions != null)
+            {
+                foreach (var jvmOpt in cfg.JvmOptions)
+                    args.Add(CmdJvmOpt + jvmOpt);
+            }
+
+            if (cfg.Assemblies != null)
+            {
+                foreach (var assembly in cfg.Assemblies)
+                    args.Add(CmdAssembly + assembly);
+            }
+
+            args.Add(CmdJvmMinMem + cfg.JvmInitialMemoryMb);
+            args.Add(CmdJvmMaxMem + cfg.JvmMaxMemoryMb);
+
+            return args.ToArray();
+        }
+
+        /// <summary>
+        /// Convert arguments to configuration.
+        /// </summary>
+        /// <param name="args">Arguments.</param>
+        /// <returns>Configuration.</returns>
+        internal static IgniteConfiguration FromArgs(string[] args)
+        {
+            var cfg = new IgniteConfiguration();
+
+            new ArgsConfigurator().Configure(cfg, args);
+
+            return cfg;
+        }
+
+        /** <inheritDoc /> */
+        public void Configure(IgniteConfiguration cfg, string[] src)
+        {
+            var jvmOpts = new List<string>();
+            var assemblies = new List<string>();
+
+            foreach (var arg in src)
+            {
+                var argLow = arg.ToLower();
+
+                if (argLow.StartsWith(CmdIgniteHome))
+                    cfg.IgniteHome = arg.Substring(CmdIgniteHome.Length);
+                else if (argLow.StartsWith(CmdSpringCfgUrl))
+                    cfg.SpringConfigUrl = arg.Substring(CmdSpringCfgUrl.Length);
+                else if (argLow.StartsWith(CmdJvmDll))
+                    cfg.JvmDllPath = arg.Substring(CmdJvmDll.Length);
+                else if (argLow.StartsWith(CmdJvmClasspath))
+                    cfg.JvmClasspath = arg.Substring(CmdJvmClasspath.Length);
+                else if (argLow.StartsWith(CmdSuppressWarn))
+                {
+                    var val = arg.Substring(CmdSuppressWarn.Length);
+
+                    cfg.SuppressWarnings = bool.TrueString.ToLower().Equals(val.ToLower());
+                }
+                else if (argLow.StartsWith(CmdJvmMinMem))
+                    cfg.JvmInitialMemoryMb = ConfigValueParser.ParseInt(arg.Substring(CmdJvmMinMem.Length),
+                        CmdJvmMinMem);
+                else if (argLow.StartsWith(CmdJvmMaxMem))
+                    cfg.JvmMaxMemoryMb = ConfigValueParser.ParseInt(arg.Substring(CmdJvmMaxMem.Length),
+                        CmdJvmMaxMem);
+                else if (argLow.StartsWith(CmdJvmOpt))
+                    jvmOpts.Add(arg.Substring(CmdJvmOpt.Length));
+                else if (argLow.StartsWith(CmdAssembly))
+                    assemblies.Add(arg.Substring(CmdAssembly.Length));
+            }
+
+            if (jvmOpts.Count > 0)
+            {
+                if (cfg.JvmOptions == null)
+                    cfg.JvmOptions = jvmOpts;
+                else
+                    jvmOpts.ForEach(val => cfg.JvmOptions.Add(val));
+            }
+
+            if (assemblies.Count > 0)
+            {
+                if (cfg.Assemblies == null)
+                    cfg.Assemblies = assemblies;
+                else
+                    assemblies.ForEach(val => cfg.Assemblies.Add(val));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/Config/ConfigValueParser.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Config/ConfigValueParser.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Config/ConfigValueParser.cs
new file mode 100644
index 0000000..796b8e1
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Config/ConfigValueParser.cs
@@ -0,0 +1,42 @@
+/*
+ * 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.Config
+{
+    using System;
+
+    /// <summary>
+    /// Parses Ignite config values.
+    /// </summary>
+    internal class ConfigValueParser
+    {
+        /// <summary>
+        /// Parses provided string to int. Throws a custom exception if failed.
+        /// </summary>
+        public static int ParseInt(string value, string propertyName)
+        {
+            int result;
+
+            if (int.TryParse(value, out result))
+                return result;
+
+            throw new InvalidOperationException(
+                string.Format("Failed to configure Ignite: property '{0}' has value '{1}', which is not an integer.",
+                    propertyName, value));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/Config/IConfigurator.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Config/IConfigurator.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Config/IConfigurator.cs
new file mode 100644
index 0000000..f5c0acf
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Config/IConfigurator.cs
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Config
+{
+    using Apache.Ignite.Core;
+
+    /// <summary>
+    /// Configurator which is capable of setting configuration properties taken from somewhere.
+    /// </summary>
+    internal interface IConfigurator<in T>
+    {
+        /// <summary>
+        /// Set configuration.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        /// <param name="src">Source.</param>
+        void Configure(IgniteConfiguration cfg, T src);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/IgniteRunner.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/IgniteRunner.cs b/modules/platform/src/main/dotnet/Apache.Ignite/IgniteRunner.cs
new file mode 100644
index 0000000..122994f
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/IgniteRunner.cs
@@ -0,0 +1,171 @@
+/*
+ * 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
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Configuration;
+    using System.Linq;
+    using System.ServiceProcess;
+    using Apache.Ignite.Config;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Service;
+
+    /// <summary>
+    /// Runner class.
+    /// </summary>
+    public class IgniteRunner
+    {
+        /** Help commands. */
+        private static readonly IList<string> Help = new List<string> { "/help", "-help", "--help" };
+
+        /** Argument meaning that this is service call. */
+        internal static readonly string Svc = "/service";
+
+        /** Service install command. */
+        internal static readonly string SvcInstall = "/install";
+
+        /** Service uninstall command. */
+        internal static readonly string SvcUninstall = "/uninstall";
+
+        /// <summary>
+        /// Application entry point.
+        /// </summary>
+        internal static void Main(string[] args)
+        {
+            IgniteConfiguration cfg;
+
+            bool svc = false;
+            bool install = false;
+
+            try
+            {
+                // Check for special cases.
+                if (args.Length > 0)
+                {
+                    string first = args[0].ToLower();
+
+                    if (Help.Contains(first))
+                    {
+                        PrintHelp();
+
+                        return;
+                    }
+                    
+                    if (Svc.Equals(first))
+                    {
+                        args = RemoveFirstArg(args);
+
+                        svc = true;
+                    }
+
+                    else if (SvcInstall.Equals(first))
+                    {
+                        args = RemoveFirstArg(args);
+
+                        install = true;
+                    }
+                    else if (SvcUninstall.Equals(first))
+                    {
+                        IgniteService.Uninstall();
+
+                        return;
+                    }
+                }
+
+                if (!svc)
+                {
+                    // Pick application configuration.
+                    cfg = new IgniteConfiguration();
+
+                    new AppSettingsConfigurator().Configure(cfg, ConfigurationManager.AppSettings);
+
+                    // Pick command line arguments.
+                    new ArgsConfigurator().Configure(cfg, args);
+
+                    if (install)
+                        IgniteService.DoInstall(cfg);
+                    else
+                    {
+                        Ignition.Start(cfg);
+
+                        IgniteManager.DestroyJvm();
+                    }
+
+                    return;
+                }
+            }
+            catch (Exception e)
+            {
+                Console.WriteLine("ERROR: " + e.Message);
+
+                Environment.Exit(-1);
+            }
+
+            // If we are here, then this is a service call.
+            cfg = new IgniteConfiguration();
+
+            // Use only arguments, not app.config.
+            new ArgsConfigurator().Configure(cfg, args);
+
+            ServiceBase.Run(new IgniteService(cfg));
+        }
+
+        /// <summary>
+        /// Prints help.
+        /// </summary>
+        private static void PrintHelp()
+        {
+            Console.WriteLine("Usage: Apache.Ignite.exe [/install] [/uninstall] [-options]");
+            Console.WriteLine("");
+            Console.WriteLine("\t/install [-options]    installs Ignite Windows service with provided options");
+            Console.WriteLine("\t/uninstall             uninstalls Ignite Windows service");
+            Console.WriteLine("");
+            Console.WriteLine("Options:");
+            Console.WriteLine("\t-IgniteHome            path to Ignite installation directory (if not provided IGNITE_HOME environment variable is used)");
+            Console.WriteLine("\t-springConfigUrl       path to spring configuration file (if not provided \"config/default-config.xml\" is used)");
+            Console.WriteLine("\t-jvmDllPath            path to JVM library jvm.dll (if not provided JAVA_HOME environment variable is used)");
+            Console.WriteLine("\t-jvmClasspath          classpath passed to JVM (enlist additional jar files here)");
+            Console.WriteLine("\t-suppressWarnings      wether to print warnings");
+            Console.WriteLine("\t-J<javaOption>         JVM options passed to created JVM");
+            Console.WriteLine("\t-assembly=userLib.dll  additional .Net assemblies");
+            Console.WriteLine("\t-jvmInitialMemoryMB    Initial Java heap size, in megabytes. Maps to -Xms Java parameter. Defaults to 512.");
+            Console.WriteLine("\t-jvmMaxMemoryMB        Maximum Java heap size, in megabytes. Maps to -Xmx Java parameter. Defaults to 1024.");
+            Console.WriteLine("");
+            Console.WriteLine("Examples:");
+            Console.WriteLine("\tApache.Ignite.exe -J-Xms1024m -J-Xmx1024m -springConfigUrl=C:/woer/gg-test/my-test-gg-confignative.xml");
+            Console.WriteLine("\tApache.Ignite.exe -IgniteHome=c:/apache-ignite -jvmClasspath=libs/myLib1.jar;libs/myLib2.jar");
+            Console.WriteLine("\tApache.Ignite.exe -assembly=c:/myProject/libs/lib1.dll -assembly=c:/myProject/libs/lib2.dll");
+            Console.WriteLine("\tApache.Ignite.exe -jvmInitialMemoryMB=1024 -jvmMaxMemoryMB=4096");
+            Console.WriteLine("");
+            Console.WriteLine("Note:");
+            Console.WriteLine("Command line settings have priority over Apache.Ignite.exe.config settings. JVM options and assemblies are concatenated; data from config file comes first, then data from command line.");
+        }
+
+        /// <summary>
+        /// Remove the first argument.
+        /// </summary>
+        /// <param name="args">Arguments.</param>
+        /// <returns>New arguments.</returns>
+        private static string[] RemoveFirstArg(string[] args)
+        {
+            return args.Skip(1).ToArray();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..03f7fb9
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+using System.Reflection;
+using System.Runtime.InteropServices;
+
+[assembly: AssemblyTitle("Apache.Ignite")]
+[assembly: AssemblyDescription("Apache Ignite .NET Executable")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("Apache Software Foundation")]
+[assembly: AssemblyProduct("Apache Ignite")]
+[assembly: AssemblyCopyright("Copyright ©  2015")]
+[assembly: AssemblyTrademark("")]
+[assembly: AssemblyCulture("")]
+
+[assembly: ComVisible(false)]
+
+[assembly: Guid("0f9702ec-da7d-4ce5-b4b7-73310c885355")]
+
+[assembly: AssemblyVersion("1.5.0")]
+[assembly: AssemblyFileVersion("1.5.0")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/Service/IgniteService.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Service/IgniteService.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Service/IgniteService.cs
new file mode 100644
index 0000000..a818171
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Service/IgniteService.cs
@@ -0,0 +1,219 @@
+/*
+ * 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.Service
+{
+    using System;
+    using System.ComponentModel;
+    using System.IO;
+    using System.Linq;
+    using System.Reflection;
+    using System.Runtime.InteropServices;
+    using System.ServiceProcess;
+    using System.Text;
+    using Apache.Ignite.Config;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Ignite windows service.
+    /// </summary>
+    internal class IgniteService : ServiceBase
+    {
+        /** Service name. */
+        internal static readonly string SvcName = "Apache Ignite";
+
+        /** Service display name. */
+        internal static readonly string SvcDisplayName = "Apache Ignite .NET " + 
+            Assembly.GetExecutingAssembly().GetName().Version.ToString(4);
+
+        /** Service description. */
+        internal static readonly string SvcDesc = "Apache Ignite .Net Service.";
+
+        /** Current executable name. */
+        internal static readonly string ExeName =
+            new FileInfo(new Uri(Assembly.GetExecutingAssembly().CodeBase).LocalPath).FullName;
+
+        /** Current executable fully qualified name. */
+        internal static readonly string FullExeName = Path.GetFileName(FullExeName);
+
+        /** Ignite configuration to start with. */
+        private readonly IgniteConfiguration _cfg;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        public IgniteService(IgniteConfiguration cfg)
+        {
+            AutoLog = true;
+            CanStop = true;
+            ServiceName = SvcName;
+
+            _cfg = cfg;
+        }
+
+        /** <inheritDoc /> */
+        protected override void OnStart(string[] args)
+        {
+            Ignition.Start(_cfg);
+        }
+
+        /** <inheritDoc /> */
+        protected override void OnStop()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Install service programmatically.
+        /// </summary>
+        /// <param name="cfg">Ignite configuration.</param>
+        internal static void DoInstall(IgniteConfiguration cfg)
+        {
+            // 1. Check if already defined.
+            if (ServiceController.GetServices().Any(svc => SvcName.Equals(svc.ServiceName)))
+            {
+                throw new IgniteException("Ignite service is already installed (uninstall it using \"" +
+                                          ExeName + " " + IgniteRunner.SvcUninstall + "\" first)");
+            }
+
+            // 2. Create startup arguments.
+            var args = ArgsConfigurator.ToArgs(cfg);
+
+            if (args.Length > 0)
+            {
+                Console.WriteLine("Installing \"" + SvcName + "\" service with the following startup " +
+                    "arguments:");
+
+                foreach (var arg in args)
+                    Console.WriteLine("\t" + arg);
+            }
+            else
+                Console.WriteLine("Installing \"" + SvcName + "\" service ...");
+
+            // 3. Actual installation.
+            Install0(args);
+
+            Console.WriteLine("\"" + SvcName + "\" service installed successfully.");
+        }
+
+        /// <summary>
+        /// Uninstall service programmatically.
+        /// </summary>
+        internal static void Uninstall()
+        {
+            var svc = ServiceController.GetServices().FirstOrDefault(x => SvcName == x.ServiceName);
+
+            if (svc == null)
+            {
+                Console.WriteLine("\"" + SvcName + "\" service is not installed.");
+            }
+            else if (svc.Status != ServiceControllerStatus.Stopped)
+            {
+                throw new IgniteException("Ignite service is running, please stop it first.");
+            }
+            else
+            {
+                Console.WriteLine("Uninstalling \"" + SvcName + "\" service ...");
+
+                Uninstall0();
+
+                Console.WriteLine("\"" + SvcName + "\" service uninstalled successfully.");
+            }
+        }
+
+        /// <summary>
+        /// Native service installation.
+        /// </summary>
+        /// <param name="args">Arguments.</param>
+        private static void Install0(string[] args)
+        {
+            // 1. Prepare arguments.
+            var binPath = new StringBuilder(FullExeName).Append(" ").Append(IgniteRunner.Svc);
+
+            foreach (var arg in args)
+                binPath.Append(" ").Append(arg);
+
+            // 2. Get SC manager.
+            var scMgr = OpenServiceControlManager();
+
+            // 3. Create service.
+            var svc = NativeMethods.CreateService(
+                scMgr,
+                SvcName,
+                SvcDisplayName,
+                983551, // Access constant. 
+                0x10,   // Service type SERVICE_WIN32_OWN_PROCESS.
+                0x2,    // Start type SERVICE_AUTO_START.
+                0x2,    // Error control SERVICE_ERROR_SEVERE.
+                binPath.ToString(),
+                null,
+                IntPtr.Zero,
+                null,
+                null,   // Use priviliged LocalSystem account.
+                null
+            );
+
+            if (svc == IntPtr.Zero)
+                throw new IgniteException("Failed to create the service.", new Win32Exception());
+
+            // 4. Set description.
+            var desc = new ServiceDescription {desc = Marshal.StringToHGlobalUni(SvcDesc)};
+
+
+            try 
+            {
+                if (!NativeMethods.ChangeServiceConfig2(svc, 1u, ref desc)) 
+                    throw new IgniteException("Failed to set service description.", new Win32Exception());
+            }
+            finally 
+            {
+                Marshal.FreeHGlobal(desc.desc);
+            }
+        }
+
+        /// <summary>
+        /// Native service uninstallation.
+        /// </summary>
+        private static void Uninstall0()
+        {
+            var scMgr = OpenServiceControlManager();
+
+            var svc = NativeMethods.OpenService(scMgr, SvcName, 65536);
+
+            if (svc == IntPtr.Zero)
+                throw new IgniteException("Failed to uninstall the service.", new Win32Exception());
+
+            NativeMethods.DeleteService(svc);
+        }
+
+        /// <summary>
+        /// Opens SC manager.
+        /// </summary>
+        /// <returns>SC manager pointer.</returns>
+        private static IntPtr OpenServiceControlManager()
+        {
+            var ptr = NativeMethods.OpenSCManager(null, null, 983103);
+
+            if (ptr == IntPtr.Zero)
+                throw new IgniteException("Failed to initialize Service Control manager " +
+                                          "(did you run the command as administrator?)", new Win32Exception());
+
+            return ptr;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/Service/NativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Service/NativeMethods.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Service/NativeMethods.cs
new file mode 100644
index 0000000..56ab15d
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Service/NativeMethods.cs
@@ -0,0 +1,57 @@
+/*
+ * 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.Service
+{
+    using System;
+    using System.Runtime.InteropServices;
+
+    /// <summary>
+    /// Native methods.
+    /// </summary>
+    internal class NativeMethods
+    {
+        [DllImport("advapi32.dll", CharSet = CharSet.Unicode, SetLastError = true)]
+        public static extern IntPtr OpenSCManager(string machineName, string dbName, int access);
+
+        [DllImport("advapi32.dll", CharSet = CharSet.Unicode, SetLastError = true)]
+        public static extern IntPtr CreateService(
+            IntPtr db,
+            string svcName,
+            string displayName,
+            int access,
+            int svcType,
+            int startType,
+            int errControl,
+            string binPath,
+            string loadOrderGrp,
+            IntPtr pTagId,
+            string dependencies,
+            string servicesStartName,
+            string pwd
+            );
+
+        [DllImport("advapi32.dll", CharSet = CharSet.Unicode, SetLastError = true)]
+        public static extern IntPtr OpenService(IntPtr db, string svcName, int access);
+
+        [DllImport("advapi32.dll", CharSet = CharSet.Unicode, SetLastError = true)]
+        public static extern bool DeleteService(IntPtr svc);
+
+        [DllImport("advapi32.dll", CharSet = CharSet.Unicode, SetLastError = true)]
+        public static extern bool ChangeServiceConfig2(IntPtr svc,  uint infoLevel, ref ServiceDescription desc);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/main/dotnet/Apache.Ignite/Service/ServiceDescription.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Service/ServiceDescription.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Service/ServiceDescription.cs
new file mode 100644
index 0000000..a81a737
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Service/ServiceDescription.cs
@@ -0,0 +1,32 @@
+/*
+ * 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.Service
+{
+    using System;
+    using System.Runtime.InteropServices;
+
+    /// <summary>
+    /// Service description structure.
+    /// </summary>
+    [StructLayout(LayoutKind.Sequential, CharSet = CharSet.Unicode)]
+    public struct ServiceDescription
+    {
+        /** Pointer to description. */
+        public IntPtr desc;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 62440a1..226f95f 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -190,9 +190,6 @@
     <Content Include="Config\native-client-test-cache-parallel-store.xml">
       <CopyToOutputDirectory>Always</CopyToOutputDirectory>
     </Content>
-    <Content Include="Config\native-client-test-cache-portables.xml">
-      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
-    </Content>
     <Content Include="Config\native-client-test-cache-store.xml">
       <CopyToOutputDirectory>Always</CopyToOutputDirectory>
     </Content>
@@ -213,7 +210,7 @@
     </Content>
   </ItemGroup>
   <ItemGroup>
-    <Content Include="Config\Ignite.exe.config.test">
+    <Content Include="Config\Apache.Ignite.exe.config.test">
       <CopyToOutputDirectory>Always</CopyToOutputDirectory>
     </Content>
   </ItemGroup>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheForkedTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheForkedTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheForkedTest.cs
index 04aff5f..2f3e0d0 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheForkedTest.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Cache/CacheForkedTest.cs
@@ -24,7 +24,6 @@ namespace Apache.Ignite.Core.Tests.Cache
     /// <summary>
     /// Tests cache with a standalone process.
     /// </summary>
-    [Ignore("IGNITE-1367")]
     public class CacheForkedTest
     {
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedPortableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedPortableClosureTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedPortableClosureTaskTest.cs
index 4b303ce..4ce917b 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedPortableClosureTaskTest.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedPortableClosureTaskTest.cs
@@ -17,12 +17,9 @@
 
 namespace Apache.Ignite.Core.Tests.Compute.Forked
 {
-    using NUnit.Framework;
-
     /// <summary>
     /// Forked closure execution tests for portable objects.
     /// </summary>
-    [Ignore("IGNITE-1367")]
     public class ForkedPortableClosureTaskTest : PortableClosureTaskTest
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedResourceTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedResourceTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedResourceTaskTest.cs
index 00fd28a..84c1ba2 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedResourceTaskTest.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedResourceTaskTest.cs
@@ -22,7 +22,7 @@ namespace Apache.Ignite.Core.Tests.Compute.Forked
     /// <summary>
     /// Forked resource task test.
     /// </summary>
-    [Ignore("IGNITE-1367")]
+    [Ignore("IGNITE-1381")]
     public class ForkedResourceTaskTest : ResourceTaskTest
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedSerializableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedSerializableClosureTaskTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedSerializableClosureTaskTest.cs
index edc18f2..0324125 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedSerializableClosureTaskTest.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedSerializableClosureTaskTest.cs
@@ -22,7 +22,7 @@ namespace Apache.Ignite.Core.Tests.Compute.Forked
     /// <summary>
     /// Forked closure execution tests for serializable objects.
     /// </summary>
-    [Ignore("IGNITE-1367")]
+    [Ignore("IGNITE-1381")]
     public class ForkedSerializableClosureTaskTest : SerializableClosureTaskTest
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedTaskAdapterTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedTaskAdapterTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedTaskAdapterTest.cs
index e7bad17..a4cf182 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedTaskAdapterTest.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Compute/Forked/ForkedTaskAdapterTest.cs
@@ -17,12 +17,9 @@
 
 namespace Apache.Ignite.Core.Tests.Compute.Forked
 {
-    using NUnit.Framework;
-
     /// <summary>
     /// Forked task adapter test.
     /// </summary>
-    [Ignore("IGNITE-1367")]
     public class ForkedTaskAdapterTest : TaskAdapterTest
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Apache.Ignite.exe.config.test
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Apache.Ignite.exe.config.test b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Apache.Ignite.exe.config.test
new file mode 100644
index 0000000..2bda365
--- /dev/null
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Apache.Ignite.exe.config.test
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="utf-8"?>
+
+<!--
+  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.
+-->
+
+<configuration>
+    <startup>
+        <supportedRuntime version="v4.0" sku=".NETFramework,Version=v4.0"/>
+    </startup>
+
+    <runtime>
+        <gcServer enabled="true" />
+    </runtime>
+
+    <appSettings>
+        <add key="Ignite.SpringConfigUrl" value="config\compute\compute-standalone.xml" />
+        <add key="Ignite.Assembly.1" value="test-1.dll" />
+        <add key="Ignite.Assembly.2" value="test-2.dll" />
+        <add key="Ignite.JvmOption.1" value="-DOPT1" />
+        <add key="Ignite.JvmOption.2" value="-DOPT2" />
+        <add key="Ignite.JvmOption.3" value="-Xms601m" />
+        <add key="Ignite.JvmOption.4" value="-Xmx702m" />
+        <add key="Ignite.JvmInitialMemoryMB" value="601" />
+        <add key="Ignite.JvmMaxMemoryMB" value="702" />
+    </appSettings>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Ignite.exe.config.test
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Ignite.exe.config.test b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Ignite.exe.config.test
deleted file mode 100644
index 305fbd5..0000000
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/Ignite.exe.config.test
+++ /dev/null
@@ -1,41 +0,0 @@
-<?xml version="1.0" encoding="utf-8"?>
-
-<!--
-  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.
--->
-
-<configuration>
-    <startup>
-        <supportedRuntime version="v4.0" sku=".NETFramework,Version=v4.0"/>
-    </startup>
-
-    <runtime>
-        <gcServer enabled="true" />
-    </runtime>
-
-    <appSettings>
-        <add key="GridGain.SpringConfigUrl" value="config\compute\compute-standalone.xml" />
-        <add key="GridGain.Assembly.1" value="test-1.dll" />
-        <add key="GridGain.Assembly.2" value="test-2.dll" />
-        <add key="GridGain.JvmOption.1" value="-DOPT1" />
-        <add key="GridGain.JvmOption.2" value="-DOPT2" />
-        <add key="GridGain.JvmOption.3" value="-Xms601m" />
-        <add key="GridGain.JvmOption.4" value="-Xmx702m" />
-        <add key="GridGain.JvmInitialMemoryMB" value="601" />
-        <add key="GridGain.JvmMaxMemoryMB" value="702" />
-    </appSettings>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-portables.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-portables.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-portables.xml
deleted file mode 100644
index 1d90c99..0000000
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache-portables.xml
+++ /dev/null
@@ -1,226 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:util="http://www.springframework.org/schema/util"
-       xsi:schemaLocation="
-        http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <property name="localHost" value="127.0.0.1"/>
-
-        <property name="marshaller">
-            <bean class="org.gridgain.grid.marshaller.optimized.GridOptimizedMarshaller">
-                <property name="requireSerializable" value="false"/>
-            </bean>
-        </property>
-
-        <property name="includeEventTypes">
-            <util:constant static-field="org.apache.ignite.events.IgniteEventType.EVTS_CACHE"/>
-        </property>
-
-        <property name="portableConfiguration">
-            <bean class="org.gridgain.grid.configuration.PortableConfiguration">
-                <property name="typeConfigurations">
-                    <list>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
-                            <property name="className" value="org.gridgain.client.GridClientTestPortable"/>
-                        </bean>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
-                            <property name="className" value="org.gridgain.client.model.PortablePerson"/>
-                        </bean>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
-                            <property name="className" value="org.gridgain.client.model.ImplicitPortablePerson"/>
-                        </bean>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
-                            <property name="className" value="NoDefPortablePerson"/>
-                        </bean>
-                    </list>
-                </property>
-            </bean>
-        </property>
-
-        <property name="cacheConfiguration">
-            <list>
-                <bean parent="cache-template">
-                    <property name="name" value="local"/>
-                    <property name="cacheMode" value="LOCAL"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                </bean>
-
-                <bean parent="cache-template">
-                    <property name="name" value="local_atomic"/>
-                    <property name="cacheMode" value="LOCAL"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                </bean>
-
-                <bean parent="cache-template">
-                    <property name="name" value="partitioned"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                </bean>
-
-                <bean parent="cache-template">
-                    <property name="name" value="partitioned_atomic"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
-                </bean>
-
-                <bean parent="cache-template">
-                    <property name="name" value="partitioned_near"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="nearConfiguration">
-                        <bean class="org.apache.ignite.configuration.NearCacheConfiguration" />
-                    </property>
-                </bean>
-
-                <bean parent="cache-template">
-                    <property name="name" value="partitioned_atomic_near"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
-                    <property name="nearConfiguration">
-                        <bean class="org.apache.ignite.configuration.NearCacheConfiguration" />
-                    </property>
-                </bean>
-
-                <bean parent="cache-template">
-                    <property name="name" value="replicated"/>
-                    <property name="cacheMode" value="REPLICATED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                </bean>
-
-                <bean parent="cache-template">
-                    <property name="name" value="replicated_atomic"/>
-                    <property name="cacheMode" value="REPLICATED"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                    <property name="atomicWriteOrderMode" value="PRIMARY"/>
-                </bean>
-            </list>
-        </property>
-
-        <property name="swapSpaceSpi">
-            <bean class="org.gridgain.grid.spi.swapspace.inmemory.GridTestSwapSpaceSpi"/>
-        </property>
-
-        <property name="indexingSpi">
-            <list>
-                <bean class="org.gridgain.grid.spi.indexing.h2.GridH2IndexingSpi">
-                </bean>
-            </list>
-        </property>
-
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500..47501</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
-
-        <property name="platformConfiguration">
-            <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration">
-                <property name="portableConfiguration">
-                    <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetPortableConfiguration">
-                        <property name="types">
-                            <util:list value-type="java.lang.String">
-
-                            </util:list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
-    </bean>
-
-    <bean id="cache-template" abstract="true" class="org.apache.ignite.cache.CacheConfiguration">
-        <property name="rebalanceMode" value="SYNC"/>
-        <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-        <property name="swapEnabled" value="true"/>
-        <property name="portableEnabled" value="true"/>
-        <property name="txSerializableEnabled" value="true"/>
-        <property name="backups" value="1"/>
-
-        <property name="store">
-            <bean class="org.gridgain.grid.internal.processors.cache.GridCacheGenericTestStore"/>
-        </property>
-
-        <property name="typeMetadata">
-            <list>
-                <bean class="org.apache.ignite.cache.CacheTypeMetadata">
-                    <property name="valueType" value="PortablePerson"/>
-                    <property name="ascendingFields">
-                        <map>
-                            <entry key="age" value="java.lang.Integer"/>
-                        </map>
-                    </property>
-                    <property name="queryFields">
-                        <map>
-                            <entry key="name" value="java.lang.String"/>
-                        </map>
-                    </property>
-                    <property name="textFields">
-                        <list>
-                            <value>address</value>
-                        </list>
-                    </property>
-                </bean>
-                <bean class="org.apache.ignite.cache.CacheTypeMetadata">
-                    <property name="valueType" value="ImplicitPortablePerson"/>
-                    <property name="ascendingFields">
-                        <map>
-                            <entry key="age" value="java.lang.Integer"/>
-                        </map>
-                    </property>
-                    <property name="queryFields">
-                        <map>
-                            <entry key="name" value="java.lang.String"/>
-                        </map>
-                    </property>
-                </bean>
-                <bean class="org.apache.ignite.cache.CacheTypeMetadata">
-                    <property name="valueType" value="NoDefPortablePerson"/>
-                    <property name="ascendingFields">
-                        <map>
-                            <entry key="age" value="java.lang.Integer"/>
-                        </map>
-                    </property>
-                    <property name="queryFields">
-                        <map>
-                            <entry key="name" value="java.lang.String"/>
-                        </map>
-                    </property>
-                </bean>
-            </list>
-        </property>
-    </bean>
-</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache.xml
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache.xml b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache.xml
index 5d65ae8..c48e867 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache.xml
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Config/native-client-test-cache.xml
@@ -29,27 +29,6 @@
         <property name="localHost" value="127.0.0.1"/>
         <property name="connectorConfiguration"><null/></property>
 
-        <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.portable.PortableMarshaller">
-                <property name="typeConfigurations">
-                    <list>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
-                            <property name="className" value="org.gridgain.client.ClientTestPortable"/>
-                        </bean>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
-                            <property name="className" value="org.gridgain.internal.client.model.PortablePerson"/>
-                        </bean>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
-                            <property name="className" value="org.gridgain.internal.client.model.ImplicitPortablePerson"/>
-                        </bean>
-                        <bean class="org.apache.ignite.portable.PortableTypeConfiguration">
-                            <property name="className" value="NoDefPortablePerson"/>
-                        </bean>
-                    </list>
-                </property>
-            </bean>
-        </property>
-
         <property name="includeEventTypes">
             <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
         </property>
@@ -132,15 +111,6 @@
             <bean class="org.apache.ignite.spi.swapspace.inmemory.GridTestSwapSpaceSpi"/>
         </property>
 
-        <!-- TODO: -->
-        <!--<property name="indexingSpi">
-            <list>
-                <bean class="org.gridgain.grid.spi.indexing.h2.GridH2IndexingSpi">
-                    <property name="defaultIndexFixedTyping" value="false"/>
-                </bean>
-            </list>
-        </property>-->
-
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="ipFinder">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExecutableTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExecutableTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExecutableTest.cs
index 9c47cbc..e4530cb 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExecutableTest.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/ExecutableTest.cs
@@ -33,7 +33,6 @@ namespace Apache.Ignite.Core.Tests
     /// <summary>
     /// Tests for executable.
     /// </summary>
-    [Ignore("IGNITE-1367")]
     public class ExecutableTest
     {
         /** Spring configuration path. */
@@ -98,18 +97,18 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestConfig()
         {
-            IgniteProcess.ReplaceConfiguration("config\\Ignite.exe.config.test");
+            IgniteProcess.ReplaceConfiguration("config\\Apache.Ignite.exe.config.test");
 
             GenerateDll("test-1.dll");
             GenerateDll("test-2.dll");
 
-            IgniteProcess proc = new IgniteProcess(
+            var proc = new IgniteProcess(
                 "-jvmClasspath=" + TestUtils.CreateTestClasspath()
                 );
 
             Assert.IsTrue(_grid.WaitTopology(2, 30000));
 
-            RemoteConfiguration cfg = RemoteConfig();
+            var cfg = RemoteConfig();
 
             Assert.AreEqual(SpringCfgPath, cfg.SpringConfigUrl);
             Assert.IsTrue(cfg.JvmOptions.Contains("-DOPT1") && cfg.JvmOptions.Contains("-DOPT2"));
@@ -127,7 +126,7 @@ namespace Apache.Ignite.Core.Tests
             GenerateDll("test-1.dll");
             GenerateDll("test-2.dll");
 
-            IgniteProcess proc = new IgniteProcess(
+            var proc = new IgniteProcess(
                 "-jvmClasspath=" + TestUtils.CreateTestClasspath(),
                 "-springConfigUrl=" + SpringCfgPath,
                 "-assembly=test-1.dll",
@@ -136,7 +135,7 @@ namespace Apache.Ignite.Core.Tests
 
             Assert.IsTrue(_grid.WaitTopology(2, 30000));
 
-            RemoteConfiguration cfg = RemoteConfig();
+            var cfg = RemoteConfig();
 
             Assert.IsTrue(cfg.Assemblies.Contains("test-1.dll") && cfg.Assemblies.Contains("test-2.dll"));
         }
@@ -147,7 +146,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestJvmOptsCmd()
         {
-            IgniteProcess proc = new IgniteProcess(
+            var proc = new IgniteProcess(
                 "-jvmClasspath=" + TestUtils.CreateTestClasspath(),
                 "-springConfigUrl=" + SpringCfgPath,
                 "-J-DOPT1",
@@ -156,7 +155,7 @@ namespace Apache.Ignite.Core.Tests
 
             Assert.IsTrue(_grid.WaitTopology(2, 30000));
 
-            RemoteConfiguration cfg = RemoteConfig();
+            var cfg = RemoteConfig();
 
             Assert.IsTrue(cfg.JvmOptions.Contains("-DOPT1") && cfg.JvmOptions.Contains("-DOPT2"));
         }
@@ -211,7 +210,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestJvmMemoryOptsAppConfig()
         {
-            IgniteProcess.ReplaceConfiguration("config\\Ignite.exe.config.test");
+            IgniteProcess.ReplaceConfiguration("config\\Apache.Ignite.exe.config.test");
 
             GenerateDll("test-1.dll");
             GenerateDll("test-2.dll");
@@ -285,10 +284,10 @@ namespace Apache.Ignite.Core.Tests
         /// <returns>Node configuration.</returns>
         private static IgniteConfiguration Configuration(string path)
         {
-            IgniteConfiguration cfg = new IgniteConfiguration();
+            var cfg = new IgniteConfiguration();
 
 
-            PortableConfiguration portCfg = new PortableConfiguration();
+            var portCfg = new PortableConfiguration();
 
             ICollection<PortableTypeConfiguration> portTypeCfgs = new List<PortableTypeConfiguration>();
 
@@ -307,7 +306,7 @@ namespace Apache.Ignite.Core.Tests
                 "-Xcheck:jni",
                 "-Xms4g",
                 "-Xmx4g",
-                "-DGRIDGAIN_QUIET=false",
+                "-DIGNITE_QUIET=false",
                 "-Xnoagent",
                 "-Djava.compiler=NONE",
                 "-Xdebug",
@@ -326,21 +325,21 @@ namespace Apache.Ignite.Core.Tests
         /// <param name="outputPath"></param>
         private static void GenerateDll(string outputPath)
         {
-            CSharpCodeProvider codeProvider = new CSharpCodeProvider();
+            var codeProvider = new CSharpCodeProvider();
 
 #pragma warning disable 0618
 
-            ICodeCompiler icc = codeProvider.CreateCompiler();
+            var icc = codeProvider.CreateCompiler();
 
 #pragma warning restore 0618
 
-            CompilerParameters parameters = new CompilerParameters();
+            var parameters = new CompilerParameters();
             parameters.GenerateExecutable = false;
             parameters.OutputAssembly = outputPath;
 
-            string src = "namespace GridGain.Client.Test { public class Foo {}}";
+            var src = "namespace Apache.Ignite.Client.Test { public class Foo {}}";
 
-            CompilerResults results = icc.CompileAssemblyFromSource(parameters, src);
+            var results = icc.CompileAssemblyFromSource(parameters, src);
 
             Assert.False(results.Errors.HasErrors);
         }
@@ -372,11 +371,11 @@ namespace Apache.Ignite.Core.Tests
 
             public RemoteConfiguration Invoke()
             {
-                Ignite grid0 = (Ignite) ((IgniteProxy) _grid).Target;
+                var grid0 = (Ignite) ((IgniteProxy) _grid).Target;
 
-                IgniteConfiguration cfg = grid0.Configuration;
+                var cfg = grid0.Configuration;
 
-                RemoteConfiguration res = new RemoteConfiguration
+                var res = new RemoteConfiguration
                 {
                     IgniteHome = cfg.IgniteHome,
                     SpringConfigUrl = cfg.SpringConfigUrl,

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
index d2b2efa..ec7e157 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
@@ -186,34 +186,6 @@ namespace Apache.Ignite.Core.Tests
             }
         }
 
-        /*
-        [Test]
-        public void TestStartInvalidJvmOptions()
-        {
-            GridGain.Impl.IgniteManager.DestroyJvm();
-
-            IgniteConfiguration cfg = new IgniteConfiguration();
-
-            cfg.NativeXmlConfig = "config\\start-test-grid1.xml";
-            cfg.NativeJvmOptions = new List<string> { "invalid_option"};
-
-            try
-            {
-                Ignition.Start(cfg);
-
-                Assert.Fail("Start should fail.");
-            }
-            catch (IgniteException e)
-            {
-                Console.WriteLine("Expected exception: " + e);
-            }
-
-            cfg.NativeJvmOptions = new List<string> { "-Xmx1g", "-Xms1g" };
-
-            Ignition.Start(cfg);
-        }
-        */
-
         /// <summary>
         /// 
         /// </summary>


[07/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4


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

Branch: refs/heads/ignite-843
Commit: bd8c60740d8535a3a4434533bdc4b9876ccbddb9
Parents: 039c48d 826cb2f
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 11:34:05 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 11:34:05 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2IndexBase.java           | 42 ++++++++++++++++++++
 .../query/IgniteSqlSplitterSelfTest.java        | 35 ++++++++++++++--
 2 files changed, 74 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[39/39] ignite git commit: Merge branches 'ignite-843' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-843

Posted by ak...@apache.org.
Merge branches 'ignite-843' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-843

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java


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

Branch: refs/heads/ignite-843
Commit: 5625672bc4164900f2b8e1e648a84f35101cf2c2
Parents: aec68a4 f8aafe4
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Sep 8 05:49:50 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Sep 8 05:49:50 2015 +0700

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  13 +-
 .../messaging/MessagingPingPongExample.java     |   6 +-
 .../CacheClientPortableQueryExample.java        |   3 -
 .../messaging/MessagingPingPongExample.java     |   2 +-
 .../examples/MessagingExamplesSelfTest.java     |   3 +-
 .../examples/MessagingExamplesSelfTest.java     |   3 +-
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../ignite/internal/MarshallerContextImpl.java  |  31 +--
 .../distributed/dht/GridDhtLocalPartition.java  |  51 ++++-
 .../dht/GridPartitionedGetFuture.java           |  14 +-
 .../distributed/near/GridNearGetFuture.java     |  13 ++
 .../query/GridCacheDistributedQueryFuture.java  |   5 +-
 .../query/GridCacheDistributedQueryManager.java |   9 +-
 .../cache/query/GridCacheQueryManager.java      | 177 +++++----------
 .../cache/query/GridCacheQueryRequest.java      |  59 ++++-
 .../visor/cache/VisorCacheStartTask.java        |   6 +-
 .../messages/TcpDiscoveryClientAckResponse.java |   2 +-
 .../cache/GridCacheIncrementTransformTest.java  |   2 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   5 -
 .../IgniteDaemonNodeMarshallerCacheTest.java    |  10 +
 .../IgniteCacheNodeJoinAbstractTest.java        |  42 ++++
 .../main/cpp/common/project/vs/common.vcxproj   |  10 +-
 .../platform/src/main/cpp/common/src/java.cpp   |   2 +-
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |   2 +-
 .../Impl/Common/DelegateTypeDescriptor.cs       |  17 +-
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |   1 +
 .../Impl/Common/IFutureConverter.cs             |   2 +
 .../Impl/Common/IFutureInternal.cs              |   1 +
 .../Impl/Compute/ComputeImpl.cs                 |   2 +-
 .../Impl/Compute/ComputeJobHolder.cs            |   6 +
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |   2 +-
 .../Impl/Memory/IPlatformMemory.cs              |   3 +
 .../Impl/Memory/PlatformMemory.cs               |   1 +
 .../Impl/Memory/PlatformMemoryManager.cs        |   2 +
 .../Impl/Memory/PlatformMemoryPool.cs           |   1 +
 .../Impl/Memory/PlatformMemoryStream.cs         |   1 +
 .../Impl/Memory/PlatformMemoryUtils.cs          |   1 +
 .../Impl/Memory/PlatformRawMemory.cs            |   1 +
 .../Impl/Portable/Io/PortableAbstractStream.cs  |   1 -
 .../Properties/AssemblyInfo.cs                  |   3 +-
 .../platform/src/main/dotnet/Apache.Ignite.sln  |  10 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |  76 +++++++
 .../src/main/dotnet/Apache.Ignite/App.config    |  56 +++++
 .../Config/AppSettingsConfigurator.cs           | 113 ++++++++++
 .../Apache.Ignite/Config/ArgsConfigurator.cs    | 164 ++++++++++++++
 .../Apache.Ignite/Config/ConfigValueParser.cs   |  42 ++++
 .../Apache.Ignite/Config/IConfigurator.cs       |  34 +++
 .../main/dotnet/Apache.Ignite/IgniteRunner.cs   | 171 ++++++++++++++
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |  35 +++
 .../Apache.Ignite/Service/IgniteService.cs      | 219 ++++++++++++++++++
 .../Apache.Ignite/Service/NativeMethods.cs      |  57 +++++
 .../Apache.Ignite/Service/ServiceDescription.cs |  32 +++
 .../Apache.Ignite.Core.Tests.csproj             |   5 +-
 .../Cache/CacheForkedTest.cs                    |   1 -
 .../Forked/ForkedPortableClosureTaskTest.cs     |   3 -
 .../Compute/Forked/ForkedResourceTaskTest.cs    |   2 +-
 .../Forked/ForkedSerializableClosureTaskTest.cs |   2 +-
 .../Compute/Forked/ForkedTaskAdapterTest.cs     |   3 -
 .../Config/Apache.Ignite.exe.config.test        |  41 ++++
 .../Config/Ignite.exe.config.test               |  41 ----
 .../native-client-test-cache-portables.xml      | 226 -------------------
 .../Config/native-client-test-cache.xml         |  30 ---
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  39 ++--
 .../IgniteStartStopTest.cs                      |  28 ---
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |   2 +-
 .../Process/IgniteProcess.cs                    |  33 +--
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   2 +-
 parent/pom.xml                                  |   1 +
 69 files changed, 1405 insertions(+), 582 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5625672b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/5625672b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 3df473e,2041464..07b6fa1
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@@ -3126,25 -3057,4 +3082,4 @@@ public abstract class GridCacheQueryMan
              false,
              keepPortable);
      }
- 
-     /**
-      * Creates SQL fields query which will include results metadata if needed.
-      *
-      * @param qry SQL query.
-      * @param incMeta Whether to include results metadata.
-      * @param keepPortable Keep portable flag.
-      * @return Created query.
-      */
-     public CacheQuery<List<?>> createSqlFieldsQuery(String qry, boolean incMeta, boolean keepPortable) {
-         assert qry != null;
- 
-         return new GridCacheQueryAdapter<>(cctx,
-             SQL_FIELDS,
-             null,
-             qry,
-             null,
-             null,
-             incMeta,
-             keepPortable);
-     }
 -}
 +}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5625672b/parent/pom.xml
----------------------------------------------------------------------


[16/39] ignite git commit: Release notes - ignite-1.4

Posted by ak...@apache.org.
Release notes - ignite-1.4


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

Branch: refs/heads/ignite-843
Commit: 83454cc6fbe9faff1ed3b0b37d6dbc29d250bd08
Parents: cb78a14
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Sep 4 14:34:48 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Sep 4 14:34:48 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 12 +++++++++++-
 1 file changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/83454cc6/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index ce180c7..3f3ac7b 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -3,8 +3,18 @@ Apache Ignite Release Notes
 
 Apache Ignite In-Memory Data Fabric 1.4
 ---------------------------------------
+* Added SSL support to communication and discovery.
 * Added support for log4j2.
-
+* Added versioned entry to cache API.
+* Fixed IGNITE_HOME resolution with JBoss.
+* Fixed configuration file resolution on classpath.
+* Fixed MBean registration for caches.
+* Fixed near cache evictions with offheap.
+* Fixed SQL queries group index logic.
+* Fixed memory leak in partitioned transactional cache.
+* Fixed auto-retries for cache operations in recoverable cases.
+* Fixed affinity routing in compute grid.
+* Many stability and fault-tolerance fixes.
 
 Apache Ignite In-Memory Data Fabric 1.3
 ---------------------------------------


[36/39] ignite git commit: IGNITE-1367: Moved .Net executable to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LoadDllTest.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LoadDllTest.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LoadDllTest.cs
index af9387c..25ffab3 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LoadDllTest.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/LoadDllTest.cs
@@ -224,7 +224,7 @@ namespace Apache.Ignite.Core.Tests
                 OutputAssembly = outputPath
             };
 
-            var src = "namespace GridGain.Client.Test { public class Foo {}}";
+            var src = "namespace Apache.Ignite.Client.Test { public class Foo {}}";
 
             var results = icc.CompileAssemblyFromSource(parameters, src);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
index 9e332fe..fd08116 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Process/IgniteProcess.cs
@@ -31,10 +31,10 @@ namespace Apache.Ignite.Core.Tests.Process
     public class IgniteProcess
     {
         /** Executable file name. */
-        private static readonly string ExeName = "Ignite.exe";
+        private static readonly string ExeName = "Apache.Ignite.exe";
 
         /** Executable process name. */
-        private static readonly string ExeProcName = ExeName.Substring(0, ExeName.IndexOf('.'));
+        private static readonly string ExeProcName = ExeName.Substring(0, ExeName.LastIndexOf('.'));
 
         /** Executable configuration file name. */
         private static readonly string ExeCfgName = ExeName + ".config";
@@ -71,17 +71,14 @@ namespace Apache.Ignite.Core.Tests.Process
             // ReSharper disable once PossibleNullReferenceException
             ExeDir = dir.FullName;
 
-            FileInfo[] exe = dir.GetFiles(ExeName);
+            var exe = dir.GetFiles(ExeName);
 
-
-            // TODO: IGNITE-1367
-            /*
             if (exe.Length == 0)
                 throw new Exception(ExeName + " is not found in test output directory: " + dir.FullName);
 
             ExePath = exe[0].FullName;
 
-            FileInfo[] exeCfg = dir.GetFiles(ExeCfgName);
+            var exeCfg = dir.GetFiles(ExeCfgName);
 
             if (exeCfg.Length == 0)
                 throw new Exception(ExeCfgName + " is not found in test output directory: " + dir.FullName);
@@ -90,7 +87,7 @@ namespace Apache.Ignite.Core.Tests.Process
 
             ExeCfgBakPath = Path.Combine(ExeDir, ExeCfgBakName);
 
-            File.Delete(ExeCfgBakPath);*/
+            File.Delete(ExeCfgBakPath);
         }
 
         /// <summary>
@@ -119,7 +116,7 @@ namespace Apache.Ignite.Core.Tests.Process
         }
 
         /// <summary>
-        /// Kill all GridGain processes.
+        /// Kill all Ignite processes.
         /// </summary>
         public static void KillAll()
         {
@@ -157,7 +154,7 @@ namespace Apache.Ignite.Core.Tests.Process
         /// Starts a grid process.
         /// </summary>
         /// <param name="exePath">Exe path.</param>
-        /// <param name="ggHome">GridGain home.</param>
+        /// <param name="ggHome">Ignite home.</param>
         /// <param name="outReader">Output reader.</param>
         /// <param name="args">Arguments.</param>
         /// <returns>Started process.</returns>
@@ -182,7 +179,7 @@ namespace Apache.Ignite.Core.Tests.Process
             if (!string.IsNullOrEmpty(ggHome))
                 procStart.EnvironmentVariables[IgniteManager.EnvIgniteHome] = ggHome;
 
-            procStart.EnvironmentVariables["GRIDGAIN_NATIVE_TEST_CLASSPATH"] = "true";
+            procStart.EnvironmentVariables[IgniteManager.EnvIgniteNativeTestClasspath] = "true";
 
             procStart.CreateNoWindow = true;
             procStart.UseShellExecute = false;
@@ -195,7 +192,7 @@ namespace Apache.Ignite.Core.Tests.Process
             if (workDir != null)
                 procStart.WorkingDirectory = workDir;
 
-            Console.WriteLine("About to run Ignite.exe process [exePath=" + exePath + ", arguments=" + sb + ']');
+            Console.WriteLine("About to run Apache.Ignite.exe process [exePath=" + exePath + ", arguments=" + sb + ']');
 
             // 2. Start.
             var proc = Process.Start(procStart);
@@ -216,10 +213,7 @@ namespace Apache.Ignite.Core.Tests.Process
         /// </summary>
         public bool Alive
         {
-            get
-            {
-                return !_proc.HasExited;
-            }
+            get { return !_proc.HasExited; }
         }
 
         /// <summary>
@@ -279,14 +273,11 @@ namespace Apache.Ignite.Core.Tests.Process
         /// <param name="err">Whether this is error stream.</param>
         private static void Attach(Process proc, StreamReader reader, IIgniteProcessOutputReader outReader, bool err)
         {
-            Thread thread = new Thread(() =>
+            new Thread(() =>
             {
                 while (!proc.HasExited)
                     outReader.OnOutput(proc, reader.ReadLine(), err);
-            }) {IsBackground = true};
-
-
-            thread.Start();
+            }) {IsBackground = true}.Start();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2d19b17/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index ca45d68..a60efa7 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -71,7 +71,7 @@ namespace Apache.Ignite.Core.Tests
         private static int _seed = Environment.TickCount;
 
         /// <summary>
-        /// Kill GridGain processes.
+        /// Kill Ignite processes.
         /// </summary>
         public static void KillProcesses()
         {


[20/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4


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

Branch: refs/heads/ignite-843
Commit: d1c2db860d408c617a4ec98247433610bfb062df
Parents: b41309c ecc492d
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 17:23:01 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 17:23:01 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                                      | 13 ++++++++++++-
 .../datagrid/CacheClientPortableQueryExample.java      |  3 ---
 2 files changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[11/39] ignite git commit: Platform .Net: update version

Posted by ak...@apache.org.
Platform .Net: update version


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

Branch: refs/heads/ignite-843
Commit: cb78a14b89384edecfdef31a8b20fb25878dd39e
Parents: 83b6e5f
Author: ptupitsyn <pt...@gridgain.com>
Authored: Fri Sep 4 13:58:19 2015 +0300
Committer: ptupitsyn <pt...@gridgain.com>
Committed: Fri Sep 4 13:58:19 2015 +0300

----------------------------------------------------------------------
 .../main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cb78a14b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index 8deafec..326df2a 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("97db45a8-f922-456a-a819-7b3c6e5e03ba")]
 
-[assembly: AssemblyVersion("1.4.1.0")]
-[assembly: AssemblyFileVersion("1.4.1.0")]
+[assembly: AssemblyVersion("1.4.0.0")]
+[assembly: AssemblyFileVersion("1.4.0.0")]
 
 [assembly: CLSCompliant(true)]
 


[04/39] ignite git commit: IGNITE-1333 - SQL Group index can return wrong restult in half-bounded conditions - Fixes #50.

Posted by ak...@apache.org.
IGNITE-1333 - SQL Group index can return wrong restult in half-bounded conditions - Fixes #50.

Signed-off-by: S.Vladykin <sv...@gridgain.com>


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

Branch: refs/heads/ignite-843
Commit: bf64c6edc8f44a0f2d4ef2707d8bdd2a98589297
Parents: 8901575
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Sep 4 10:48:03 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Sep 4 10:50:47 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2IndexBase.java           | 42 ++++++++++++++++++++
 .../query/IgniteSqlSplitterSelfTest.java        | 35 ++++++++++++++--
 2 files changed, 74 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bf64c6ed/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index ff9aa23..39664ff 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -27,6 +27,8 @@ import org.h2.index.BaseIndex;
 import org.h2.message.DbException;
 import org.h2.result.Row;
 import org.h2.result.SearchRow;
+import org.h2.result.SortOrder;
+import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -106,6 +108,46 @@ public abstract class GridH2IndexBase extends BaseIndex {
         // No-op.
     }
 
+    /** {@inheritDoc} */
+    @Override public int compareRows(SearchRow rowData, SearchRow compare) {
+        if (rowData == compare)
+            return 0;
+
+        for (int i = 0, len = indexColumns.length; i < len; i++) {
+            int index = columnIds[i];
+
+            Value v1 = rowData.getValue(index);
+            Value v2 = compare.getValue(index);
+
+            if (v1 == null || v2 == null)
+                return 0;
+
+            int c = compareValues(v1, v2, indexColumns[i].sortType);
+
+            if (c != 0)
+                return c;
+        }
+        return 0;
+    }
+
+    /**
+     * @param a First value.
+     * @param b Second value.
+     * @param sortType Sort type.
+     * @return Comparison result.
+     */
+    private int compareValues(Value a, Value b, int sortType) {
+        if (a == b)
+            return 0;
+
+        int comp = table.compareTypeSave(a, b);
+
+        if ((sortType & SortOrder.DESCENDING) != 0)
+            comp = -comp;
+
+        return comp;
+    }
+
     /**
      * Filters rows from expired ones and using predicate.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf64c6ed/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index f70c218..75112fd 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -158,15 +158,44 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
 
             // Check results.
             assertEquals(1, columnQuery(c, qry + "where a = 1 and b = 1").size());
+            assertEquals(0, columnQuery(c, qry + "where a = 1 and b = 2").size());
+            assertEquals(1, columnQuery(c, qry + "where a = 1 and b = 3").size());
             assertEquals(2, columnQuery(c, qry + "where a = 1 and b < 4").size());
             assertEquals(2, columnQuery(c, qry + "where a = 1 and b <= 3").size());
             assertEquals(1, columnQuery(c, qry + "where a = 1 and b < 3").size());
             assertEquals(2, columnQuery(c, qry + "where a = 1 and b > 0").size());
             assertEquals(1, columnQuery(c, qry + "where a = 1 and b > 1").size());
             assertEquals(2, columnQuery(c, qry + "where a = 1 and b >= 1").size());
-            assertEquals(4, columnQuery(c, qry + "where a > 0 and b > 0").size());
-            assertEquals(4, columnQuery(c, qry + "where a > 0 and b >= 1").size());
-            assertEquals(3, columnQuery(c, qry + "where a > 0 and b > 1").size());
+
+            assertEquals(4, columnQuery(c, qry + "where a > 0").size());
+            assertEquals(4, columnQuery(c, qry + "where a >= 1").size());
+            assertEquals(4, columnQuery(c, qry + "where b > 0").size());
+            assertEquals(4, columnQuery(c, qry + "where b >= 1").size());
+
+            assertEquals(4, columnQuery(c, qry + "where a < 2").size());
+            assertEquals(4, columnQuery(c, qry + "where a <= 1").size());
+            assertEquals(4, columnQuery(c, qry + "where b < 3").size());
+            assertEquals(5, columnQuery(c, qry + "where b <= 3").size());
+
+            assertEquals(3, columnQuery(c, qry + "where a > 0 and b > 0").size());
+            assertEquals(2, columnQuery(c, qry + "where a > 0 and b >= 2").size());
+            assertEquals(3, columnQuery(c, qry + "where a >= 1 and b > 0").size());
+            assertEquals(2, columnQuery(c, qry + "where a >= 1 and b >= 2").size());
+
+            assertEquals(3, columnQuery(c, qry + "where a > 0 and b < 3").size());
+            assertEquals(2, columnQuery(c, qry + "where a > 0 and b <= 1").size());
+            assertEquals(3, columnQuery(c, qry + "where a >= 1 and b < 3").size());
+            assertEquals(2, columnQuery(c, qry + "where a >= 1 and b <= 1").size());
+
+            assertEquals(2, columnQuery(c, qry + "where a < 2 and b < 3").size());
+            assertEquals(2, columnQuery(c, qry + "where a < 2 and b <= 1").size());
+            assertEquals(2, columnQuery(c, qry + "where a <= 1 and b < 3").size());
+            assertEquals(2, columnQuery(c, qry + "where a <= 1 and b <= 1").size());
+
+            assertEquals(3, columnQuery(c, qry + "where a < 2 and b > 0").size());
+            assertEquals(2, columnQuery(c, qry + "where a < 2 and b >= 3").size());
+            assertEquals(3, columnQuery(c, qry + "where a <= 1 and b > 0").size());
+            assertEquals(2, columnQuery(c, qry + "where a <= 1 and b >= 3").size());
         }
         finally {
             c.destroy();


[34/39] ignite git commit: Merge branch 'master-bak'

Posted by ak...@apache.org.
Merge branch 'master-bak'


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

Branch: refs/heads/ignite-843
Commit: 5f01d00b0e7c30e6174d8534efbf3fdc69058c7a
Parents: 27eb860 fb46316
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Sep 7 16:34:46 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Sep 7 16:34:46 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  13 +-
 .../messaging/MessagingPingPongExample.java     |   6 +-
 .../CacheClientPortableQueryExample.java        |   3 -
 .../messaging/MessagingPingPongExample.java     |   2 +-
 .../examples/MessagingExamplesSelfTest.java     |   3 +-
 .../examples/MessagingExamplesSelfTest.java     |   3 +-
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../ignite/internal/MarshallerContextImpl.java  |  31 ++--
 .../distributed/dht/GridDhtLocalPartition.java  |  51 +++++-
 .../dht/GridPartitionedGetFuture.java           |  14 +-
 .../distributed/near/GridNearGetFuture.java     |  13 ++
 .../query/GridCacheDistributedQueryFuture.java  |   5 +-
 .../query/GridCacheDistributedQueryManager.java |   9 +-
 .../cache/query/GridCacheQueryManager.java      | 177 ++++++-------------
 .../cache/query/GridCacheQueryRequest.java      |  59 ++++++-
 .../visor/cache/VisorCacheStartTask.java        |   6 +-
 .../messages/TcpDiscoveryClientAckResponse.java |   2 +-
 .../cache/GridCacheIncrementTransformTest.java  |   2 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   5 -
 .../IgniteCacheNodeJoinAbstractTest.java        |  42 +++++
 .../main/cpp/common/project/vs/common.vcxproj   |  10 +-
 .../platform/src/main/cpp/common/src/java.cpp   |   2 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 24 files changed, 282 insertions(+), 182 deletions(-)
----------------------------------------------------------------------



[38/39] ignite git commit: .Net: Set proper "CLSCompliant" attributes to unsafe public classes.

Posted by ak...@apache.org.
.Net: Set proper "CLSCompliant" attributes to unsafe public classes.


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

Branch: refs/heads/ignite-843
Commit: f8aafe448335792448be8483a4b3ebc5bd5b83ba
Parents: c2d19b1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Sep 7 16:52:59 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Sep 7 16:52:59 2015 +0300

----------------------------------------------------------------------
 .../src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs      | 1 +
 .../dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs     | 2 ++
 .../main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs | 1 +
 .../main/dotnet/Apache.Ignite.Core/Impl/Memory/IPlatformMemory.cs | 3 +++
 .../main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemory.cs  | 1 +
 .../Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs       | 2 ++
 .../dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryPool.cs   | 1 +
 .../dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs | 1 +
 .../dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs    | 1 +
 9 files changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f8aafe44/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
index 0bbc1a2..92b4fce 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
@@ -29,6 +29,7 @@ namespace Apache.Ignite.Core.Impl.Common
     /// Grid future implementation.
     /// </summary>
     [SuppressMessage("ReSharper", "ParameterHidesMember")]
+    [CLSCompliant(false)]
     public sealed class Future<T> : IFutureInternal, IFuture<T>
     {
         /** Converter. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8aafe44/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs
index 4169c61..e07597d 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs
@@ -17,11 +17,13 @@
 
 namespace Apache.Ignite.Core.Impl.Common
 {
+    using System;
     using Apache.Ignite.Core.Impl.Portable.IO;
 
     /// <summary>
     /// Marshals and converts future value.
     /// </summary>
+    [CLSCompliant(false)]
     public interface IFutureConverter<out T>
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8aafe44/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs
index 8547545..90f06be 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs
@@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Impl.Common
     /// <summary>
     /// Internal future interface.
     /// </summary>
+    [CLSCompliant(false)]
     public interface IFutureInternal
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8aafe44/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/IPlatformMemory.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/IPlatformMemory.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/IPlatformMemory.cs
index ff91f48..93fd164 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/IPlatformMemory.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/IPlatformMemory.cs
@@ -17,9 +17,12 @@
 
 namespace Apache.Ignite.Core.Impl.Memory
 {
+    using System;
+
     /// <summary>
     /// Platform memory chunk.
     /// </summary>
+    [CLSCompliant(false)]
     public interface IPlatformMemory
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8aafe44/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemory.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemory.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemory.cs
index e19505c..3a9ed26 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemory.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemory.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Impl.Memory
     /// <summary>
     /// Abstract memory chunk.
     /// </summary>
+    [CLSCompliant(false)]
     public abstract class PlatformMemory : IPlatformMemory
     {
         /** Memory pointer. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8aafe44/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs
index 3dc3953..b280140 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryManager.cs
@@ -17,6 +17,7 @@
 
 namespace Apache.Ignite.Core.Impl.Memory
 {
+    using System;
     using System.Diagnostics.CodeAnalysis;
     using System.Threading;
 
@@ -25,6 +26,7 @@ namespace Apache.Ignite.Core.Impl.Memory
     /// </summary>
     [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable",
         Justification = "This class instance usually lives as long as the app runs.")]
+    [CLSCompliant(false)]
     public class PlatformMemoryManager
     {
         /** Default capacity. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8aafe44/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryPool.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryPool.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryPool.cs
index 3cdcd2d..75e8965 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryPool.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryPool.cs
@@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Impl.Memory
     /// <summary>
     /// Platform memory pool.
     /// </summary>
+    [CLSCompliant(false)]
     public class PlatformMemoryPool : SafeHandleMinusOneIsInvalid
     {
         /** First pooled memory chunk. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8aafe44/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
index f151763..71da18f 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
@@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Impl.Memory
     /// <summary>
     /// Platform memory stream.
     /// </summary>
+    [CLSCompliant(false)]
     public unsafe class PlatformMemoryStream : IPortableStream
     {
         /** Length: 1 byte. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8aafe44/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
index 6de9f43..59c915b 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Impl.Memory
     /// <summary>
     /// Non-resizeable raw memory chunk without metadata header.
     /// </summary>
+    [CLSCompliant(false)]
     public class PlatformRawMemory : IPlatformMemory
     {
         /** */


[29/39] ignite git commit: Merge branch 'ignite-1.4'

Posted by ak...@apache.org.
Merge branch 'ignite-1.4'

Conflicts:
	examples/pom.xml
	modules/aop/pom.xml
	modules/apache-license-gen/pom.xml
	modules/aws/pom.xml
	modules/clients/pom.xml
	modules/cloud/pom.xml
	modules/codegen/pom.xml
	modules/core/pom.xml
	modules/core/src/main/resources/ignite.properties
	modules/extdata/p2p/pom.xml
	modules/extdata/uri/modules/uri-dependency/pom.xml
	modules/extdata/uri/pom.xml
	modules/gce/pom.xml
	modules/geospatial/pom.xml
	modules/hadoop/pom.xml
	modules/hibernate/pom.xml
	modules/indexing/pom.xml
	modules/jcl/pom.xml
	modules/jms11/pom.xml
	modules/jta/pom.xml
	modules/kafka/pom.xml
	modules/log4j/pom.xml
	modules/log4j2/pom.xml
	modules/mesos/pom.xml
	modules/platform/pom.xml
	modules/platform/src/main/cpp/common/configure.ac
	modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
	modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
	modules/rest-http/pom.xml
	modules/scalar-2.10/pom.xml
	modules/scalar/pom.xml
	modules/schedule/pom.xml
	modules/schema-import/pom.xml
	modules/slf4j/pom.xml
	modules/spark-2.10/pom.xml
	modules/spark/pom.xml
	modules/spring/pom.xml
	modules/ssh/pom.xml
	modules/tools/pom.xml
	modules/urideploy/pom.xml
	modules/visor-console-2.10/pom.xml
	modules/visor-console/pom.xml
	modules/visor-plugins/pom.xml
	modules/web/pom.xml
	modules/yardstick/pom.xml
	modules/yarn/pom.xml
	modules/zookeeper/pom.xml
	pom.xml


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

Branch: refs/heads/ignite-843
Commit: 8b214d5d2cd4253ca02f6118b5c434314b569380
Parents: 2b16b57 55027dc
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Sep 7 10:39:13 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Sep 7 10:39:13 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  13 +-
 .../messaging/MessagingPingPongExample.java     |   6 +-
 .../CacheClientPortableQueryExample.java        |   3 -
 .../messaging/MessagingPingPongExample.java     |   2 +-
 .../examples/MessagingExamplesSelfTest.java     |   3 +-
 .../examples/MessagingExamplesSelfTest.java     |   3 +-
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../ignite/internal/MarshallerContextImpl.java  |  31 ++--
 .../distributed/dht/GridDhtLocalPartition.java  |  51 +++++-
 .../dht/GridPartitionedGetFuture.java           |  14 +-
 .../distributed/near/GridNearGetFuture.java     |  13 ++
 .../query/GridCacheDistributedQueryFuture.java  |   5 +-
 .../query/GridCacheDistributedQueryManager.java |   9 +-
 .../cache/query/GridCacheQueryManager.java      | 177 ++++++-------------
 .../cache/query/GridCacheQueryRequest.java      |  59 ++++++-
 .../visor/cache/VisorCacheStartTask.java        |   6 +-
 .../messages/TcpDiscoveryClientAckResponse.java |   2 +-
 .../cache/GridCacheIncrementTransformTest.java  |   2 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   5 -
 .../IgniteCacheNodeJoinAbstractTest.java        |  42 +++++
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 22 files changed, 276 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8b214d5d/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index 0de6294,7d2d4cc..5dbfce9
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--/*
++/*
   * Licensed to the Apache Software Foundation (ASF) under one or more
   * contributor license agreements.  See the NOTICE file distributed with
   * this work for additional information regarding copyright ownership.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8b214d5d/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index bfaf7f2,b8eb3b8..1ebcf24
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--/*
++/*
   * 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.


[13/39] ignite git commit: Merge branch 'ignite-1361-pingpong-example' of https://github.com/ashutakGG/incubator-ignite into ignite-1.4

Posted by ak...@apache.org.
Merge branch 'ignite-1361-pingpong-example' of https://github.com/ashutakGG/incubator-ignite into ignite-1.4


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

Branch: refs/heads/ignite-843
Commit: 5a37cefba9aea47e1f530425eafeac239a0c1d2e
Parents: cb78a14 cd81fff
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 14:08:31 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 14:08:31 2015 +0300

----------------------------------------------------------------------
 .../ignite/examples/messaging/MessagingPingPongExample.java    | 6 +++++-
 .../examples/java8/messaging/MessagingPingPongExample.java     | 2 +-
 .../org/apache/ignite/examples/MessagingExamplesSelfTest.java  | 3 +--
 .../ignite/java8/examples/MessagingExamplesSelfTest.java       | 3 ++-
 4 files changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[22/39] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.4' into ignite-1.4


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

Branch: refs/heads/ignite-843
Commit: fc0568991876cef5e36771cefa5e765af6a3f6c7
Parents: c631a37 d1c2db8
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Sep 4 17:28:34 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Sep 4 17:28:34 2015 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[19/39] ignite git commit: Fixed version.

Posted by ak...@apache.org.
Fixed version.


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

Branch: refs/heads/ignite-843
Commit: b41309c48abd9d9daad55c53d75dba7ea33893d7
Parents: 5a37cef
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 17:22:41 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 17:22:41 2015 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b41309c4/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java
index 0376d04..6505765 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java
@@ -30,7 +30,7 @@ public class TcpDiscoveryClientAckResponse extends TcpDiscoveryAbstractMessage {
     private static final long serialVersionUID = 0L;
 
     /** */
-    public static final IgniteProductVersion CLIENT_ACK_SINCE_VERSION = IgniteProductVersion.fromString("1.4.1");
+    public static final IgniteProductVersion CLIENT_ACK_SINCE_VERSION = IgniteProductVersion.fromString("1.4.0");
 
     /** */
     private final IgniteUuid msgId;


[25/39] ignite git commit: IGNITE-1047 - Fixed warning message in marshaller context

Posted by ak...@apache.org.
IGNITE-1047 - Fixed warning message in marshaller context


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

Branch: refs/heads/ignite-843
Commit: a58a16a951da4095b0fac0191fc009264a5bc858
Parents: 38baf59
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Fri Sep 4 15:08:18 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Sep 4 15:08:18 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/MarshallerContextImpl.java  | 31 +++++++++++---------
 1 file changed, 17 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a58a16a9/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 52d3b1e..5c9b54f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -17,17 +17,6 @@
 
 package org.apache.ignite.internal;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.Writer;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.CacheEntryListenerException;
-import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
@@ -37,6 +26,18 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.PluginProvider;
 
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
+import javax.cache.event.CacheEntryUpdatedListener;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.Writer;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
 /**
  * Marshaller context implementation.
  */
@@ -122,8 +123,10 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
         }
         catch (CachePartialUpdateCheckedException | GridCacheTryPutFailedException e) {
             if (++failedCnt > 10) {
-                U.quietAndWarn(log, e, "Failed to register marshalled class for more than 10 times in a row " +
-                    "(may affect performance).");
+                String msg = "Failed to register marshalled class for more than 10 times in a row " +
+                    "(may affect performance).";
+
+                U.quietAndWarn(log, msg, msg);
 
                 failedCnt = 0;
             }
@@ -206,4 +209,4 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
             }
         }
     }
-}
\ No newline at end of file
+}


[21/39] ignite git commit: Platforms version update

Posted by ak...@apache.org.
Platforms version update


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

Branch: refs/heads/ignite-843
Commit: c631a37669b85cc3d9fd209af820264155a0a32d
Parents: ecc492d
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Sep 4 17:28:13 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Sep 4 17:28:13 2015 +0300

----------------------------------------------------------------------
 pom.xml | 118 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 118 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c631a376/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 22ebad7..8a143cd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -737,6 +737,124 @@
                 </plugins>
             </build>
         </profile>
+
+        <profile>
+            <id>update-versions</id>
+            <!-- updates dotnet & cpp versions -->
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <version>1.7</version>
+                        <executions>
+                            <execution>
+                                <id>update-versions</id>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                                <phase>validate</phase>
+                                <configuration>
+                                    <target>
+                                        <script language="javascript">
+                                            function setClientVersion(ggVer, clientVer) {
+                                            var p = project.getProperty(ggVer);
+
+                                            if (java.util.regex.Pattern.matches(".*-p\\d+", p))
+                                            project.setProperty(clientVer, p.replace("-p", "."));
+                                            else
+                                            if (java.util.regex.Pattern.matches(".*-[a-zA-Z]+\\d+.*", p))
+                                            project.setProperty(clientVer, p.replaceAll("-[a-zA-Z]+(\\d+)?.*", ".$1"));
+                                            else
+                                            project.setProperty(clientVer, p);
+                                            }
+
+                                            function fix(dest, source) {
+                                            project.setProperty(dest, project.getProperty(source).replace("-SNAPSHOT",
+                                            ""));
+                                            }
+
+                                            fix('ignite.version.fixed', 'project.version');
+                                            fix('new.ignite.version.fixed', 'new.ignite.version');
+                                            setClientVersion('ignite.version.fixed', 'old.client.version');
+                                            setClientVersion('new.ignite.version.fixed', 'new.client.version');
+                                        </script>
+
+                                        <echo message="Update ignite.version in dotnet client"/>
+                                        <echo message="${new.client.version}"/>
+
+                                        <replaceregexp byline="true" encoding="UTF-8">
+                                            <regexp
+                                                pattern="(\[assembly:\s*Assembly\w*Version\w*\(&quot;)\d.\d.\d(.\d)?(&quot;\)\])"/>
+                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/AssemblyInfo.cs"/>
+                                                <include name="**/AssemblyInfo.cpp"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <echo message="Update ignite.version in cpp client"/>
+                                        <replaceregexp byline="true" encoding="UTF-8">
+                                            <regexp pattern="(AC_INIT.+\[)\d.\d.\d(.\d)?(\].+)"/>
+                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/configure.ac"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <replaceregexp byline="true" encoding="UTF-8">
+                                            <regexp pattern="(define GG_VERSION_STR_WIN &quot;)\d.\d.\d(.\d)?(&quot;)"/>
+                                            <substitution expression="\1${new.client.version}\3"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/resource.h"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <replaceregexp byline="true" encoding="UTF-16">
+                                            <regexp pattern="(Version&quot;, &quot;)\d.\d.\d.\d(&quot;)"/>
+                                            <substitution expression="\1${new.client.version}\2"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/Resource.rc"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <script language="javascript">
+                                            function setBinVersion(clientVer, binVer) {
+                                            var p = project.getProperty(clientVer).replace(".", ",");
+
+                                            if (p.split(',').length == 3)
+                                            project.setProperty(binVer, p + ',0');
+                                            else
+                                            project.setProperty(binVer, p);
+                                            }
+
+                                            setBinVersion('old.client.version', 'old.bin.version');
+                                            setBinVersion('new.client.version', 'new.bin.version');
+                                        </script>
+
+                                        <replaceregexp byline="true" encoding="UTF-8">
+                                            <regexp pattern="(define GG_VERSION_BIN_WIN )\d,\d,\d,\d"/>
+                                            <substitution expression="\1${new.bin.version}"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/resource.h"/>
+                                            </fileset>
+                                        </replaceregexp>
+
+                                        <replaceregexp byline="true" encoding="UTF-16">
+                                            <regexp pattern="(VERSION )\d,\d,\d,\d"/>
+                                            <substitution expression="\1${new.bin.version}"/>
+                                            <fileset dir="${basedir}/">
+                                                <include name="**/Resource.rc"/>
+                                            </fileset>
+                                        </replaceregexp>
+                                    </target>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
     </profiles>
 
     <build>


[31/39] ignite git commit: Added in the test workaround for IGNITE-1365.

Posted by ak...@apache.org.
Added in the test workaround for IGNITE-1365.


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

Branch: refs/heads/ignite-843
Commit: 1fe7e956995ca168e0a701d1194bd7b01adb8dec
Parents: 3e09aa3
Author: sboikov <sb...@gridgain.com>
Authored: Mon Sep 7 15:27:25 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Sep 7 15:27:25 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteDaemonNodeMarshallerCacheTest.java        | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1fe7e956/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
index 8e19fe9..d463a37 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
@@ -22,6 +22,9 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -104,6 +107,13 @@ public class IgniteDaemonNodeMarshallerCacheTest extends GridCommonAbstractTest
 
         awaitPartitionMapExchange();
 
+        // Workaround for IGNITE-1365.
+        IgniteInternalFuture<?> fut = ((IgniteKernal) daemonNode).context().cache().context().exchange().
+            affinityReadyFuture(new AffinityTopologyVersion(2, 0));
+
+        if (fut != null)
+            fut.get();
+
         TestClass1 res1 = daemonNode.compute(daemonNode.cluster().forRemotes()).call(new TestCallable1());
 
         assertNotNull(res1);


[03/39] ignite git commit: Enabled test.

Posted by ak...@apache.org.
Enabled test.


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

Branch: refs/heads/ignite-843
Commit: 1f6d2ff3e769a694721cd5349ad446dc5ae1d9dd
Parents: d223a70
Author: sboikov <sb...@gridgain.com>
Authored: Fri Sep 4 10:36:32 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Sep 4 10:36:32 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java  | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f6d2ff3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index ecb2698..b62cc48 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -42,11 +42,6 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     protected static int key = 0;
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1366");
-    }
-
-    /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 3;
     }


[05/39] ignite git commit: Merge branch 'ignite-1.4' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.4

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


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

Branch: refs/heads/ignite-843
Commit: 826cb2f3b4bf1637fa607dcd148c53335bfb7c3f
Parents: bf64c6e 1f6d2ff
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Sep 4 10:51:51 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Sep 4 10:51:51 2015 +0300

----------------------------------------------------------------------
 .../CacheClientPortablePutGetExample.java       |   4 +
 .../CacheClientPortableQueryExample.java        |   5 +
 .../configuration/CacheConfiguration.java       |  31 ++--
 .../apache/ignite/internal/IgniteKernal.java    |  10 +-
 .../dht/GridPartitionedGetFuture.java           |  14 +-
 .../distributed/near/GridNearGetFuture.java     |  13 ++
 .../query/GridCacheDistributedQueryFuture.java  |   5 +-
 .../query/GridCacheDistributedQueryManager.java |   9 +-
 .../cache/query/GridCacheQueryManager.java      | 177 ++++++-------------
 .../cache/query/GridCacheQueryRequest.java      |  59 ++++++-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   5 -
 .../IgniteCacheNodeJoinAbstractTest.java        |  42 +++++
 .../testsuites/IgniteCacheTestSuite3.java       |   3 +-
 13 files changed, 212 insertions(+), 165 deletions(-)
----------------------------------------------------------------------



[14/39] ignite git commit: ignite-1369: deleted info from the doc that is possible to run a server node from the console

Posted by ak...@apache.org.
ignite-1369: deleted info from the doc that is possible to run a server node from the console


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

Branch: refs/heads/ignite-843
Commit: 55cbf2773dadf6b66bd3192b73fd9f4241a97b5d
Parents: 83b6e5f
Author: Denis Magda <dm...@gridgain.com>
Authored: Fri Sep 4 14:13:00 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Fri Sep 4 14:13:00 2015 +0300

----------------------------------------------------------------------
 .../portable/datagrid/CacheClientPortableQueryExample.java        | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/55cbf277/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
index b0048fa..c77a6b2 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
@@ -47,9 +47,6 @@ import org.apache.ignite.portable.PortableObject;
  * <p>
  * Remote nodes should always be started with special configuration file which
  * enables the portable marshaller: {@code 'ignite.{sh|bat} examples/config/portable/example-ignite-portable.xml'}.
- * <p>
- * Alternatively you can run {@link ExamplePortableNodeStartup} in another JVM which will
- * start node with {@code examples/config/portable/example-ignite-portable.xml} configuration.
  */
 public class CacheClientPortableQueryExample {
     /** Organization cache name. */


[26/39] ignite git commit: ignite-1.4 Fixed start caches from Visor.

Posted by ak...@apache.org.
ignite-1.4 Fixed start caches from Visor.


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

Branch: refs/heads/ignite-843
Commit: e35aa50b6b06ebe27f85b8fdef0815b822e477da
Parents: a58a16a
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Mon Sep 7 10:45:11 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Mon Sep 7 10:45:11 2015 +0700

----------------------------------------------------------------------
 .../ignite/internal/visor/cache/VisorCacheStartTask.java       | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e35aa50b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
index 75cd1a9..758f39a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
@@ -139,12 +139,12 @@ public class VisorCacheStartTask extends
                 if (arg.near) {
                     NearCacheConfiguration nearCfg = Ignition.loadSpringBean(bais, "nearCacheConfiguration");
 
-                    ignite.createNearCache(VisorTaskUtils.unescapeName(arg.name()), nearCfg);
+                    ignite.getOrCreateNearCache(VisorTaskUtils.unescapeName(arg.name()), nearCfg);
                 }
                 else {
                     CacheConfiguration cacheCfg = Ignition.loadSpringBean(bais, "cacheConfiguration");
 
-                    ignite.createCache(cacheCfg);
+                    ignite.getOrCreateCache(cacheCfg);
                 }
             }
             catch (IOException e) {
@@ -159,4 +159,4 @@ public class VisorCacheStartTask extends
             return S.toString(VisorCacheStartJob.class, this);
         }
     }
-}
\ No newline at end of file
+}


[15/39] ignite git commit: Merge branch 'ignite-1.4' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.4-main

Posted by ak...@apache.org.
Merge branch 'ignite-1.4' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.4-main


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

Branch: refs/heads/ignite-843
Commit: 58597111c2eb79bd9b1581525ee1974c4afae6a6
Parents: 55cbf27 5a37cef
Author: Denis Magda <dm...@gridgain.com>
Authored: Fri Sep 4 14:13:19 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Fri Sep 4 14:13:19 2015 +0300

----------------------------------------------------------------------
 .../ignite/examples/messaging/MessagingPingPongExample.java    | 6 +++++-
 .../examples/java8/messaging/MessagingPingPongExample.java     | 2 +-
 .../org/apache/ignite/examples/MessagingExamplesSelfTest.java  | 3 +--
 .../ignite/java8/examples/MessagingExamplesSelfTest.java       | 3 ++-
 .../main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs  | 4 ++--
 5 files changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[10/39] ignite git commit: 1.4.0-SNAPSHOT

Posted by ak...@apache.org.
1.4.0-SNAPSHOT


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

Branch: refs/heads/ignite-843
Commit: 83b6e5f07125cabf119194a97c170330aef5fc1a
Parents: acfd99b
Author: Ignite Teamcity <ig...@apache.org>
Authored: Fri Sep 4 13:28:23 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Fri Sep 4 13:28:23 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                   | 2 +-
 modules/aop/pom.xml                                | 2 +-
 modules/apache-license-gen/pom.xml                 | 5 ++---
 modules/aws/pom.xml                                | 2 +-
 modules/clients/pom.xml                            | 2 +-
 modules/cloud/pom.xml                              | 2 +-
 modules/codegen/pom.xml                            | 2 +-
 modules/core/pom.xml                               | 2 +-
 modules/core/src/main/resources/ignite.properties  | 2 +-
 modules/extdata/p2p/pom.xml                        | 2 +-
 modules/extdata/uri/modules/uri-dependency/pom.xml | 2 +-
 modules/extdata/uri/pom.xml                        | 2 +-
 modules/gce/pom.xml                                | 2 +-
 modules/geospatial/pom.xml                         | 2 +-
 modules/hadoop/pom.xml                             | 2 +-
 modules/hibernate/pom.xml                          | 2 +-
 modules/indexing/pom.xml                           | 2 +-
 modules/jcl/pom.xml                                | 2 +-
 modules/jms11/pom.xml                              | 5 ++---
 modules/jta/pom.xml                                | 2 +-
 modules/kafka/pom.xml                              | 2 +-
 modules/log4j/pom.xml                              | 2 +-
 modules/log4j2/pom.xml                             | 2 +-
 modules/mesos/pom.xml                              | 2 +-
 modules/platform/pom.xml                           | 2 +-
 modules/rest-http/pom.xml                          | 2 +-
 modules/scalar-2.10/pom.xml                        | 2 +-
 modules/scalar/pom.xml                             | 2 +-
 modules/schedule/pom.xml                           | 2 +-
 modules/schema-import/pom.xml                      | 2 +-
 modules/slf4j/pom.xml                              | 2 +-
 modules/spark-2.10/pom.xml                         | 2 +-
 modules/spark/pom.xml                              | 2 +-
 modules/spring/pom.xml                             | 2 +-
 modules/ssh/pom.xml                                | 2 +-
 modules/tools/pom.xml                              | 2 +-
 modules/urideploy/pom.xml                          | 2 +-
 modules/visor-console-2.10/pom.xml                 | 2 +-
 modules/visor-console/pom.xml                      | 2 +-
 modules/visor-plugins/pom.xml                      | 2 +-
 modules/web/pom.xml                                | 2 +-
 modules/yardstick/pom.xml                          | 2 +-
 modules/yarn/pom.xml                               | 2 +-
 modules/zookeeper/pom.xml                          | 2 +-
 pom.xml                                            | 2 +-
 45 files changed, 47 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index a2ac452..64e8be9 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index a080b57..391f5b8 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index 2d15ad4..c17f98f 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -20,8 +20,7 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -32,7 +31,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 63d454d..a0ca6b0 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 303a274..61f6694 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index e735804..b7aa43b 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index a39da2f..89cadf5 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 57abba0..2f0dde7 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <repositories>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index c1c2d0f..50725fa 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.4.1-SNAPSHOT
+ignite.version=1.4.0-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index b48859b..aca3132 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/extdata/uri/modules/uri-dependency/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/modules/uri-dependency/pom.xml b/modules/extdata/uri/modules/uri-dependency/pom.xml
index f863bed..98d3fcf 100644
--- a/modules/extdata/uri/modules/uri-dependency/pom.xml
+++ b/modules/extdata/uri/modules/uri-dependency/pom.xml
@@ -27,7 +27,7 @@
     <artifactId>ignite-extdata-uri-dep</artifactId>
     <packaging>jar</packaging>
 
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <modelVersion>4.0.0</modelVersion>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index 0b4f92f..7613f0c 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -32,7 +32,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index b3fbb32..05f043f 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index 1330cf5..766114b 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 9fdfd99..839ffb8 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index 2d6d893..a3ae729 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 4bcadc8..e4713b4 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 0b2c48b..335b6c8 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/jms11/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jms11/pom.xml b/modules/jms11/pom.xml
index ff80007..dbb385d 100644
--- a/modules/jms11/pom.xml
+++ b/modules/jms11/pom.xml
@@ -20,8 +20,7 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -32,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jms11</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 7965dd9..c0ce618 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index ed192ab..5103a32 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-kafka</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index c3c4a84..aaf1dca 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/log4j2/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j2/pom.xml b/modules/log4j2/pom.xml
index 0628b47..2c0fd94 100644
--- a/modules/log4j2/pom.xml
+++ b/modules/log4j2/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j2</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 07bd13d..a652b60 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/platform/pom.xml
----------------------------------------------------------------------
diff --git a/modules/platform/pom.xml b/modules/platform/pom.xml
index 4bff370..87f0368 100644
--- a/modules/platform/pom.xml
+++ b/modules/platform/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-platform</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 8156887..c506df6 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 42fb9b9..627f2b4 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 982fcaf..ab03a5d 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index e64059a..207a97c 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 0ba8597..ef36af9 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 5cdf791..807ed0d 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 6c44006..283855e 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 1fb2753..f7b2110 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 4ed1fd7..bf093ef 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 7922f53..5b407ad 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 5e3465e..ca0d0f4 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 7dd0431..df2b0fa 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index a0edeaf..fa541dc 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index d6f373a..dc841bf 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index 04142fa..c0e5b5e 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 4432d81..d814f47 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 3f3774a..eda9429 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index 079a173..c93efa4 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index ba10101..3993bd5 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-zookeeper</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83b6e5f0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 96305a3..22ebad7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[18/39] ignite git commit: Release notes - ignite-1.4

Posted by ak...@apache.org.
Release notes - ignite-1.4


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

Branch: refs/heads/ignite-843
Commit: ecc492d7f6a8e37783920fd02b95aa08e38b82ac
Parents: b18a844
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Sep 4 14:44:02 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Sep 4 14:44:02 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ecc492d7/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index 3f3ac7b..661ebe4 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -4,6 +4,7 @@ Apache Ignite Release Notes
 Apache Ignite In-Memory Data Fabric 1.4
 ---------------------------------------
 * Added SSL support to communication and discovery.
+* Added portable objects API.
 * Added support for log4j2.
 * Added versioned entry to cache API.
 * Fixed IGNITE_HOME resolution with JBoss.