You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/11 15:12:27 UTC

[01/50] incubator-ignite git commit: ignite-389 Avoid backups filtering in case of partition scan query

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-745 ed2360877 -> 01bcfd8a7 (forced update)


ignite-389 Avoid backups filtering in case of partition scan query


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

Branch: refs/heads/ignite-745
Commit: 5d6bb532c7de35cfea7674b5fc1446e72a5fa985
Parents: f00a9e9
Author: agura <ag...@gridgain.com>
Authored: Thu May 28 18:30:08 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu May 28 18:30:08 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/query/ScanQuery.java    |  12 +-
 .../cache/query/GridCacheQueryAdapter.java      | 122 +++----------------
 .../cache/query/GridCacheQueryManager.java      |   9 +-
 3 files changed, 28 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5d6bb532/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
index f56b0c7..e6b69bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
@@ -46,6 +46,11 @@ public final class ScanQuery<K, V> extends Query<Cache.Entry<K, V>> {
         this(null, null);
     }
 
+    /**
+     * Creates partition scan query returning all entries for given partition.
+     *
+     * @param part Partition.
+     */
     public ScanQuery(int part) {
         this(part, null);
     }
@@ -62,9 +67,10 @@ public final class ScanQuery<K, V> extends Query<Cache.Entry<K, V>> {
     /**
      * Create scan query with filter.
      *
+     * @param part Partition.
      * @param filter Filter. If {@code null} then all entries will be returned.
      */
-    public ScanQuery(Integer part, @Nullable IgniteBiPredicate<K, V> filter) {
+    public ScanQuery(@Nullable Integer part, @Nullable IgniteBiPredicate<K, V> filter) {
         setPartition(part);
         setFilter(filter);
     }
@@ -96,7 +102,7 @@ public final class ScanQuery<K, V> extends Query<Cache.Entry<K, V>> {
      *
      * @return Partition number or {@code null}.
      */
-    public Integer getPartition() {
+    @Nullable public Integer getPartition() {
         return part;
     }
 
@@ -106,7 +112,7 @@ public final class ScanQuery<K, V> extends Query<Cache.Entry<K, V>> {
      *
      * @param part Partition number over which this query should iterate.
      */
-    public void setPartition(Integer part) {
+    public void setPartition(@Nullable Integer part) {
         this.part = part;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5d6bb532/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 6574f0a..2f32faa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -26,14 +26,15 @@ import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.query.*;
+import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.security.*;
+
 import org.jetbrains.annotations.*;
 
 import java.util.*;
-import java.util.concurrent.*;
 
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.*;
 
@@ -457,7 +458,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
             return (CacheQueryFuture<R>)(loc ? qryMgr.queryFieldsLocal(bean) :
                 qryMgr.queryFieldsDistributed(bean, nodes));
         else if (type == SCAN && part != null && nodes.size() > 1)
-            return new CacheQueryFallbackFuture(nodes, bean, qryMgr);
+            return new CacheQueryFallbackFuture<>(nodes, bean, qryMgr);
         else
             return (CacheQueryFuture<R>)(loc ? qryMgr.queryLocal(bean) : qryMgr.queryDistributed(bean, nodes));
     }
@@ -524,9 +525,10 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     /**
      * Wrapper for queries with fallback.
      */
-    private static class CacheQueryFallbackFuture<R> extends GridCacheQueryFutureAdapter<Object, Object, R> {
-        /** Target. */
-        private GridCacheQueryFutureAdapter<?, ?, R> fut;
+    private static class CacheQueryFallbackFuture<R> extends GridFutureAdapter<Collection<R>>
+        implements CacheQueryFuture<R> {
+        /** Query future. */
+        private volatile GridCacheQueryFutureAdapter<?, ?, R> fut;
 
         /** Backups. */
         private final Queue<ClusterNode> nodes;
@@ -559,13 +561,10 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
 
             ClusterNode node = F.first(F.view(nodes, IS_LOC_NODE));
 
-            if (node != null) {
+            if (node != null)
                 fallbacks.add(node);
 
-                fallbacks.addAll(F.view(nodes, F.not(IS_LOC_NODE)));
-            }
-            else
-                fallbacks.addAll(nodes);
+            fallbacks.addAll(node != null ? F.view(nodes, F.not(IS_LOC_NODE)) : nodes);
 
             return fallbacks;
         }
@@ -576,10 +575,11 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         private void init() {
             ClusterNode node = nodes.poll();
 
-            fut = (GridCacheQueryFutureAdapter<?, ?, R>)(node.isLocal() ? qryMgr.queryLocal(bean) :
-                qryMgr.queryDistributed(bean, Collections.singleton(node)));
+            GridCacheQueryFutureAdapter<?, ?, R> fut0 =
+                (GridCacheQueryFutureAdapter<?, ?, R>)(node.isLocal() ? qryMgr.queryLocal(bean) :
+                    qryMgr.queryDistributed(bean, Collections.singleton(node)));
 
-            fut.listen(new IgniteInClosure<IgniteInternalFuture<Collection<R>>>() {
+            fut0.listen(new IgniteInClosure<IgniteInternalFuture<Collection<R>>>() {
                 @Override public void apply(IgniteInternalFuture<Collection<R>> fut) {
                     try {
                         onDone(fut.get());
@@ -592,26 +592,8 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                     }
                 }
             });
-        }
-
-        /** {@inheritDoc} */
-        @Override protected boolean onPage(UUID nodeId, boolean last) {
-            return fut.onPage(nodeId, last);
-        }
 
-        /** {@inheritDoc} */
-        @Override protected void loadPage() {
-            fut.loadPage();
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-            fut.loadAllPages();
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void cancelQuery() throws IgniteCheckedException {
-            fut.cancelQuery();
+            fut = fut0;
         }
 
         /** {@inheritDoc} */
@@ -625,84 +607,8 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         }
 
         /** {@inheritDoc} */
-        @Override void clear() {
-            fut.clear();
-        }
-
-        /** {@inheritDoc} */
-        @Override public long endTime() {
-            return fut.endTime();
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void enqueue(Collection<?> col) {
-            fut.enqueue(col);
-        }
-
-        /** {@inheritDoc} */
-        @Override boolean fields() {
-            return fut.fields();
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<R> get() throws IgniteCheckedException {
-            return fut.get();
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteCheckedException {
-            return fut.get(timeout, unit);
-        }
-
-        /** {@inheritDoc} */
         @Override public R next() {
             return fut.next();
         }
-
-        /** {@inheritDoc} */
-        @Override public Collection<R> nextPage() throws IgniteCheckedException {
-            return fut.nextPage();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean onDone(Collection<R> res, Throwable err) {
-            return fut.onDone(res, err);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<R> nextPage(long timeout) throws IgniteCheckedException {
-            return fut.nextPage(timeout);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void onNodeLeft(UUID evtNodeId) {
-            fut.onNodeLeft(evtNodeId);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onPage(@Nullable UUID nodeId, @Nullable Collection<?> data,
-            @Nullable Throwable err, boolean finished) {
-            fut.onPage(nodeId, data, err, finished);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onTimeout() {
-            fut.onTimeout();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void printMemoryStats() {
-            fut.printMemoryStats();
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridCacheQueryBean query() {
-            return fut.query();
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgniteUuid timeoutId() {
-            return fut.timeoutId();
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5d6bb532/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 fac3d8f..652d62e 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
@@ -795,7 +795,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                             !locPart.reserve())
                             throw new GridDhtInvalidPartitionException(part, "Partition can't be reserved");
 
-
                         iter = new Iterator<K>() {
                             private Iterator<KeyCacheObject> iter0 = locPart.keySet().iterator();
 
@@ -1329,9 +1328,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                     K key = row.getKey();
 
-                    // Filter backups for SCAN queries. Other types are filtered in indexing manager.
-                    if (!cctx.isReplicated() && cctx.config().getCacheMode() != LOCAL && qry.type() == SCAN &&
-                        !incBackups && !cctx.affinity().primary(cctx.localNode(), key, topVer)) {
+                    // Filter backups for SCAN queries, if it isn't partition scan.
+                    // Other types are filtered in indexing manager.
+                    if (!cctx.isReplicated() && qry.type() == SCAN && qry.partition() == null &&
+                        cctx.config().getCacheMode() != LOCAL && !incBackups &&
+                        !cctx.affinity().primary(cctx.localNode(), key, topVer)) {
                         if (log.isDebugEnabled())
                             log.debug("Ignoring backup element [row=" + row +
                                 ", cacheMode=" + cctx.config().getCacheMode() + ", incBackups=" + incBackups +


[10/50] incubator-ignite git commit: # ignite-970

Posted by sb...@apache.org.
# ignite-970


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

Branch: refs/heads/ignite-745
Commit: 39ce1cbfe190a709c6a2711e42160727fb01ce02
Parents: 104a13f
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 14:37:42 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 14:37:42 2015 +0300

----------------------------------------------------------------------
 .../loadtests/communication/GridIoManagerBenchmark0.java      | 1 +
 .../ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java  | 7 +++++++
 .../ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java   | 7 +++++++
 3 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/39ce1cbf/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
index 422d608..ea5b716 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
@@ -455,6 +455,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
 
         spi.setTcpNoDelay(true);
         spi.setConnectionBufferSize(0);
+        spi.setSharedMemoryPort(-1);
 
         info("Comm SPI: " + spi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/39ce1cbf/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java
index 7cea968..a89e586 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.hadoop.fs.v2.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -197,6 +198,12 @@ public class IgniteHadoopFileSystemHandshakeSelfTest extends IgfsCommonAbstractT
 
         cfg.setDiscoverySpi(discoSpi);
 
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
         CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
 
         metaCacheCfg.setName("replicated");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/39ce1cbf/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
index 2c17ba9..6773366 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -86,6 +87,12 @@ public class IgniteHadoopFileSystemIpcCacheSelfTest extends IgfsCommonAbstractTe
 
         cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
 
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
         cnt++;
 
         return cfg;


[33/50] incubator-ignite git commit: Merge branch 'ignite-389' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-389

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


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

Branch: refs/heads/ignite-745
Commit: b812c0f165162a94f766bf8e4e12beb7e8170e5d
Parents: 9fca6b5 79ae323
Author: agura <ag...@gridgain.com>
Authored: Tue Jun 9 14:43:14 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jun 9 14:43:14 2015 +0300

----------------------------------------------------------------------
 modules/hadoop/pom.xml        | 1 +
 modules/spark-2.10/README.txt | 4 ----
 modules/spark/README.txt      | 4 ++++
 3 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[30/50] incubator-ignite git commit: IGNITE-389 - Rebuilt message.

Posted by sb...@apache.org.
IGNITE-389 - Rebuilt message.


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

Branch: refs/heads/ignite-745
Commit: 7e8f6485a61c7dbdb6cc167cccc5777366f274e6
Parents: 224cbcb
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 8 17:59:00 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 8 17:59:00 2015 -0700

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryRequest.java      | 40 ++++++++++----------
 1 file changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7e8f6485/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 7577954..2113e7a 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
@@ -112,7 +112,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
     private int taskHash;
 
     /** Partition. */
-    private Integer part;
+    private int part;
 
     /**
      * Required by {@link Externalizable}
@@ -218,7 +218,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
         this.clause = clause;
         this.clsName = clsName;
         this.keyValFilter = keyValFilter;
-        this.part = part;
+        this.part = part == null ? -1 : part;
         this.rdc = rdc;
         this.trans = trans;
         this.pageSize = pageSize;
@@ -426,7 +426,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
      * @return partition.
      */
     @Nullable public Integer partition() {
-        return part;
+        return part == -1 ? null : part;
     }
 
     /** {@inheritDoc} */
@@ -523,40 +523,41 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 writer.incrementState();
 
             case 16:
-                if (!writer.writeByteArray("rdcBytes", rdcBytes))
+                if (!writer.writeInt("part", part))
                     return false;
 
                 writer.incrementState();
 
             case 17:
-                if (!writer.writeUuid("subjId", subjId))
+                if (!writer.writeByteArray("rdcBytes", rdcBytes))
                     return false;
 
                 writer.incrementState();
 
             case 18:
-                if (!writer.writeInt("taskHash", taskHash))
+                if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
             case 19:
-                if (!writer.writeByteArray("transBytes", transBytes))
+                if (!writer.writeInt("taskHash", taskHash))
                     return false;
 
                 writer.incrementState();
 
             case 20:
-                if (!writer.writeByte("type", type != null ? (byte)type.ordinal() : -1))
+                if (!writer.writeByteArray("transBytes", transBytes))
                     return false;
 
                 writer.incrementState();
 
             case 21:
-                if (!writer.writeInt("part", part != null ? part : -1))
+                if (!writer.writeByte("type", type != null ? (byte)type.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
+
         }
 
         return true;
@@ -678,7 +679,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 reader.incrementState();
 
             case 16:
-                rdcBytes = reader.readByteArray("rdcBytes");
+                part = reader.readInt("part");
 
                 if (!reader.isLastRead())
                     return false;
@@ -686,7 +687,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 reader.incrementState();
 
             case 17:
-                subjId = reader.readUuid("subjId");
+                rdcBytes = reader.readByteArray("rdcBytes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -694,7 +695,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 reader.incrementState();
 
             case 18:
-                taskHash = reader.readInt("taskHash");
+                subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -702,7 +703,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 reader.incrementState();
 
             case 19:
-                transBytes = reader.readByteArray("transBytes");
+                taskHash = reader.readInt("taskHash");
 
                 if (!reader.isLastRead())
                     return false;
@@ -710,26 +711,25 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
                 reader.incrementState();
 
             case 20:
-                byte typeOrd;
-
-                typeOrd = reader.readByte("type");
+                transBytes = reader.readByteArray("transBytes");
 
                 if (!reader.isLastRead())
                     return false;
 
-                type = GridCacheQueryType.fromOrdinal(typeOrd);
-
                 reader.incrementState();
 
             case 21:
-                int part0 = reader.readInt("part");
+                byte typeOrd;
 
-                part = part0 == -1 ? null : part0;
+                typeOrd = reader.readByte("type");
 
                 if (!reader.isLastRead())
                     return false;
 
+                type = GridCacheQueryType.fromOrdinal(typeOrd);
+
                 reader.incrementState();
+
         }
 
         return true;


[05/50] incubator-ignite git commit: #IGNITE-389 - More functions on API.

Posted by sb...@apache.org.
#IGNITE-389 - More functions on API.


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

Branch: refs/heads/ignite-745
Commit: 8503dec1135755d4068f7538a044df7edbacdacc
Parents: 41afb37
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri May 29 10:58:12 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri May 29 10:58:12 2015 -0700

----------------------------------------------------------------------
 .../org/apache/ignite/spark/examples/IgniteProcessExample.scala   | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8503dec1/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
index e1d3326..db8b5a3 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.spark.examples
 
-import org.apache.ignite.configuration.CacheConfiguration
 import org.apache.ignite.spark.IgniteContext
 import org.apache.spark.rdd.RDD
 import org.apache.spark.{SparkContext, SparkConf}
@@ -47,8 +46,6 @@ object IgniteProcessExample {
         // SQL query
         ignite.fromCache("indexed").objectSql("Person", "age > ? and organizationId = ?", 20, 12).collect()
 
-        ignite.fromCache(new CacheConfiguration[Object, String]("ad"))
-
         // SQL fields query
         val sqlRes: RDD[Seq[Any]] = ignite.fromCache("indexed").sql("select name, age from Person where age > ?", 20)
     }


[46/50] incubator-ignite git commit: Idea code styles added

Posted by sb...@apache.org.
Idea code styles added


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

Branch: refs/heads/ignite-745
Commit: d36610f41d299a11986525e21c79c23355e426c3
Parents: 89a4f7c
Author: agura <ag...@gridgain.com>
Authored: Thu Jun 11 13:22:32 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Jun 11 13:22:32 2015 +0300

----------------------------------------------------------------------
 idea/ignite_codeStyle.xml | 147 +++++++++++++++++++++++++++++++++++++++++
 1 file changed, 147 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d36610f4/idea/ignite_codeStyle.xml
----------------------------------------------------------------------
diff --git a/idea/ignite_codeStyle.xml b/idea/ignite_codeStyle.xml
new file mode 100644
index 0000000..14faf54
--- /dev/null
+++ b/idea/ignite_codeStyle.xml
@@ -0,0 +1,147 @@
+<code_scheme name="ignite">
+  <option name="USE_SAME_INDENTS" value="true" />
+  <option name="IGNORE_SAME_INDENTS_FOR_LANGUAGES" value="true" />
+  <option name="OTHER_INDENT_OPTIONS">
+    <value>
+      <option name="INDENT_SIZE" value="4" />
+      <option name="CONTINUATION_INDENT_SIZE" value="4" />
+      <option name="TAB_SIZE" value="4" />
+      <option name="USE_TAB_CHARACTER" value="false" />
+      <option name="SMART_TABS" value="false" />
+      <option name="LABEL_INDENT_SIZE" value="0" />
+      <option name="LABEL_INDENT_ABSOLUTE" value="false" />
+      <option name="USE_RELATIVE_INDENTS" value="false" />
+    </value>
+  </option>
+  <option name="PREFER_LONGER_NAMES" value="false" />
+  <option name="USE_SINGLE_CLASS_IMPORTS" value="false" />
+  <option name="CLASS_COUNT_TO_USE_IMPORT_ON_DEMAND" value="0" />
+  <option name="NAMES_COUNT_TO_USE_IMPORT_ON_DEMAND" value="0" />
+  <option name="PACKAGES_TO_USE_IMPORT_ON_DEMAND">
+    <value />
+  </option>
+  <option name="IMPORT_LAYOUT_TABLE">
+    <value>
+      <package name="org.apache" withSubpackages="true" static="false" />
+      <package name="import org.gridgain" withSubpackages="true" static="false" />
+      <emptyLine />
+      <package name="" withSubpackages="true" static="false" />
+      <emptyLine />
+      <package name="javax" withSubpackages="true" static="false" />
+      <package name="java" withSubpackages="true" static="false" />
+      <emptyLine />
+      <package name="" withSubpackages="true" static="true" />
+    </value>
+  </option>
+  <option name="JD_ALIGN_PARAM_COMMENTS" value="false" />
+  <option name="JD_ALIGN_EXCEPTION_COMMENTS" value="false" />
+  <option name="JD_P_AT_EMPTY_LINES" value="false" />
+  <option name="JD_KEEP_INVALID_TAGS" value="false" />
+  <option name="JD_DO_NOT_WRAP_ONE_LINE_COMMENTS" value="true" />
+  <option name="KEEP_CONTROL_STATEMENT_IN_ONE_LINE" value="false" />
+  <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1" />
+  <option name="KEEP_BLANK_LINES_IN_CODE" value="1" />
+  <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1" />
+  <option name="ELSE_ON_NEW_LINE" value="true" />
+  <option name="WHILE_ON_NEW_LINE" value="true" />
+  <option name="CATCH_ON_NEW_LINE" value="true" />
+  <option name="FINALLY_ON_NEW_LINE" value="true" />
+  <option name="ALIGN_MULTILINE_PARAMETERS" value="false" />
+  <option name="ALIGN_MULTILINE_FOR" value="false" />
+  <option name="SPACE_AFTER_TYPE_CAST" value="false" />
+  <option name="SPACE_BEFORE_ARRAY_INITIALIZER_LBRACE" value="true" />
+  <option name="METHOD_PARAMETERS_WRAP" value="1" />
+  <option name="ARRAY_INITIALIZER_LBRACE_ON_NEXT_LINE" value="true" />
+  <option name="LABELED_STATEMENT_WRAP" value="1" />
+  <option name="WRAP_COMMENTS" value="true" />
+  <option name="METHOD_ANNOTATION_WRAP" value="1" />
+  <option name="CLASS_ANNOTATION_WRAP" value="1" />
+  <option name="FIELD_ANNOTATION_WRAP" value="1" />
+  <JavaCodeStyleSettings>
+    <option name="CLASS_NAMES_IN_JAVADOC" value="3" />
+  </JavaCodeStyleSettings>
+  <XML>
+    <option name="XML_LEGACY_SETTINGS_IMPORTED" value="true" />
+  </XML>
+  <ADDITIONAL_INDENT_OPTIONS fileType="haml">
+    <option name="INDENT_SIZE" value="2" />
+  </ADDITIONAL_INDENT_OPTIONS>
+  <codeStyleSettings language="Groovy">
+    <option name="KEEP_CONTROL_STATEMENT_IN_ONE_LINE" value="false" />
+    <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1" />
+    <option name="KEEP_BLANK_LINES_IN_CODE" value="1" />
+    <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1" />
+    <option name="ELSE_ON_NEW_LINE" value="true" />
+    <option name="CATCH_ON_NEW_LINE" value="true" />
+    <option name="FINALLY_ON_NEW_LINE" value="true" />
+    <option name="ALIGN_MULTILINE_PARAMETERS" value="false" />
+    <option name="ALIGN_MULTILINE_FOR" value="false" />
+    <option name="SPACE_AFTER_TYPE_CAST" value="false" />
+    <option name="METHOD_PARAMETERS_WRAP" value="1" />
+    <option name="METHOD_ANNOTATION_WRAP" value="1" />
+    <option name="CLASS_ANNOTATION_WRAP" value="1" />
+    <option name="FIELD_ANNOTATION_WRAP" value="1" />
+    <option name="PARENT_SETTINGS_INSTALLED" value="true" />
+    <indentOptions>
+      <option name="CONTINUATION_INDENT_SIZE" value="4" />
+    </indentOptions>
+  </codeStyleSettings>
+  <codeStyleSettings language="HOCON">
+    <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1" />
+    <option name="PARENT_SETTINGS_INSTALLED" value="true" />
+  </codeStyleSettings>
+  <codeStyleSettings language="JAVA">
+    <option name="KEEP_CONTROL_STATEMENT_IN_ONE_LINE" value="false" />
+    <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1" />
+    <option name="KEEP_BLANK_LINES_IN_CODE" value="1" />
+    <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1" />
+    <option name="ELSE_ON_NEW_LINE" value="true" />
+    <option name="WHILE_ON_NEW_LINE" value="true" />
+    <option name="CATCH_ON_NEW_LINE" value="true" />
+    <option name="FINALLY_ON_NEW_LINE" value="true" />
+    <option name="ALIGN_MULTILINE_PARAMETERS" value="false" />
+    <option name="ALIGN_MULTILINE_FOR" value="false" />
+    <option name="SPACE_AFTER_TYPE_CAST" value="false" />
+    <option name="SPACE_BEFORE_ARRAY_INITIALIZER_LBRACE" value="true" />
+    <option name="METHOD_PARAMETERS_WRAP" value="1" />
+    <option name="ARRAY_INITIALIZER_LBRACE_ON_NEXT_LINE" value="true" />
+    <option name="LABELED_STATEMENT_WRAP" value="1" />
+    <option name="METHOD_ANNOTATION_WRAP" value="1" />
+    <option name="CLASS_ANNOTATION_WRAP" value="1" />
+    <option name="FIELD_ANNOTATION_WRAP" value="1" />
+    <option name="PARENT_SETTINGS_INSTALLED" value="true" />
+    <indentOptions>
+      <option name="CONTINUATION_INDENT_SIZE" value="4" />
+    </indentOptions>
+  </codeStyleSettings>
+  <codeStyleSettings language="JSON">
+    <option name="KEEP_BLANK_LINES_IN_CODE" value="1" />
+    <option name="PARENT_SETTINGS_INSTALLED" value="true" />
+  </codeStyleSettings>
+  <codeStyleSettings language="Scala">
+    <option name="KEEP_BLANK_LINES_IN_DECLARATIONS" value="1" />
+    <option name="KEEP_BLANK_LINES_IN_CODE" value="1" />
+    <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="1" />
+    <option name="ELSE_ON_NEW_LINE" value="true" />
+    <option name="WHILE_ON_NEW_LINE" value="true" />
+    <option name="CATCH_ON_NEW_LINE" value="true" />
+    <option name="FINALLY_ON_NEW_LINE" value="true" />
+    <option name="ALIGN_MULTILINE_PARAMETERS" value="false" />
+    <option name="ALIGN_MULTILINE_FOR" value="false" />
+    <option name="METHOD_PARAMETERS_WRAP" value="1" />
+    <option name="METHOD_ANNOTATION_WRAP" value="1" />
+    <option name="CLASS_ANNOTATION_WRAP" value="1" />
+    <option name="FIELD_ANNOTATION_WRAP" value="1" />
+    <option name="PARENT_SETTINGS_INSTALLED" value="true" />
+    <indentOptions>
+      <option name="INDENT_SIZE" value="4" />
+      <option name="CONTINUATION_INDENT_SIZE" value="4" />
+      <option name="TAB_SIZE" value="4" />
+    </indentOptions>
+  </codeStyleSettings>
+  <codeStyleSettings language="XML">
+    <indentOptions>
+      <option name="CONTINUATION_INDENT_SIZE" value="4" />
+    </indentOptions>
+  </codeStyleSettings>
+</code_scheme>
\ No newline at end of file


[17/50] incubator-ignite git commit: 389-3

Posted by sb...@apache.org.
389-3


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

Branch: refs/heads/ignite-745
Commit: ac9dd30f640ccb1ca4708578437f090a41546cd7
Parents: aa3bb63
Author: avinogradov <av...@gridgain.com>
Authored: Wed Jun 3 16:59:16 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Wed Jun 3 16:59:16 2015 +0300

----------------------------------------------------------------------
 modules/scalar-2.10/pom.xml | 9 +++++++++
 modules/spark-2.10/pom.xml  | 2 ++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac9dd30f/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 23eb13e..e3d6125 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -83,6 +83,8 @@
     </dependencies>
 
     <build>
+        <sourceDirectory>../scalar/src/main/scala</sourceDirectory>
+
         <resources>
             <resource>
                 <directory>../scalar/src/main/scala</directory>
@@ -101,6 +103,13 @@
             </testResource>
         </testResources>
 
+        <plugins>
+            <plugin>
+                <groupId>net.alchim31.maven</groupId>
+                <artifactId>scala-maven-plugin</artifactId>
+            </plugin>
+        </plugins>
+
         <!-- TODO IGNITE-956 FIX scaladocs plugins-->
             <!--<plugin>-->
                 <!--<groupId>org.apache.maven.plugins</groupId>-->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac9dd30f/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 2cf0af5..2547f7c 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -90,6 +90,8 @@
     </dependencies>
 
     <build>
+        <sourceDirectory>../spark/src/main/scala</sourceDirectory>
+
         <resources>
             <resource>
                 <directory>../spark/src/main/scala</directory>


[09/50] incubator-ignite git commit: # ignite-970

Posted by sb...@apache.org.
# ignite-970


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

Branch: refs/heads/ignite-745
Commit: 104a13fd2118804e42b5035df4340d7374c36e82
Parents: 1dbdd42
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 14:25:08 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 14:25:08 2015 +0300

----------------------------------------------------------------------
 .../util/nio/GridShmemCommunicationClient.java  | 151 +++++++
 .../communication/tcp/TcpCommunicationSpi.java  | 414 ++++++++++++++++++-
 .../tcp/TcpCommunicationSpiMBean.java           |   8 +
 .../IgniteCacheMessageRecoveryAbstractTest.java |   1 +
 .../spi/GridTcpSpiForwardingSelfTest.java       |   1 +
 .../GridTcpCommunicationSpiAbstractTest.java    |  13 +
 ...mmunicationSpiConcurrentConnectSelfTest.java |   4 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |  11 +-
 ...pCommunicationSpiMultithreadedShmemTest.java |  28 ++
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |   1 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   1 +
 .../GridTcpCommunicationSpiShmemSelfTest.java   |  38 ++
 .../tcp/GridTcpCommunicationSpiTcpSelfTest.java |   7 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |   2 +
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  13 +
 ...oopSecondaryFileSystemConfigurationTest.java |  14 +
 .../hadoop/HadoopAbstractSelfTest.java          |   6 +
 17 files changed, 695 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
new file mode 100644
index 0000000..f3dc46f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util.nio;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.ipc.shmem.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.nio.*;
+import java.util.*;
+
+/**
+ *
+ */
+public class GridShmemCommunicationClient extends GridAbstractCommunicationClient {
+    /** */
+    private final IpcSharedMemoryClientEndpoint shmem;
+
+    /** */
+    private final ByteBuffer writeBuf;
+
+    /** */
+    private final MessageFormatter formatter;
+
+    /**
+     * @param metricsLsnr Metrics listener.
+     * @param port Shared memory IPC server port.
+     * @param connTimeout Connection timeout.
+     * @param log Logger.
+     * @param formatter Message formatter.
+     * @throws IgniteCheckedException If failed.
+     */
+    public GridShmemCommunicationClient(GridNioMetricsListener metricsLsnr,
+        int port,
+        long connTimeout,
+        IgniteLogger log,
+        MessageFormatter formatter)
+        throws IgniteCheckedException
+    {
+        super(metricsLsnr);
+
+        assert metricsLsnr != null;
+        assert port > 0 && port < 0xffff;
+        assert connTimeout >= 0;
+
+        shmem = new IpcSharedMemoryClientEndpoint(port, (int)connTimeout, log);
+
+        writeBuf = ByteBuffer.allocate(8 << 10);
+
+        writeBuf.order(ByteOrder.nativeOrder());
+
+        this.formatter = formatter;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void doHandshake(IgniteInClosure2X<InputStream, OutputStream> handshakeC)
+        throws IgniteCheckedException {
+        handshakeC.applyx(shmem.inputStream(), shmem.outputStream());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean close() {
+        boolean res = super.close();
+
+        if (res)
+            shmem.close();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void forceClose() {
+        super.forceClose();
+
+        // Do not call forceClose() here.
+        shmem.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void sendMessage(byte[] data, int len) throws IgniteCheckedException {
+        if (closed())
+            throw new IgniteCheckedException("Communication client was closed: " + this);
+
+        try {
+            shmem.outputStream().write(data, 0, len);
+
+            metricsLsnr.onBytesSent(len);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to send message to remote node: " + shmem, e);
+        }
+
+        markUsed();
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized boolean sendMessage(@Nullable UUID nodeId, Message msg)
+        throws IgniteCheckedException {
+        if (closed())
+            throw new IgniteCheckedException("Communication client was closed: " + this);
+
+        assert writeBuf.hasArray();
+
+        try {
+            int cnt = U.writeMessageFully(msg, shmem.outputStream(), writeBuf, formatter.writer());
+
+            metricsLsnr.onBytesSent(cnt);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to send message to remote node: " + shmem, e);
+        }
+
+        markUsed();
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendMessage(ByteBuffer data) throws IgniteCheckedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void flushIfNeeded(long timeout) throws IOException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridShmemCommunicationClient.class, this, super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 19e54c8..3768db5 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -25,15 +25,19 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.ipc.*;
+import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.internal.util.worker.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.communication.*;
+import org.apache.ignite.thread.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
@@ -139,6 +143,10 @@ import static org.apache.ignite.events.EventType.*;
 @IgniteSpiConsistencyChecked(optional = false)
 public class TcpCommunicationSpi extends IgniteSpiAdapter
     implements CommunicationSpi<Message>, TcpCommunicationSpiMBean {
+    /** IPC error message. */
+    public static final String OUT_OF_RESOURCES_TCP_MSG = "Failed to allocate shared memory segment " +
+        "(switching to TCP, may be slower).";
+
     /** Node attribute that is mapped to node IP addresses (value is <tt>comm.tcp.addrs</tt>). */
     public static final String ATTR_ADDRS = "comm.tcp.addrs";
 
@@ -148,12 +156,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Node attribute that is mapped to node port number (value is <tt>comm.tcp.port</tt>). */
     public static final String ATTR_PORT = "comm.tcp.port";
 
+    /** Node attribute that is mapped to node port number (value is <tt>comm.shmem.tcp.port</tt>). */
+    public static final String ATTR_SHMEM_PORT = "comm.shmem.tcp.port";
+
     /** Node attribute that is mapped to node's external addresses (value is <tt>comm.tcp.ext-addrs</tt>). */
     public static final String ATTR_EXT_ADDRS = "comm.tcp.ext-addrs";
 
     /** Default port which node sets listener to (value is <tt>47100</tt>). */
     public static final int DFLT_PORT = 47100;
 
+    /** Default port which node sets listener for shared memory connections (value is <tt>48100</tt>). */
+    public static final int DFLT_SHMEM_PORT = 48100;
+
     /** Default idle connection timeout (value is <tt>30000</tt>ms). */
     public static final long DFLT_IDLE_CONN_TIMEOUT = 30000;
 
@@ -293,7 +307,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     assert ses.accepted();
 
                     if (msg instanceof NodeIdMessage)
-                        sndId = U.bytesToUuid(((NodeIdMessage) msg).nodeIdBytes, 0);
+                        sndId = U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes, 0);
                     else {
                         assert msg instanceof HandshakeMessage : msg;
 
@@ -322,6 +336,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                     GridCommunicationClient oldClient = clients.get(sndId);
 
+                    boolean hasShmemClient = false;
+
                     if (oldClient != null) {
                         if (oldClient instanceof GridTcpNioCommunicationClient) {
                             if (log.isDebugEnabled())
@@ -333,6 +349,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                             return;
                         }
+                        else {
+                            assert oldClient instanceof GridShmemCommunicationClient;
+
+                            hasShmemClient = true;
+                        }
                     }
 
                     GridFutureAdapter<GridCommunicationClient> fut = new GridFutureAdapter<>();
@@ -359,10 +380,15 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                                 return;
                             }
+                            else {
+                                assert oldClient instanceof GridShmemCommunicationClient;
+
+                                hasShmemClient = true;
+                            }
                         }
 
                         boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
-                            new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, fut));
+                            new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
 
                         if (log.isDebugEnabled())
                             log.debug("Received incoming connection from remote node " +
@@ -371,7 +397,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         if (reserved) {
                             try {
                                 GridTcpNioCommunicationClient client =
-                                    connected(recoveryDesc, ses, rmtNode, msg0.received(), true);
+                                    connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
 
                                 fut.onDone(client);
                             }
@@ -393,11 +419,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         }
                         else {
                             boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
-                                new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, fut));
+                                new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
 
                             if (reserved) {
                                 GridTcpNioCommunicationClient client =
-                                    connected(recoveryDesc, ses, rmtNode, msg0.received(), true);
+                                    connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
 
                                 fut.onDone(client);
                             }
@@ -465,6 +491,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
              * @param node Node.
              * @param rcvCnt Number of received messages..
              * @param sndRes If {@code true} sends response for recovery handshake.
+             * @param createClient If {@code true} creates NIO communication client.
              * @return Client.
              */
             private GridTcpNioCommunicationClient connected(
@@ -472,7 +499,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 GridNioSession ses,
                 ClusterNode node,
                 long rcvCnt,
-                boolean sndRes) {
+                boolean sndRes,
+                boolean createClient) {
                 recovery.onHandshake(rcvCnt);
 
                 ses.recoveryDescriptor(recovery);
@@ -484,12 +512,16 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                 recovery.connected();
 
-                GridTcpNioCommunicationClient client = new GridTcpNioCommunicationClient(ses, log);
+                GridTcpNioCommunicationClient client = null;
+
+                if (createClient) {
+                    client = new GridTcpNioCommunicationClient(ses, log);
 
-                GridCommunicationClient oldClient = clients.putIfAbsent(node.id(), client);
+                    GridCommunicationClient oldClient = clients.putIfAbsent(node.id(), client);
 
-                assert oldClient == null : "Client already created [node=" + node + ", client=" + client +
+                    assert oldClient == null : "Client already created [node=" + node + ", client=" + client +
                         ", oldClient=" + oldClient + ", recoveryDesc=" + recovery + ']';
+                }
 
                 return client;
             }
@@ -517,22 +549,28 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 /** */
                 private final GridFutureAdapter<GridCommunicationClient> fut;
 
+                /** */
+                private final boolean createClient;
+
                 /**
                  * @param ses Incoming session.
                  * @param recoveryDesc Recovery descriptor.
                  * @param rmtNode Remote node.
                  * @param msg Handshake message.
+                 * @param createClient If {@code true} creates NIO communication client..
                  * @param fut Connect future.
                  */
                 ConnectClosure(GridNioSession ses,
                     GridNioRecoveryDescriptor recoveryDesc,
                     ClusterNode rmtNode,
                     HandshakeMessage msg,
+                    boolean createClient,
                     GridFutureAdapter<GridCommunicationClient> fut) {
                     this.ses = ses;
                     this.recoveryDesc = recoveryDesc;
                     this.rmtNode = rmtNode;
                     this.msg = msg;
+                    this.createClient = createClient;
                     this.fut = fut;
                 }
 
@@ -545,7 +583,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                                     msgFut.get();
 
                                     GridTcpNioCommunicationClient client =
-                                        connected(recoveryDesc, ses, rmtNode, msg.received(), false);
+                                        connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient);
 
                                     fut.onDone(client);
                                 }
@@ -594,6 +632,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Local port range. */
     private int locPortRange = DFLT_PORT_RANGE;
 
+    /** Local port which node uses to accept shared memory connections. */
+    private int shmemPort = DFLT_SHMEM_PORT;
+
     /** Allocate direct buffer or heap buffer. */
     private boolean directBuf = true;
 
@@ -635,6 +676,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** NIO server. */
     private GridNioServer<Message> nioSrvr;
 
+    /** Shared memory server. */
+    private IpcSharedMemoryServerEndpoint shmemSrv;
+
     /** {@code TCP_NODELAY} option value for created sockets. */
     private boolean tcpNoDelay = DFLT_TCP_NODELAY;
 
@@ -647,6 +691,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Socket write timeout. */
     private long sockWriteTimeout = DFLT_SOCK_WRITE_TIMEOUT;
 
+    /** Shared memory accept worker. */
+    private ShmemAcceptWorker shmemAcceptWorker;
+
     /** Idle client worker. */
     private IdleClientWorker idleClientWorker;
 
@@ -659,6 +706,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Recovery worker. */
     private RecoveryWorker recoveryWorker;
 
+    /** Shared memory workers. */
+    private final Collection<ShmemWorker> shmemWorkers = new ConcurrentLinkedDeque8<>();
+
     /** Clients. */
     private final ConcurrentMap<UUID, GridCommunicationClient> clients = GridConcurrentFactory.newMap();
 
@@ -668,6 +718,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Bound port. */
     private int boundTcpPort = -1;
 
+    /** Bound port for shared memory server. */
+    private int boundTcpShmemPort = -1;
+
     /** Count of selectors to use in TCP server. */
     private int selectorsCnt = DFLT_SELECTORS_CNT;
 
@@ -811,6 +864,25 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     }
 
     /**
+     * Sets local port to accept shared memory connections.
+     * <p>
+     * If set to {@code -1} shared memory communication will be disabled.
+     * <p>
+     * If not provided, default value is {@link #DFLT_SHMEM_PORT}.
+     *
+     * @param shmemPort Port number.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setSharedMemoryPort(int shmemPort) {
+        this.shmemPort = shmemPort;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getSharedMemoryPort() {
+        return shmemPort;
+    }
+
+    /**
      * Sets maximum idle connection timeout upon which a connection
      * to client will be closed.
      * <p>
@@ -1179,6 +1251,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         assertParameter(sockRcvBuf >= 0, "sockRcvBuf >= 0");
         assertParameter(sockSndBuf >= 0, "sockSndBuf >= 0");
         assertParameter(msgQueueLimit >= 0, "msgQueueLimit >= 0");
+        assertParameter(shmemPort > 0 || shmemPort == -1, "shmemPort > 0 || shmemPort == -1");
         assertParameter(reconCnt > 0, "reconnectCnt > 0");
         assertParameter(selectorsCnt > 0, "selectorsCnt > 0");
         assertParameter(minBufferedMsgCnt >= 0, "minBufferedMsgCnt >= 0");
@@ -1204,6 +1277,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         }
 
         try {
+            shmemSrv = resetShmemServer();
+        }
+        catch (IgniteCheckedException e) {
+            U.warn(log, "Failed to start shared memory communication server.", e);
+        }
+
+        try {
             // This method potentially resets local port to the value
             // local node was bound to.
             nioSrvr = resetNioServer();
@@ -1223,6 +1303,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 createSpiAttributeName(ATTR_ADDRS), addrs.get1(),
                 createSpiAttributeName(ATTR_HOST_NAMES), addrs.get2(),
                 createSpiAttributeName(ATTR_PORT), boundTcpPort,
+                createSpiAttributeName(ATTR_SHMEM_PORT), boundTcpShmemPort >= 0 ? boundTcpShmemPort : null,
                 createSpiAttributeName(ATTR_EXT_ADDRS), extAddrs);
         }
         catch (IOException | IgniteCheckedException e) {
@@ -1251,6 +1332,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("tcpNoDelay", tcpNoDelay));
             log.debug(configInfo("sockSndBuf", sockSndBuf));
             log.debug(configInfo("sockRcvBuf", sockRcvBuf));
+            log.debug(configInfo("shmemPort", shmemPort));
             log.debug(configInfo("msgQueueLimit", msgQueueLimit));
             log.debug(configInfo("minBufferedMsgCnt", minBufferedMsgCnt));
             log.debug(configInfo("connTimeout", connTimeout));
@@ -1272,6 +1354,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
 
+        if (shmemSrv != null) {
+            shmemAcceptWorker = new ShmemAcceptWorker(shmemSrv);
+
+            new IgniteThread(shmemAcceptWorker).start();
+        }
+
         nioSrvr.start();
 
         idleClientWorker = new IdleClientWorker();
@@ -1301,6 +1389,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
         spiCtx.registerPort(boundTcpPort, IgnitePortProtocol.TCP);
 
+        // SPI can start without shmem port.
+        if (boundTcpShmemPort > 0)
+            spiCtx.registerPort(boundTcpShmemPort, IgnitePortProtocol.TCP);
+
         spiCtx.addLocalEventListener(discoLsnr, EVT_NODE_LEFT, EVT_NODE_FAILED);
 
         ctxInitLatch.countDown();
@@ -1341,7 +1433,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         // If configured TCP port is busy, find first available in range.
         for (int port = locPort; port < locPort + locPortRange; port++) {
             try {
-                MessageFactory messageFactory = new MessageFactory() {
+                MessageFactory msgFactory = new MessageFactory() {
                     private MessageFactory impl;
 
                     @Nullable @Override public Message create(byte type) {
@@ -1354,7 +1446,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     }
                 };
 
-                MessageFormatter messageFormatter = new MessageFormatter() {
+                MessageFormatter msgFormatter = new MessageFormatter() {
                     private MessageFormatter impl;
 
                     @Override public MessageWriter writer() {
@@ -1376,7 +1468,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     }
                 };
 
-                GridDirectParser parser = new GridDirectParser(messageFactory, messageFormatter);
+                GridDirectParser parser = new GridDirectParser(msgFactory, msgFormatter);
 
                 IgnitePredicate<Message> skipRecoveryPred = new IgnitePredicate<Message>() {
                     @Override public boolean apply(Message msg) {
@@ -1403,7 +1495,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         .writeTimeout(sockWriteTimeout)
                         .filters(new GridNioCodecFilter(parser, log, true),
                             new GridConnectionBytesVerifyFilter(log))
-                        .messageFormatter(messageFormatter)
+                        .messageFormatter(msgFormatter)
                         .skipRecoveryPredicate(skipRecoveryPred)
                         .build();
 
@@ -1435,6 +1527,55 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             ", portRange=" + locPortRange + ", locHost=" + locHost + ']', lastEx);
     }
 
+    /**
+     * Creates new shared memory communication server.
+     * @return Server.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable private IpcSharedMemoryServerEndpoint resetShmemServer() throws IgniteCheckedException {
+        if (boundTcpShmemPort >= 0)
+            throw new IgniteCheckedException("Shared memory server was already created on port " + boundTcpShmemPort);
+
+        if (shmemPort == -1 || U.isWindows())
+            return null;
+
+        IgniteCheckedException lastEx = null;
+
+        // If configured TCP port is busy, find first available in range.
+        for (int port = shmemPort; port < shmemPort + locPortRange; port++) {
+            try {
+                IpcSharedMemoryServerEndpoint srv =
+                    new IpcSharedMemoryServerEndpoint(log, ignite.configuration().getNodeId(), gridName);
+
+                srv.setPort(port);
+
+                srv.omitOutOfResourcesWarning(true);
+
+                srv.start();
+
+                boundTcpShmemPort = port;
+
+                // Ack Port the TCP server was bound to.
+                if (log.isInfoEnabled())
+                    log.info("Successfully bound shared memory communication to TCP port [port=" + boundTcpShmemPort +
+                        ", locHost=" + locHost + ']');
+
+                return srv;
+            }
+            catch (IgniteCheckedException e) {
+                lastEx = e;
+
+                if (log.isDebugEnabled())
+                    log.debug("Failed to bind to local port (will try next port within range) [port=" + port +
+                        ", locHost=" + locHost + ']');
+            }
+        }
+
+        // If free port wasn't found.
+        throw new IgniteCheckedException("Failed to bind shared memory communication to any port within range [startPort=" +
+            locPort + ", portRange=" + locPortRange + ", locHost=" + locHost + ']', lastEx);
+    }
+
     /** {@inheritDoc} */
     @Override public void spiStop() throws IgniteSpiException {
         assert isNodeStopping();
@@ -1445,6 +1586,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (nioSrvr != null)
             nioSrvr.stop();
 
+        U.cancel(shmemAcceptWorker);
+        U.join(shmemAcceptWorker, log);
+
         U.interrupt(idleClientWorker);
         U.interrupt(clientFlushWorker);
         U.interrupt(sockTimeoutWorker);
@@ -1455,6 +1599,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         U.join(sockTimeoutWorker, log);
         U.join(recoveryWorker, log);
 
+        U.cancel(shmemWorkers);
+        U.join(shmemWorkers, log);
+
+        shmemWorkers.clear();
+
         // Force closing on stop (safety).
         for (GridCommunicationClient client : clients.values())
             client.forceClose();
@@ -1665,13 +1814,110 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     @Nullable protected GridCommunicationClient createNioClient(ClusterNode node) throws IgniteCheckedException {
         assert node != null;
 
-        if (getSpiContext().localNode() == null)
+        Integer shmemPort = node.attribute(createSpiAttributeName(ATTR_SHMEM_PORT));
+
+        ClusterNode locNode = getSpiContext().localNode();
+
+        if (locNode == null)
             throw new IgniteCheckedException("Failed to create NIO client (local node is stopping)");
 
+        // If remote node has shared memory server enabled and has the same set of MACs
+        // then we are likely to run on the same host and shared memory communication could be tried.
+        if (shmemPort != null && U.sameMacs(locNode, node)) {
+            try {
+                return createShmemClient(node, shmemPort);
+            }
+            catch (IgniteCheckedException e) {
+                if (e.hasCause(IpcOutOfSystemResourcesException.class))
+                    // Has cause or is itself the IpcOutOfSystemResourcesException.
+                    LT.warn(log, null, OUT_OF_RESOURCES_TCP_MSG);
+                else if (getSpiContext().node(node.id()) != null)
+                    LT.warn(log, null, e.getMessage());
+                else if (log.isDebugEnabled())
+                    log.debug("Failed to establish shared memory connection with local node (node has left): " +
+                        node.id());
+            }
+        }
+
         return createTcpClient(node);
     }
 
     /**
+     * @param node Node.
+     * @param port Port.
+     * @return Client.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable protected GridCommunicationClient createShmemClient(ClusterNode node, Integer port) throws IgniteCheckedException {
+        int attempt = 1;
+
+        int connectAttempts = 1;
+
+        long connTimeout0 = connTimeout;
+
+        while (true) {
+            GridCommunicationClient client;
+
+            try {
+                client = new GridShmemCommunicationClient(metricsLsnr,
+                    port,
+                    connTimeout,
+                    log,
+                    getSpiContext().messageFormatter());
+            }
+            catch (IgniteCheckedException e) {
+                // Reconnect for the second time, if connection is not established.
+                if (connectAttempts < 2 && X.hasCause(e, ConnectException.class)) {
+                    connectAttempts++;
+
+                    continue;
+                }
+
+                throw e;
+            }
+
+            try {
+                safeHandshake(client, null, node.id(), connTimeout0);
+            }
+            catch (HandshakeTimeoutException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Handshake timedout (will retry with increased timeout) [timeout=" + connTimeout0 +
+                        ", err=" + e.getMessage() + ", client=" + client + ']');
+
+                client.forceClose();
+
+                if (attempt == reconCnt || connTimeout0 > maxConnTimeout) {
+                    if (log.isDebugEnabled())
+                        log.debug("Handshake timedout (will stop attempts to perform the handshake) " +
+                            "[timeout=" + connTimeout0 + ", maxConnTimeout=" + maxConnTimeout +
+                            ", attempt=" + attempt + ", reconCnt=" + reconCnt +
+                            ", err=" + e.getMessage() + ", client=" + client + ']');
+
+                    throw e;
+                }
+                else {
+                    attempt++;
+
+                    connTimeout0 *= 2;
+
+                    continue;
+                }
+            }
+            catch (IgniteCheckedException | RuntimeException | Error e) {
+                if (log.isDebugEnabled())
+                    log.debug(
+                        "Caught exception (will close client) [err=" + e.getMessage() + ", client=" + client + ']');
+
+                client.forceClose();
+
+                throw e;
+            }
+
+            return client;
+        }
+    }
+
+    /**
      * Establish TCP connection to remote node and returns client.
      *
      * @param node Remote node.
@@ -2154,6 +2400,144 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     }
 
     /**
+     * This worker takes responsibility to shut the server down when stopping,
+     * No other thread shall stop passed server.
+     */
+    private class ShmemAcceptWorker extends GridWorker {
+        /** */
+        private final IpcSharedMemoryServerEndpoint srv;
+
+        /**
+         * @param srv Server.
+         */
+        ShmemAcceptWorker(IpcSharedMemoryServerEndpoint srv) {
+            super(gridName, "shmem-communication-acceptor", TcpCommunicationSpi.this.log);
+
+            this.srv = srv;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            try {
+                while (!Thread.interrupted()) {
+                    ShmemWorker e = new ShmemWorker(srv.accept());
+
+                    shmemWorkers.add(e);
+
+                    new IgniteThread(e).start();
+                }
+            }
+            catch (IgniteCheckedException e) {
+                if (!isCancelled())
+                    U.error(log, "Shmem server failed.", e);
+            }
+            finally {
+                srv.close();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            super.cancel();
+
+            srv.close();
+        }
+    }
+
+    /**
+     *
+     */
+    private class ShmemWorker extends GridWorker {
+        /** */
+        private final IpcEndpoint endpoint;
+
+        /**
+         * @param endpoint Endpoint.
+         */
+        private ShmemWorker(IpcEndpoint endpoint) {
+            super(gridName, "shmem-worker", TcpCommunicationSpi.this.log);
+
+            this.endpoint = endpoint;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            try {
+                MessageFactory msgFactory = new MessageFactory() {
+                    private MessageFactory impl;
+
+                    @Nullable @Override public Message create(byte type) {
+                        if (impl == null)
+                            impl = getSpiContext().messageFactory();
+
+                        assert impl != null;
+
+                        return impl.create(type);
+                    }
+                };
+
+                MessageFormatter msgFormatter = new MessageFormatter() {
+                    private MessageFormatter impl;
+
+                    @Override public MessageWriter writer() {
+                        if (impl == null)
+                            impl = getSpiContext().messageFormatter();
+
+                        assert impl != null;
+
+                        return impl.writer();
+                    }
+
+                    @Override public MessageReader reader(MessageFactory factory) {
+                        if (impl == null)
+                            impl = getSpiContext().messageFormatter();
+
+                        assert impl != null;
+
+                        return impl.reader(factory);
+                    }
+                };
+
+                IpcToNioAdapter<Message> adapter = new IpcToNioAdapter<>(
+                    metricsLsnr,
+                    log,
+                    endpoint,
+                    srvLsnr,
+                    msgFormatter,
+                    new GridNioCodecFilter(new GridDirectParser(msgFactory, msgFormatter), log, true),
+                    new GridConnectionBytesVerifyFilter(log)
+                );
+
+                adapter.serve();
+            }
+            finally {
+                shmemWorkers.remove(this);
+
+                endpoint.close();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            super.cancel();
+
+            endpoint.close();
+        }
+
+        /** @{@inheritDoc} */
+        @Override protected void cleanup() {
+            super.cleanup();
+
+            endpoint.close();
+        }
+
+        /** @{@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ShmemWorker.class, this);
+        }
+    }
+
+    /**
      *
      */
     private class IdleClientWorker extends IgniteSpiThread {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index 5c80e6e..3c6b64e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -44,6 +44,14 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
     public int getLocalPort();
 
     /**
+     * Gets local port for shared memory communication.
+     *
+     * @return Port number.
+     */
+    @MXBeanDescription("Shared memory endpoint port number.")
+    public int getSharedMemoryPort();
+
+    /**
      * Gets maximum number of local ports tried if all previously
      * tried ports are occupied.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
index 96abe5f..8031315 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
@@ -50,6 +50,7 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
         TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
 
         commSpi.setSocketWriteTimeout(1000);
+        commSpi.setSharedMemoryPort(-1);
 
         cfg.setCommunicationSpi(commSpi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
index ed9e0cf..744635d 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
@@ -115,6 +115,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         commSpi.setLocalAddress("127.0.0.1");
         commSpi.setLocalPort(commLocPort);
         commSpi.setLocalPortRange(1);
+        commSpi.setSharedMemoryPort(-1);
 
         cfg.setCommunicationSpi(commSpi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
index ea51aff..2d3f506 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
@@ -37,10 +37,23 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
     /** */
     public static final int IDLE_CONN_TIMEOUT = 2000;
 
+    /** */
+    private final boolean useShmem;
+
+    /**
+     * @param useShmem Use shared mem flag.
+     */
+    protected GridTcpCommunicationSpiAbstractTest(boolean useShmem) {
+        this.useShmem = useShmem;
+    }
+
     /** {@inheritDoc} */
     @Override protected CommunicationSpi getSpi(int idx) {
         TcpCommunicationSpi spi = new TcpCommunicationSpi();
 
+        if (!useShmem)
+            spi.setSharedMemoryPort(-1);
+
         spi.setLocalPort(GridTestUtils.getNextCommPort(getClass()));
         spi.setIdleConnectionTimeout(IDLE_CONN_TIMEOUT);
         spi.setTcpNoDelay(tcpNoDelay());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
index c038ee7..26e1120 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
@@ -181,8 +181,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
 
         if (load) {
             loadFut = GridTestUtils.runMultiThreadedAsync(new Callable<Long>() {
-                @Override
-                public Long call() throws Exception {
+                @Override public Long call() throws Exception {
                     long dummyRes = 0;
 
                     List<String> list = new ArrayList<>();
@@ -300,6 +299,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
         spi.setLocalPort(port++);
         spi.setIdleConnectionTimeout(60_000);
         spi.setConnectTimeout(10_000);
+        spi.setSharedMemoryPort(-1);
 
         return spi;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index e7ae957..9909d76 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -55,6 +55,9 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
     /** Message id sequence. */
     private AtomicLong msgId = new AtomicLong();
 
+    /** */
+    private final boolean useShmem;
+
     /** SPI resources. */
     private static final Collection<IgniteTestResources> spiRsrcs = new ArrayList<>();
 
@@ -80,9 +83,12 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
     }
 
     /**
+     * @param useShmem Use shared mem.
      */
-    public GridTcpCommunicationSpiMultithreadedSelfTest() {
+    protected GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
         super(false);
+
+        this.useShmem = useShmem;
     }
 
     /**
@@ -413,6 +419,9 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
     private CommunicationSpi<Message> newCommunicationSpi() {
         TcpCommunicationSpi spi = new TcpCommunicationSpi();
 
+        if (!useShmem)
+            spi.setSharedMemoryPort(-1);
+
         spi.setLocalPort(GridTestUtils.getNextCommPort(getClass()));
         spi.setIdleConnectionTimeout(IDLE_CONN_TIMEOUT);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedShmemTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedShmemTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedShmemTest.java
new file mode 100644
index 0000000..590b426
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedShmemTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+/**
+ *
+ */
+public class GridTcpCommunicationSpiMultithreadedShmemTest extends GridTcpCommunicationSpiMultithreadedSelfTest {
+    /** */
+    public GridTcpCommunicationSpiMultithreadedShmemTest() {
+        super(true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
index c0f0b11..1a4ba22 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
@@ -324,6 +324,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
         spi.setTcpNoDelay(true);
         spi.setAckSendThreshold(ackCnt);
         spi.setMessageQueueLimit(queueLimit);
+        spi.setSharedMemoryPort(-1);
 
         return spi;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
index 7463388..5d3afd9 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
@@ -608,6 +608,7 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
     protected TcpCommunicationSpi getSpi(int idx) {
         TcpCommunicationSpi spi = new TcpCommunicationSpi();
 
+        spi.setSharedMemoryPort(-1);
         spi.setLocalPort(port++);
         spi.setIdleConnectionTimeout(10_000);
         spi.setConnectTimeout(10_000);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiShmemSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiShmemSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiShmemSelfTest.java
new file mode 100644
index 0000000..5746a3c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiShmemSelfTest.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+import org.apache.ignite.testframework.junits.spi.*;
+
+/**
+ *
+ */
+@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
+public class GridTcpCommunicationSpiShmemSelfTest extends GridTcpCommunicationSpiAbstractTest {
+    /**
+     *
+     */
+    public GridTcpCommunicationSpiShmemSelfTest() {
+        super(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean tcpNoDelay() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiTcpSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiTcpSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiTcpSelfTest.java
index 32bced2..c27a86f 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiTcpSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiTcpSelfTest.java
@@ -24,6 +24,13 @@ import org.apache.ignite.testframework.junits.spi.*;
  */
 @GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
 public class GridTcpCommunicationSpiTcpSelfTest extends GridTcpCommunicationSpiAbstractTest {
+    /**
+     *
+     */
+    public GridTcpCommunicationSpiTcpSelfTest() {
+        super(false);
+    }
+
     /** {@inheritDoc} */
     @Override protected boolean tcpNoDelay() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
index 1d3bfcd..ff86bda 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
@@ -38,10 +38,12 @@ public class IgniteSpiCommunicationSelfTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpSelfTest.class));
         suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpNoDelayOffSelfTest.class));
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiShmemSelfTest.class));
 
         suite.addTest(new TestSuite(GridTcpCommunicationSpiStartStopSelfTest.class));
 
         suite.addTest(new TestSuite(GridTcpCommunicationSpiMultithreadedSelfTest.class));
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiMultithreadedShmemTest.class));
 
         suite.addTest(new TestSuite(GridTcpCommunicationSpiConfigSelfTest.class));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
index 9bcd5de..a1535ed 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
@@ -31,6 +31,8 @@ import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.communication.*;
+import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -186,6 +188,7 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
             cfg.setFileSystemConfiguration(igfsCfg);
             cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
             cfg.setLocalHost(U.getLocalHost().getHostAddress());
+            cfg.setCommunicationSpi(communicationSpi());
 
             G.start(cfg);
         }
@@ -211,6 +214,7 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
         cfg.setFileSystemConfiguration(igfsConfiguration(gridName));
         cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
         cfg.setLocalHost("127.0.0.1");
+        cfg.setCommunicationSpi(communicationSpi());
 
         return cfg;
     }
@@ -270,6 +274,15 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
         return cfg;
     }
 
+    /** @return Communication SPI. */
+    private CommunicationSpi communicationSpi() {
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        return commSpi;
+    }
+
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         G.stopAll(true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
index b089995..8c33679 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
@@ -31,6 +31,8 @@ import org.apache.ignite.internal.processors.hadoop.igfs.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.communication.*;
+import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -279,6 +281,8 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
         cfg.setFileSystemConfiguration(igfsCfg);
         cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
 
+        cfg.setCommunicationSpi(communicationSpi());
+
         G.start(cfg);
     }
 
@@ -314,6 +318,7 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
         cfg.setCacheConfiguration(cacheConfiguration());
         cfg.setFileSystemConfiguration(fsConfiguration(gridName));
         cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
+        cfg.setCommunicationSpi(communicationSpi());
 
         return cfg;
     }
@@ -371,6 +376,15 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
         return cfg;
     }
 
+    /** @return Communication SPI. */
+    private CommunicationSpi communicationSpi() {
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        return commSpi;
+    }
+
     /**
      * Case #SecondaryFileSystemProvider(null, path)
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/104a13fd/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
index af1a1e1..7fda532 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
@@ -94,6 +94,12 @@ public abstract class HadoopAbstractSelfTest extends GridCommonAbstractTest {
 
         cfg.setHadoopConfiguration(hadoopConfiguration(gridName));
 
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
         TcpDiscoverySpi discoSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
 
         discoSpi.setIpFinder(IP_FINDER);


[16/50] incubator-ignite git commit: 389-3

Posted by sb...@apache.org.
389-3


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

Branch: refs/heads/ignite-745
Commit: aa3bb6361663ef4397b79321046c37f310376f5f
Parents: bae1804
Author: avinogradov <av...@gridgain.com>
Authored: Wed Jun 3 13:44:03 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Wed Jun 3 13:44:03 2015 +0300

----------------------------------------------------------------------
 assembly/dependencies-fabric.xml                |   1 +
 modules/scalar-2.10/licenses/apache-2.0.txt     | 202 +++++++++++++++++++
 .../scalar-2.10/licenses/scala-bsd-license.txt  |  18 ++
 modules/spark-2.10/licenses/apache-2.0.txt      | 202 +++++++++++++++++++
 .../spark-2.10/licenses/scala-bsd-license.txt   |  18 ++
 modules/spark/licenses/apache-2.0.txt           | 202 +++++++++++++++++++
 modules/spark/licenses/scala-bsd-license.txt    |  18 ++
 7 files changed, 661 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa3bb636/assembly/dependencies-fabric.xml
----------------------------------------------------------------------
diff --git a/assembly/dependencies-fabric.xml b/assembly/dependencies-fabric.xml
index a294243..c6668f6 100644
--- a/assembly/dependencies-fabric.xml
+++ b/assembly/dependencies-fabric.xml
@@ -113,6 +113,7 @@
                 <exclude>org.apache.ignite:ignite-examples</exclude>
                 <exclude>org.apache.ignite:ignite-indexing</exclude>
                 <exclude>org.apache.ignite:ignite-visor-console</exclude>
+                <exclude>org.apache.ignite:ignite-visor-console_2.10</exclude>
                 <exclude>org.apache.ignite:ignite-visor-plugins</exclude>
                 <exclude>org.apache.ignite:ignite-visor-trial</exclude>
                 <exclude>org.apache.ignite:ignite-hadoop</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa3bb636/modules/scalar-2.10/licenses/apache-2.0.txt
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/licenses/apache-2.0.txt b/modules/scalar-2.10/licenses/apache-2.0.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/modules/scalar-2.10/licenses/apache-2.0.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa3bb636/modules/scalar-2.10/licenses/scala-bsd-license.txt
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/licenses/scala-bsd-license.txt b/modules/scalar-2.10/licenses/scala-bsd-license.txt
new file mode 100644
index 0000000..b2be111
--- /dev/null
+++ b/modules/scalar-2.10/licenses/scala-bsd-license.txt
@@ -0,0 +1,18 @@
+Copyright (c) 2002-2014 EPFL
+Copyright (c) 2011-2014 Typesafe, Inc.
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or
+other materials provided with the distribution. Neither the name of the EPFL nor the names of its contributors may be used to endorse or promote products
+derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO,
+THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS
+BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE
+GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
+DAMAGE.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa3bb636/modules/spark-2.10/licenses/apache-2.0.txt
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/licenses/apache-2.0.txt b/modules/spark-2.10/licenses/apache-2.0.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/modules/spark-2.10/licenses/apache-2.0.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa3bb636/modules/spark-2.10/licenses/scala-bsd-license.txt
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/licenses/scala-bsd-license.txt b/modules/spark-2.10/licenses/scala-bsd-license.txt
new file mode 100644
index 0000000..b2be111
--- /dev/null
+++ b/modules/spark-2.10/licenses/scala-bsd-license.txt
@@ -0,0 +1,18 @@
+Copyright (c) 2002-2014 EPFL
+Copyright (c) 2011-2014 Typesafe, Inc.
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or
+other materials provided with the distribution. Neither the name of the EPFL nor the names of its contributors may be used to endorse or promote products
+derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO,
+THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS
+BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE
+GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
+DAMAGE.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa3bb636/modules/spark/licenses/apache-2.0.txt
----------------------------------------------------------------------
diff --git a/modules/spark/licenses/apache-2.0.txt b/modules/spark/licenses/apache-2.0.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/modules/spark/licenses/apache-2.0.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aa3bb636/modules/spark/licenses/scala-bsd-license.txt
----------------------------------------------------------------------
diff --git a/modules/spark/licenses/scala-bsd-license.txt b/modules/spark/licenses/scala-bsd-license.txt
new file mode 100644
index 0000000..b2be111
--- /dev/null
+++ b/modules/spark/licenses/scala-bsd-license.txt
@@ -0,0 +1,18 @@
+Copyright (c) 2002-2014 EPFL
+Copyright (c) 2011-2014 Typesafe, Inc.
+
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met:
+
+Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or
+other materials provided with the distribution. Neither the name of the EPFL nor the names of its contributors may be used to endorse or promote products
+derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO,
+THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS
+BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE
+GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH
+DAMAGE.
\ No newline at end of file



[07/50] incubator-ignite git commit: IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389

Posted by sb...@apache.org.
IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389


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

Branch: refs/heads/ignite-745
Commit: f8910f6bcfe415b33558055a63445e20cacf7981
Parents: c527a04 541b1e0
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 1 15:04:05 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 1 15:04:05 2015 -0700

----------------------------------------------------------------------
 dev-tools/gradle/wrapper/gradle-wrapper.jar     |  Bin 51017 -> 0 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |   18 +-
 dev-tools/gradlew                               |  163 +-
 dev-tools/slurp.sh                              |    2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  100 +-
 examples/pom.xml                                |    2 +-
 modules/aop/pom.xml                             |    2 +-
 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 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    8 +-
 .../java/org/apache/ignite/IgniteServices.java  |    5 +-
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |  187 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   27 +-
 .../configuration/IgniteConfiguration.java      |   10 +-
 .../org/apache/ignite/igfs/IgfsUserContext.java |  119 +
 .../igfs/secondary/IgfsSecondaryFileSystem.java |    7 +
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   27 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   22 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../internal/igfs/common/IgfsMarshaller.java    |   35 +-
 .../igfs/common/IgfsPathControlRequest.java     |   22 +
 .../internal/managers/GridManagerAdapter.java   |   59 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   48 +
 .../discovery/GridDiscoveryManager.java         |  214 +-
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../processors/cache/CacheMetricsImpl.java      |  367 +-
 .../cache/CacheMetricsMXBeanImpl.java           |  100 +
 .../processors/cache/CacheMetricsSnapshot.java  |  380 +-
 .../cache/DynamicCacheChangeBatch.java          |   20 +-
 .../cache/DynamicCacheDescriptor.java           |    2 +
 .../processors/cache/GridCacheAdapter.java      |   29 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |    6 +-
 .../processors/cache/GridCacheEntryEx.java      |    6 +
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMapEntry.java     |    5 +
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   71 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |   74 +-
 .../processors/cache/GridCacheProxyImpl.java    |   12 +
 .../cache/GridCacheSharedContext.java           |    1 +
 .../processors/cache/GridCacheSwapManager.java  |  118 +-
 .../processors/cache/GridCacheUtils.java        |  234 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedCacheEntry.java  |    7 -
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |    8 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    6 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   22 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   78 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  313 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   28 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 +-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   83 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |    7 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   15 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   54 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  836 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../datastreamer/DataStreamerImpl.java          |    2 +
 .../internal/processors/hadoop/HadoopJob.java   |    2 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |    8 +-
 .../internal/processors/igfs/IgfsImpl.java      |    8 +-
 .../processors/igfs/IgfsIpcHandler.java         |  184 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |    9 +-
 .../internal/processors/igfs/IgfsServer.java    |    4 +-
 .../internal/processors/igfs/IgfsUtils.java     |   16 +
 .../service/GridServiceProcessor.java           |  121 +-
 .../ignite/internal/util/GridJavaProcess.java   |   30 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |    2 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |   80 +
 .../apache/ignite/plugin/PluginProvider.java    |   26 +-
 .../org/apache/ignite/services/Service.java     |    5 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   37 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   47 -
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |   40 +
 .../spi/discovery/DiscoverySpiListener.java     |    5 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1478 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4766 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  170 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5799 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   10 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   24 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   67 +
 .../messages/TcpDiscoveryClientPingRequest.java |   56 +
 .../TcpDiscoveryClientPingResponse.java         |   67 +
 .../TcpDiscoveryCustomEventMessage.java         |   41 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   28 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   43 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |    2 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    6 +
 .../tcp/messages/TcpDiscoveryPingResponse.java  |   15 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    8 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 .../ignite/internal/GridAffinitySelfTest.java   |    1 +
 .../internal/GridDiscoveryEventSelfTest.java    |    7 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    7 +-
 .../internal/GridProjectionAbstractTest.java    |   16 +
 .../GridProjectionForCachesSelfTest.java        |   11 +-
 .../internal/GridReleaseTypeSelfTest.java       |   77 +-
 .../apache/ignite/internal/GridSelfTest.java    |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   62 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |  122 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   46 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    1 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    3 +
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../processors/cache/GridCacheTestEntryEx.java  |    4 +
 .../GridCacheVariableTopologySelfTest.java      |   12 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |    2 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   29 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   13 +-
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   20 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   47 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |   25 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java |  412 ++
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   93 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../DataStreamerMultiThreadedSelfTest.java      |    2 -
 .../igfs/IgfsClientCacheSelfTest.java           |    3 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |  251 +
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    8 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1171 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../testframework/GridSpiTestContext.java       |   25 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/common/GridCommonAbstractTest.java   |   59 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    4 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |    1 +
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    3 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 modules/hadoop/pom.xml                          |    2 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  165 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  107 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   32 +-
 .../internal/processors/hadoop/HadoopUtils.java |   10 +-
 .../hadoop/SecondaryFileSystemProvider.java     |   53 +-
 .../hadoop/fs/HadoopDistributedFileSystem.java  |   91 -
 .../hadoop/fs/HadoopFileSystemsUtils.java       |   17 -
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |  204 +
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |    6 +
 .../hadoop/igfs/HadoopIgfsInProc.java           |  170 +-
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |    2 +-
 .../hadoop/igfs/HadoopIgfsOutProc.java          |   33 +-
 .../hadoop/igfs/HadoopIgfsWrapper.java          |   19 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |    4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   56 +-
 ...oopSecondaryFileSystemConfigurationTest.java |    4 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   63 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |    2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |    2 +
 .../hadoop/HadoopFileSystemsTest.java           |   23 +-
 .../collections/HadoopSkipListSelfTest.java     |    4 +-
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    4 +
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   16 +
 modules/jcl/pom.xml                             |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/mesos/README.txt                        |   28 +
 modules/mesos/licenses/apache-2.0.txt           |  202 +
 modules/mesos/licenses/jetty-epl-license.txt    |   69 +
 modules/mesos/pom.xml                           |  101 +
 .../apache/ignite/mesos/ClusterProperties.java  |  519 ++
 .../apache/ignite/mesos/IgniteFramework.java    |  119 +
 .../apache/ignite/mesos/IgniteScheduler.java    |  361 ++
 .../org/apache/ignite/mesos/IgniteTask.java     |   86 +
 .../org/apache/ignite/mesos/package-info.java   |   22 +
 .../ignite/mesos/resource/IgniteProvider.java   |  234 +
 .../ignite/mesos/resource/JettyServer.java      |   61 +
 .../ignite/mesos/resource/ResourceHandler.java  |  142 +
 .../ignite/mesos/resource/ResourceProvider.java |  120 +
 .../ignite/mesos/resource/package-info.java     |   22 +
 .../main/resources/ignite-default-config.xml    |   35 +
 .../org/apache/ignite/IgniteMesosTestSuite.java |   38 +
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |  464 ++
 modules/rest-http/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/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 .../licenses/jcraft-revised-bsd.txt             |   28 -
 modules/visor-console/pom.xml                   |    2 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web/pom.xml                             |    2 +-
 modules/yardstick/pom.xml                       |    2 +-
 parent/pom.xml                                  |    4 +
 pom.xml                                         |   15 +-
 scripts/git-format-patch.sh                     |   14 +-
 scripts/git-patch-prop.sh                       |    2 +-
 332 files changed, 23238 insertions(+), 11214 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 3c4c7d4,3065a2f..b61e4f0
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@@ -2633,9 -2637,9 +2653,9 @@@ public class GridCacheProcessor extend
  
          req.clientStartOnly(true);
  
 -        F.first(initiateCacheChanges(F.asList(req))).get();
 +        F.first(initiateCacheChanges(F.asList(req), false)).get();
  
-         IgniteCache cache = jCacheProxies.get(masked);
+         IgniteCacheProxy cache = jCacheProxies.get(masked);
  
          if (cache == null && failIfNotStarted)
              throw new IllegalArgumentException("Cache is not started: " + cacheName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 2f32faa,fab490f..6fc6436
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@@ -498,21 -469,17 +498,26 @@@ public class GridCacheQueryAdapter<T> i
       * @param prj Projection (optional).
       * @return Collection of data nodes in provided projection (if any).
       */
 -    private static Collection<ClusterNode> nodes(final GridCacheContext<?, ?> cctx, @Nullable final ClusterGroup prj) {
 +    private static Collection<ClusterNode> nodes(final GridCacheContext<?, ?> cctx,
 +        @Nullable final ClusterGroup prj, @Nullable final Integer part) {
          assert cctx != null;
  
 +        final AffinityTopologyVersion topVer = cctx.affinity().affinityTopologyVersion();
 +
+         Collection<ClusterNode> affNodes = CU.affinityNodes(cctx);
+ 
 -        if (prj == null)
++        if (prj == null && part == null)
+             return affNodes;
+ 
 +        final Set<ClusterNode> owners =
 +            part == null ? Collections.<ClusterNode>emptySet() : new HashSet<>(cctx.topology().owners(part, topVer));
 +
-         return F.view(CU.allNodes(cctx), new P1<ClusterNode>() {
+         return F.view(affNodes, new P1<ClusterNode>() {
              @Override public boolean apply(ClusterNode n) {
 -                return prj.node(n.id()) != null;
 +
 +                return cctx.discovery().cacheAffinityNode(n, cctx.name()) &&
 +                    (prj == null || prj.node(n.id()) != null) &&
 +                    (part == null || owners.contains(n));
              }
          });
      }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/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 652d62e,32e9d63..6e71ba7
--- 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
@@@ -769,138 -768,98 +769,138 @@@ public abstract class GridCacheQueryMan
  
          final boolean backups = qry.includeBackups() || cctx.isReplicated();
  
 -        final GridCloseableIteratorAdapter<IgniteBiTuple<K, V>> heapIt = new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
 -            private IgniteBiTuple<K, V> next;
 +        final GridCloseableIteratorAdapter<IgniteBiTuple<K, V>> heapIt =
 +            new GridCloseableIteratorAdapter<IgniteBiTuple<K, V>>() {
 +                private IgniteBiTuple<K, V> next;
  
 -            private IgniteCacheExpiryPolicy expiryPlc = cctx.cache().expiryPolicy(plc);
 +                private IgniteCacheExpiryPolicy expiryPlc = cctx.cache().expiryPolicy(plc);
  
 -            private Iterator<K> iter = backups ? prj.keySetx().iterator() : prj.primaryKeySet().iterator();
 +                private Iterator<K> iter;
  
 -            {
 -                advance();
 -            }
 +                private GridDhtLocalPartition locPart;
  
 -            @Override public boolean onHasNext() {
 -                return next != null;
 -            }
 +                {
 +                    Integer part = qry.partition();
  
 -            @Override public IgniteBiTuple<K, V> onNext() {
 -                if (next == null)
 -                    throw new NoSuchElementException();
 +                    if (part == null || dht == null)
-                         iter = backups ? prj.keySet().iterator() : prj.primaryKeySet().iterator();
++                        iter = backups ? prj.keySetx().iterator() : prj.primaryKeySet().iterator();
 +                    else if (part < 0 || part >= cctx.affinity().partitions())
 +                        iter = F.emptyIterator();
 +                    else {
 +                        AffinityTopologyVersion topVer = cctx.affinity().affinityTopologyVersion();
  
 -                IgniteBiTuple<K, V> next0 = next;
 +                        locPart = dht.topology().localPartition(part, topVer, false);
  
 -                advance();
 +                        if (locPart == null || (locPart.state() != OWNING && locPart.state() != RENTING) ||
 +                            !locPart.reserve())
 +                            throw new GridDhtInvalidPartitionException(part, "Partition can't be reserved");
  
 -                return next0;
 -            }
 +                        iter = new Iterator<K>() {
 +                            private Iterator<KeyCacheObject> iter0 = locPart.keySet().iterator();
  
 -            private void advance() {
 -                IgniteBiTuple<K, V> next0 = null;
 -
 -                while (iter.hasNext()) {
 -                    next0 = null;
 +                            @Override public boolean hasNext() {
 +                                return iter0.hasNext();
 +                            }
  
 -                    K key = iter.next();
 +                            @Override public K next() {
 +                                KeyCacheObject key = iter0.next();
  
 -                    V val;
 +                                return key.value(cctx.cacheObjectContext(), false);
 +                            }
  
 -                    try {
 -                        val = prj.localPeek(key, CachePeekModes.ONHEAP_ONLY, expiryPlc);
 +                            @Override public void remove() {
 +                                iter0.remove();
 +                            }
 +                        };
                      }
 -                    catch (IgniteCheckedException e) {
 -                        if (log.isDebugEnabled())
 -                            log.debug("Failed to peek value: " + e);
  
 -                        val = null;
 -                    }
 +                    advance();
 +                }
  
 -                    if (dht != null && expiryPlc != null && expiryPlc.readyToFlush(100)) {
 -                        dht.sendTtlUpdateRequest(expiryPlc);
 +                @Override public boolean onHasNext() {
 +                    return next != null;
 +                }
  
 -                        expiryPlc = cctx.cache().expiryPolicy(plc);
 -                    }
 +                @Override public IgniteBiTuple<K, V> onNext() {
 +                    if (next == null)
 +                        throw new NoSuchElementException();
  
 -                    if (val != null) {
 -                        next0 = F.t(key, val);
 +                    IgniteBiTuple<K, V> next0 = next;
  
 -                        if (checkPredicate(next0))
 -                            break;
 -                        else
 -                            next0 = null;
 -                    }
 +                    advance();
 +
 +                    return next0;
                  }
  
 -                next = next0 != null ?
 -                    new IgniteBiTuple<>(next0.getKey(), next0.getValue()) :
 -                    null;
 +                private void advance() {
 +                    IgniteBiTuple<K, V> next0 = null;
  
 -                if (next == null)
 -                    sendTtlUpdate();
 -            }
 +                    while (iter.hasNext()) {
 +                        next0 = null;
  
 -            @Override protected void onClose() {
 -                sendTtlUpdate();
 -            }
 +                        K key = iter.next();
 +
 +                        V val;
  
 -            private void sendTtlUpdate() {
 -                if (dht != null && expiryPlc != null) {
 -                    dht.sendTtlUpdateRequest(expiryPlc);
 +                        try {
 +                            val = prj.localPeek(key, CachePeekModes.ONHEAP_ONLY, expiryPlc);
 +                        }
 +                        catch (IgniteCheckedException e) {
 +                            if (log.isDebugEnabled())
 +                                log.debug("Failed to peek value: " + e);
 +
 +                            val = null;
 +                        }
  
 -                    expiryPlc = null;
 +                        if (dht != null && expiryPlc != null && expiryPlc.readyToFlush(100)) {
 +                            dht.sendTtlUpdateRequest(expiryPlc);
 +
 +                            expiryPlc = cctx.cache().expiryPolicy(plc);
 +                        }
 +
 +                        if (val != null) {
 +                            next0 = F.t(key, val);
 +
 +                            if (checkPredicate(next0))
 +                                break;
 +                            else
 +                                next0 = null;
 +                        }
 +                    }
 +
 +                    next = next0 != null ?
 +                        new IgniteBiTuple<>(next0.getKey(), next0.getValue()) :
 +                        null;
 +
 +                    if (next == null)
 +                        sendTtlUpdate();
                  }
 -            }
  
 -            private boolean checkPredicate(Map.Entry<K, V> e) {
 -                if (keyValFilter != null) {
 -                    Map.Entry<K, V> e0 = (Map.Entry<K, V>)cctx.unwrapPortableIfNeeded(e, qry.keepPortable());
 +                @Override protected void onClose() {
 +                    sendTtlUpdate();
  
 -                    return keyValFilter.apply(e0.getKey(), e0.getValue());
 +                    if (locPart != null)
 +                        locPart.release();
                  }
  
 -                return true;
 -            }
 -        };
 +                private void sendTtlUpdate() {
 +                    if (dht != null && expiryPlc != null) {
 +                        dht.sendTtlUpdateRequest(expiryPlc);
 +
 +                        expiryPlc = null;
 +                    }
 +                }
 +
 +                private boolean checkPredicate(Map.Entry<K, V> e) {
 +                    if (keyValFilter != null) {
 +                        Map.Entry<K, V> e0 = (Map.Entry<K, V>)cctx.unwrapPortableIfNeeded(e, qry.keepPortable());
 +
 +                        return keyValFilter.apply(e0.getKey(), e0.getValue());
 +                    }
 +
 +                    return true;
 +                }
 +            };
  
          final GridIterator<IgniteBiTuple<K, V>> it;
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f8910f6b/pom.xml
----------------------------------------------------------------------


[11/50] incubator-ignite git commit: # ignite-970

Posted by sb...@apache.org.
# ignite-970


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

Branch: refs/heads/ignite-745
Commit: d6f9b647ab92d822aebbef06315ccb0af41f8238
Parents: 39ce1cb
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 15:39:12 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 15:39:12 2015 +0300

----------------------------------------------------------------------
 modules/core/pom.xml                                               | 1 -
 .../tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java          | 2 +-
 2 files changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d6f9b647/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 370fe69..8c37a4f 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -129,7 +129,6 @@
             <groupId>org.gridgain</groupId>
             <artifactId>ignite-shmem</artifactId>
             <version>1.0.0</version>
-            <scope>test</scope>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d6f9b647/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index 9909d76..5d25299 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -85,7 +85,7 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
     /**
      * @param useShmem Use shared mem.
      */
-    protected GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
+    public GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
         super(false);
 
         this.useShmem = useShmem;


[40/50] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-389

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-389


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

Branch: refs/heads/ignite-745
Commit: 341721582e8788b78c6f84947b38fd5fa380e1e8
Parents: 980bf75 2454eb5
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Jun 10 13:20:27 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Jun 10 13:20:27 2015 -0700

----------------------------------------------------------------------
 .../client/memcache/MemcacheRestExample.java    |  32 ++--
 .../java/org/apache/ignite/IgniteCache.java     |  25 ++-
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../discovery/DiscoveryCustomMessage.java       |   6 +
 .../discovery/GridDiscoveryManager.java         |  32 ++++
 .../affinity/GridAffinityAssignmentCache.java   |   8 +-
 .../cache/DynamicCacheChangeBatch.java          |  19 ++-
 .../GridCachePartitionExchangeManager.java      |   2 +-
 .../processors/cache/IgniteInternalCache.java   |  27 +--
 .../continuous/AbstractContinuousMessage.java   |   9 +
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 .../internal/visor/util/VisorTaskUtils.java     |  16 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   6 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   2 +-
 .../RoundRobinGlobalLoadBalancer.java           |   2 +-
 .../distributed/IgniteCacheManyClientsTest.java | 169 +++++++++++++++++++
 .../DataStreamerMultinodeCreateCacheTest.java   |   6 +-
 .../ignite/testframework/GridTestUtils.java     |   2 +-
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 19 files changed, 305 insertions(+), 65 deletions(-)
----------------------------------------------------------------------



[45/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-998

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-998


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

Branch: refs/heads/ignite-745
Commit: 4ab301d2f0505ee3a67978392827dcccea60a58b
Parents: 46e6c48 89a4f7c
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 11 10:19:02 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 11 10:19:02 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   6 +
 assembly/dependencies-fabric.xml                |   1 +
 examples/pom.xml                                |  34 ++
 .../client/memcache/MemcacheRestExample.java    |  32 +-
 modules/core/pom.xml                            |   1 -
 .../java/org/apache/ignite/IgniteCache.java     |  25 +-
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../apache/ignite/cache/query/ScanQuery.java    |  45 +-
 .../configuration/CacheConfiguration.java       |   1 -
 .../affinity/GridAffinityAssignmentCache.java   |   8 +-
 .../processors/cache/GridCacheAdapter.java      |  15 +-
 .../GridCachePartitionExchangeManager.java      |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  30 +-
 .../processors/cache/GridCacheSwapManager.java  |  55 ++-
 .../processors/cache/IgniteCacheProxy.java      |  11 +-
 .../processors/cache/IgniteInternalCache.java   |  27 +-
 .../processors/cache/QueryCursorImpl.java       |  23 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   7 +
 .../processors/cache/query/CacheQuery.java      |   2 +-
 .../query/GridCacheDistributedQueryManager.java |   3 +
 .../cache/query/GridCacheQueryAdapter.java      | 147 ++++++-
 .../cache/query/GridCacheQueryManager.java      | 209 ++++++----
 .../cache/query/GridCacheQueryRequest.java      |  47 ++-
 .../processors/cache/query/QueryCursorEx.java   |   8 +
 .../datastructures/GridCacheSetImpl.java        |   4 +-
 .../processors/query/GridQueryIndexing.java     |   4 +-
 .../processors/query/GridQueryProcessor.java    |  18 +-
 .../service/GridServiceProcessor.java           |   2 +-
 .../ignite/internal/util/GridJavaProcess.java   |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |   4 +-
 .../shmem/IpcSharedMemoryClientEndpoint.java    |   2 +-
 .../ipc/shmem/IpcSharedMemoryNativeLoader.java  | 151 ++++++-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   2 +-
 .../util/nio/GridShmemCommunicationClient.java  | 146 +++++++
 .../communication/tcp/TcpCommunicationSpi.java  | 415 ++++++++++++++++++-
 .../tcp/TcpCommunicationSpiMBean.java           |   8 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   2 +-
 .../RoundRobinGlobalLoadBalancer.java           |   2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  15 +
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  19 +
 .../distributed/IgniteCacheManyClientsTest.java | 169 ++++++++
 .../IgniteCacheMessageRecoveryAbstractTest.java |   1 +
 ...achePartitionedPreloadLifecycleSelfTest.java |   2 +-
 ...CacheReplicatedPreloadLifecycleSelfTest.java |   6 +-
 .../GridCacheSwapScanQueryAbstractSelfTest.java | 112 +++--
 .../ipc/shmem/IgfsSharedMemoryTestServer.java   |   2 +
 .../IpcSharedMemoryCrashDetectionSelfTest.java  |   2 +-
 .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java |   2 +-
 .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java |   2 +-
 .../LoadWithCorruptedLibFileTestRunner.java     |   2 +-
 .../IpcSharedMemoryBenchmarkReader.java         |   2 +-
 .../IpcSharedMemoryBenchmarkWriter.java         |   2 +-
 .../communication/GridIoManagerBenchmark0.java  |   1 +
 .../spi/GridTcpSpiForwardingSelfTest.java       |   1 +
 .../GridTcpCommunicationSpiAbstractTest.java    |  13 +
 ...mmunicationSpiConcurrentConnectSelfTest.java |   4 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |  21 +-
 ...pCommunicationSpiMultithreadedShmemTest.java |  28 ++
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |   1 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   1 +
 .../GridTcpCommunicationSpiShmemSelfTest.java   |  38 ++
 .../tcp/GridTcpCommunicationSpiTcpSelfTest.java |   7 +
 .../ignite/testframework/GridTestUtils.java     |   2 +-
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |   2 +
 modules/hadoop/pom.xml                          |   1 +
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  13 +
 ...oopSecondaryFileSystemConfigurationTest.java |  14 +
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |   7 +
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   7 +
 .../hadoop/HadoopAbstractSelfTest.java          |   7 +
 .../processors/query/h2/IgniteH2Indexing.java   |  44 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   8 +-
 ...CacheScanPartitionQueryFallbackSelfTest.java | 408 ++++++++++++++++++
 .../cache/GridCacheCrossCacheQuerySelfTest.java |  12 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  77 +++-
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 modules/scalar-2.10/README.txt                  |   4 +
 modules/scalar-2.10/licenses/apache-2.0.txt     | 202 +++++++++
 .../scalar-2.10/licenses/scala-bsd-license.txt  |  18 +
 modules/scalar-2.10/pom.xml                     | 197 +++++++++
 modules/spark-2.10/README.txt                   |   4 +
 modules/spark-2.10/licenses/apache-2.0.txt      | 202 +++++++++
 .../spark-2.10/licenses/scala-bsd-license.txt   |  18 +
 modules/spark-2.10/pom.xml                      | 120 ++++++
 modules/spark/README.txt                        |   8 +
 modules/spark/licenses/apache-2.0.txt           | 202 +++++++++
 modules/spark/licenses/scala-bsd-license.txt    |  18 +
 modules/spark/pom.xml                           | 114 +++++
 .../org/apache/ignite/spark/IgniteContext.scala | 119 ++++++
 .../org/apache/ignite/spark/IgniteRDD.scala     | 244 +++++++++++
 .../apache/ignite/spark/JavaIgniteContext.scala |  63 +++
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |  99 +++++
 .../ignite/spark/impl/IgniteAbstractRDD.scala   |  39 ++
 .../ignite/spark/impl/IgnitePartition.scala     |  24 ++
 .../ignite/spark/impl/IgniteQueryIterator.scala |  27 ++
 .../apache/ignite/spark/impl/IgniteSqlRDD.scala |  41 ++
 .../spark/impl/JavaIgniteAbstractRDD.scala      |  34 ++
 .../ignite/spark/JavaIgniteRDDSelfTest.java     | 298 +++++++++++++
 .../scala/org/apache/ignite/spark/Entity.scala  |  28 ++
 .../org/apache/ignite/spark/IgniteRddSpec.scala | 231 +++++++++++
 modules/visor-console-2.10/README.txt           |   4 +
 modules/visor-console-2.10/pom.xml              | 174 ++++++++
 parent/pom.xml                                  |   4 +
 pom.xml                                         |  20 +-
 105 files changed, 4827 insertions(+), 292 deletions(-)
----------------------------------------------------------------------



[18/50] incubator-ignite git commit: IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389

Posted by sb...@apache.org.
IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389


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

Branch: refs/heads/ignite-745
Commit: 7ee51ba05dc65148835e1491b91a23695c940d12
Parents: 2aa1ace 97d0bc1
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Jun 3 15:18:44 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Jun 3 15:18:44 2015 -0700

----------------------------------------------------------------------
 assembly/dependencies-visor-console.xml         |    3 +
 .../hibernate/CacheHibernatePersonStore.java    |  202 +---
 .../hibernate/CacheHibernateStoreExample.java   |   17 +
 .../store/jdbc/CacheJdbcPersonStore.java        |  180 +--
 .../store/jdbc/CacheJdbcStoreExample.java       |   13 +
 .../store/spring/CacheSpringPersonStore.java    |  128 +++
 .../store/spring/CacheSpringStoreExample.java   |  143 +++
 .../datagrid/store/spring/package-info.java     |   22 +
 .../ignite/cache/eviction/EvictableEntry.java   |    7 +
 .../ignite/cache/eviction/EvictionPolicy.java   |    2 +
 .../cache/eviction/fifo/FifoEvictionPolicy.java |  117 +-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |   22 +
 .../cache/eviction/lru/LruEvictionPolicy.java   |  135 ++-
 .../eviction/lru/LruEvictionPolicyMBean.java    |   38 +
 .../eviction/random/RandomEvictionPolicy.java   |   10 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |  141 ++-
 .../sorted/SortedEvictionPolicyMBean.java       |   22 +
 .../apache/ignite/cache/query/QueryMetrics.java |    6 +-
 .../apache/ignite/cache/store/CacheStore.java   |    2 +
 .../ignite/cache/store/CacheStoreSession.java   |   22 +
 .../cache/store/CacheStoreSessionListener.java  |  133 +++
 .../jdbc/CacheJdbcStoreSessionListener.java     |  141 +++
 .../configuration/CacheConfiguration.java       |   32 +
 .../configuration/IgniteConfiguration.java      |   38 +-
 .../managers/communication/GridIoManager.java   |   12 +-
 .../discovery/GridDiscoveryManager.java         |    4 +-
 .../cache/CacheEvictableEntryImpl.java          |   31 +
 .../processors/cache/CacheMetricsImpl.java      |    4 +-
 .../processors/cache/GridCacheMapEntry.java     |   64 +-
 .../processors/cache/GridCacheProcessor.java    |   12 +-
 .../cache/GridCacheSharedContext.java           |   47 +-
 .../processors/cache/GridCacheUtils.java        |   54 +
 .../cache/distributed/dht/GridDhtGetFuture.java |   11 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   13 +-
 .../GridDhtPartitionsExchangeFuture.java        |   46 +-
 .../dht/preloader/GridDhtPreloader.java         |    9 +-
 .../local/atomic/GridLocalAtomicCache.java      |   25 +-
 .../cache/query/GridCacheQueryAdapter.java      |   12 +-
 .../cache/store/CacheOsStoreManager.java        |    1 -
 .../cache/store/CacheStoreManager.java          |    7 +-
 .../store/GridCacheStoreManagerAdapter.java     |  202 +++-
 .../cache/transactions/IgniteTxAdapter.java     |   33 +-
 .../transactions/IgniteTxLocalAdapter.java      |  142 ++-
 .../processors/query/GridQueryProcessor.java    |  311 ++++--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    2 +-
 ...cheStoreSessionListenerAbstractSelfTest.java |  315 ++++++
 ...heStoreSessionListenerLifecycleSelfTest.java |  395 +++++++
 .../CacheJdbcStoreSessionListenerSelfTest.java  |  175 +++
 .../cache/GridCacheAbstractFullApiSelfTest.java |   27 +
 .../cache/GridCacheAbstractMetricsSelfTest.java |   48 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |    8 +-
 ...idCacheConfigurationConsistencySelfTest.java |   14 +-
 .../cache/GridCacheMemoryModeSelfTest.java      |   23 +-
 .../processors/cache/GridCacheOffHeapTest.java  |    5 +-
 .../cache/GridCacheReloadSelfTest.java          |    6 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |    5 +-
 .../IgniteCrossCacheTxStoreSelfTest.java        |  147 ++-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   11 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |   11 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    6 +-
 .../GridCachePartitionedEvictionSelfTest.java   |   11 +-
 .../GridCachePartitionedFullApiSelfTest.java    |   32 +
 ...ePartitionedMultiThreadedPutGetSelfTest.java |    6 +-
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |    2 +-
 .../cache/eviction/EvictionAbstractTest.java    | 1056 ++++++++++++++++++
 .../GridCacheBatchEvictUnswapSelfTest.java      |    5 +-
 ...heConcurrentEvictionConsistencySelfTest.java |   82 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |   29 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |    5 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |   11 +-
 .../eviction/GridCacheEvictionAbstractTest.java |  484 --------
 .../GridCacheEvictionTouchSelfTest.java         |   22 +-
 .../cache/eviction/GridCacheMockEntry.java      |    5 +
 .../fifo/FifoEvictionPolicySelfTest.java        |  262 +++++
 ...ridCacheFifoBatchEvictionPolicySelfTest.java |  384 -------
 .../GridCacheFifoEvictionPolicySelfTest.java    |  372 ------
 .../lru/GridCacheLruEvictionPolicySelfTest.java |  417 -------
 .../GridCacheLruNearEvictionPolicySelfTest.java |  136 ---
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |  168 ---
 .../eviction/lru/LruEvictionPolicySelfTest.java |  353 ++++++
 .../lru/LruNearEvictionPolicySelfTest.java      |  140 +++
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |  172 +++
 .../GridCacheRandomEvictionPolicySelfTest.java  |  258 -----
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |    6 +
 .../random/RandomEvictionPolicySelfTest.java    |  357 ++++++
 ...dCacheSortedBatchEvictionPolicySelfTest.java |  385 -------
 ...acheSortedEvictionPolicyPerformanceTest.java |  135 ---
 .../GridCacheSortedEvictionPolicySelfTest.java  |  373 -------
 .../SortedEvictionPolicyPerformanceTest.java    |  134 +++
 .../sorted/SortedEvictionPolicySelfTest.java    |  266 +++++
 .../loadtests/GridCacheMultiNodeLoadTest.java   |    5 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |    6 +-
 .../loadtests/hashmap/GridCacheTestContext.java |    3 +-
 .../swap/GridSwapEvictAllBenchmark.java         |    6 +-
 .../junits/cache/TestCacheSession.java          |   18 +
 .../cache/TestThreadLocalCacheSession.java      |   15 +
 .../junits/common/GridCommonAbstractTest.java   |   24 +
 .../IgniteCacheEvictionSelfTestSuite.java       |   14 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    3 +
 .../CacheHibernateStoreSessionListener.java     |  216 ++++
 ...heHibernateStoreSessionListenerSelfTest.java |  228 ++++
 .../testsuites/IgniteHibernateTestSuite.java    |    2 +
 .../GridCacheOffheapIndexEntryEvictTest.java    |  200 ++++
 .../cache/GridCacheOffheapIndexGetSelfTest.java |   18 +-
 .../cache/GridCacheQueryMetricsSelfTest.java    |   84 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    6 +-
 ...QueryMultiThreadedOffHeapTieredSelfTest.java |   37 +
 ...eQueryMultiThreadedOffHeapTiredSelfTest.java |   37 -
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   11 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    6 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +-
 .../IgniteCacheWithIndexingTestSuite.java       |    1 +
 modules/spring/pom.xml                          |   14 +
 .../spring/CacheSpringStoreSessionListener.java |  207 ++++
 ...CacheSpringStoreSessionListenerSelfTest.java |  197 ++++
 .../testsuites/IgniteSpringTestSuite.java       |    3 +
 scripts/git-apply-patch.sh                      |    8 +-
 scripts/git-format-patch.sh                     |    6 +-
 scripts/git-patch-functions.sh                  |   36 +-
 119 files changed, 7299 insertions(+), 3998 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ee51ba0/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ee51ba0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ee51ba0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ee51ba0/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 31337ae,ed8e1e2..1be2a36
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@@ -675,37 -709,24 +709,27 @@@ public class GridQueryProcessor extend
              throw new IllegalStateException("Failed to execute query (grid is stopping).");
  
          try {
-             String space = cctx.name();
-             String sql = qry.getSql();
-             Object[] args = qry.getArgs();
- 
-             final GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
- 
-             if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
-                 ctx.event().record(new CacheQueryExecutedEvent<>(
-                         ctx.discovery().localNode(),
-                         "SQL query executed.",
-                         EVT_CACHE_QUERY_EXECUTED,
-                         CacheQueryType.SQL.name(),
-                         null,
-                         null,
-                         sql,
-                         null,
-                         null,
-                         args,
-                         null,
-                         null));
-             }
+             return executeQuery(cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
+                 @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                     String space = cctx.name();
+                     String sql = qry.getSql();
+                     Object[] args = qry.getArgs();
  
-             QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(new Iterable<List<?>>() {
-                 @Override public Iterator<List<?>> iterator() {
-                     return new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepPortable());
-                 }
-             });
 -                    GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
++                    final GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
+ 
+                     sendQueryExecutedEvent(sql, args);
  
-             cursor.fieldsMeta(res.metaData());
 -                    QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
 -                        new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepPortable()));
++                    QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(new Iterable<List<?>>() {
++                        @Override public Iterator<List<?>> iterator() {
++                            return new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepPortable());
++                        }
++                    });
  
-             return cursor;
+                     cursor.fieldsMeta(res.metaData());
+ 
+                     return cursor;
+                 }
+             });
          }
          catch (IgniteCheckedException e) {
              throw new CacheException(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ee51ba0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7ee51ba0/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------


[31/50] incubator-ignite git commit: GG-10406

Posted by sb...@apache.org.
GG-10406


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

Branch: refs/heads/ignite-745
Commit: 79ae3230cd36866452959a42ba1b9b60bd83a122
Parents: 7e8f648
Author: avinogradov <av...@gridgain.com>
Authored: Tue Jun 9 12:54:03 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Jun 9 12:54:03 2015 +0300

----------------------------------------------------------------------
 modules/hadoop/pom.xml        | 1 +
 modules/spark-2.10/README.txt | 4 ----
 modules/spark/README.txt      | 4 ++++
 3 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/79ae3230/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index fe11389..4c57df3 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -96,6 +96,7 @@
         <dependency>
             <groupId>org.gridgain</groupId>
             <artifactId>ignite-shmem</artifactId>
+            <scope>test</scope>
             <version>1.0.0</version>
         </dependency>
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/79ae3230/modules/spark-2.10/README.txt
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/README.txt b/modules/spark-2.10/README.txt
deleted file mode 100644
index 29d3930..0000000
--- a/modules/spark-2.10/README.txt
+++ /dev/null
@@ -1,4 +0,0 @@
-Apache Ignite Spark Module
----------------------------
-
-Apache Ignite Spark module to be build with Scala 2.10.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/79ae3230/modules/spark/README.txt
----------------------------------------------------------------------
diff --git a/modules/spark/README.txt b/modules/spark/README.txt
new file mode 100644
index 0000000..5678441
--- /dev/null
+++ b/modules/spark/README.txt
@@ -0,0 +1,4 @@
+Apache Ignite Spark Module
+---------------------------
+
+Apache Ignite Spark module.


[43/50] incubator-ignite git commit: #IGNITE-389 - Minor

Posted by sb...@apache.org.
#IGNITE-389 - Minor


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

Branch: refs/heads/ignite-745
Commit: f149c8205191a5437bf6532807c2a1b275b67b88
Parents: 9926fb8
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Jun 10 15:58:58 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Jun 10 15:58:58 2015 -0700

----------------------------------------------------------------------
 .../internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java      | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f149c820/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
index 8c345f8..5b3274d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
@@ -142,7 +142,8 @@ public class IpcSharedMemoryNativeLoader {
                 return;
 
             try {
-                U.quietAndWarn(log, "Failed to load 'igniteshmem' library from classpath. Will try to load it from IGNITE_HOME.");
+                if (log != null)
+                    LT.warn(log, null, "Failed to load 'igniteshmem' library from classpath. Will try to load it from IGNITE_HOME.");
 
                 String igniteHome = X.resolveIgniteHome();
 


[39/50] incubator-ignite git commit: # ignite-sprint-5 increased affinity history size

Posted by sb...@apache.org.
# ignite-sprint-5 increased affinity history size


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

Branch: refs/heads/ignite-745
Commit: 2454eb58ae60718f8fcf55eccb7a4fc7016e0bcf
Parents: addc91b
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 10 17:43:02 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 10 17:43:02 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../affinity/GridAffinityAssignmentCache.java   |   5 +-
 .../GridCachePartitionExchangeManager.java      |   2 +-
 .../distributed/IgniteCacheManyClientsTest.java | 169 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 5 files changed, 178 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 439ea2d..b166f39 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -337,6 +337,9 @@ public final class IgniteSystemProperties {
      */
     public static final String IGNITE_SQL_MERGE_TABLE_MAX_SIZE = "IGNITE_SQL_MERGE_TABLE_MAX_SIZE";
 
+    /** Maximum size for affinity assignment history. */
+    public static final String IGNITE_AFFINITY_HISTORY_SIZE = "IGNITE_AFFINITY_HISTORY_SIZE";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 47f222e..6989385 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -408,9 +408,10 @@ public class GridAffinityAssignmentCache {
                 throw new IllegalStateException("Getting affinity for topology version earlier than affinity is " +
                     "calculated [locNodeId=" + ctx.localNodeId() +
                     ", cache=" + cacheName +
-                    ", history=" + affCache.keySet() +
                     ", topVer=" + topVer +
-                    ", head=" + head.get().topologyVersion() + ']');
+                    ", head=" + head.get().topologyVersion() +
+                    ", history=" + affCache.keySet() +
+                    ']');
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 3236bb5..3df45cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -59,7 +59,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     private static final int EXCHANGE_HISTORY_SIZE = 1000;
 
     /** Cleanup history size. */
-    public static final int EXCH_FUT_CLEANUP_HISTORY_SIZE = 10;
+    public static final int EXCH_FUT_CLEANUP_HISTORY_SIZE = getInteger(IGNITE_AFFINITY_HISTORY_SIZE, 100);
 
     /** Atomic reference for pending timeout object. */
     private AtomicReference<ResendTimeoutObject> pendingResend = new AtomicReference<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
new file mode 100644
index 0000000..24ebb7c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int SRVS = 4;
+
+    /** */
+    private boolean client;
+
+    /** */
+    private boolean clientDiscovery;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (!clientDiscovery)
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setWriteSynchronizationMode(PRIMARY_SYNC);
+        ccfg.setBackups(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(SRVS);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testManyClients() throws Exception {
+        manyClientsPutGet();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testManyClientsClientDiscovery() throws Exception {
+        clientDiscovery = true;
+
+        manyClientsPutGet();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void manyClientsPutGet() throws Exception {
+        client = true;
+
+        final AtomicInteger idx = new AtomicInteger(SRVS);
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        final int THREADS = 30;
+
+        final CountDownLatch latch = new CountDownLatch(THREADS);
+
+        try {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    try (Ignite ignite = startGrid(idx.getAndIncrement())) {
+                        log.info("Started node: " + ignite.name());
+
+                        assertTrue(ignite.configuration().isClientMode());
+
+                        IgniteCache<Object, Object> cache = ignite.cache(null);
+
+                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                        int iter = 0;
+
+                        Integer key = rnd.nextInt(0, 1000);
+
+                        cache.put(key, iter++);
+
+                        assertNotNull(cache.get(key));
+
+                        latch.countDown();
+
+                        while (!stop.get()) {
+                            key = rnd.nextInt(0, 1000);
+
+                            cache.put(key, iter++);
+
+                            assertNotNull(cache.get(key));
+                        }
+
+                        log.info("Stopping node: " + ignite.name());
+                    }
+
+                    return null;
+                }
+            }, THREADS, "client-thread");
+
+            latch.await();
+
+            Thread.sleep(10_000);
+
+            log.info("Stop clients.");
+
+            stop.set(true);
+
+            fut.get();
+        }
+        finally {
+            stop.set(true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 713c5e5..ed9fc9a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -138,6 +138,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(CacheReadOnlyTransactionalClientSelfTest.class);
 
+        suite.addTestSuite(IgniteCacheManyClientsTest.class);
+
         return suite;
     }
 }


[25/50] incubator-ignite git commit: Merge branch 'ignite-389-ipc' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-389

Posted by sb...@apache.org.
Merge branch 'ignite-389-ipc' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-389


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

Branch: refs/heads/ignite-745
Commit: fa97def49403967db9ec8b6afdc3f18160b31ce3
Parents: 5872b7f 6b51f99
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jun 4 19:25:37 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jun 4 19:25:37 2015 -0700

----------------------------------------------------------------------
 modules/core/pom.xml                            |   1 -
 .../ignite/internal/util/IgniteUtils.java       |   4 +-
 .../shmem/IpcSharedMemoryClientEndpoint.java    |   2 +-
 .../ipc/shmem/IpcSharedMemoryNativeLoader.java  | 150 ++++++-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   2 +-
 .../util/nio/GridShmemCommunicationClient.java  | 151 +++++++
 .../communication/tcp/TcpCommunicationSpi.java  | 414 ++++++++++++++++++-
 .../tcp/TcpCommunicationSpiMBean.java           |   8 +
 .../IgniteCacheMessageRecoveryAbstractTest.java |   1 +
 .../IpcSharedMemoryCrashDetectionSelfTest.java  |   2 +-
 .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java |   2 +-
 .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java |   2 +-
 .../LoadWithCorruptedLibFileTestRunner.java     |   2 +-
 .../IpcSharedMemoryBenchmarkReader.java         |   2 +-
 .../IpcSharedMemoryBenchmarkWriter.java         |   2 +-
 .../communication/GridIoManagerBenchmark0.java  |   1 +
 .../spi/GridTcpSpiForwardingSelfTest.java       |   1 +
 .../GridTcpCommunicationSpiAbstractTest.java    |  13 +
 ...mmunicationSpiConcurrentConnectSelfTest.java |   4 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |  21 +-
 ...pCommunicationSpiMultithreadedShmemTest.java |  28 ++
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |   1 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   1 +
 .../GridTcpCommunicationSpiShmemSelfTest.java   |  38 ++
 .../tcp/GridTcpCommunicationSpiTcpSelfTest.java |   7 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |   2 +
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  13 +
 ...oopSecondaryFileSystemConfigurationTest.java |  14 +
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |   7 +
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   7 +
 .../hadoop/HadoopAbstractSelfTest.java          |   7 +
 .../org/apache/ignite/spark/IgniteContext.scala |  19 +-
 .../org/apache/ignite/spark/IgniteRDD.scala     |   8 +-
 33 files changed, 889 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa97def4/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------


[35/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-389' into ignite-389

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


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

Branch: refs/heads/ignite-745
Commit: 079bcc681343899e6ee7b0848ed57614d610ef55
Parents: f129d08 b812c0f
Author: avinogradov <av...@gridgain.com>
Authored: Tue Jun 9 15:11:56 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Jun 9 15:11:56 2015 +0300

----------------------------------------------------------------------
 ...CacheScanPartitionQueryFallbackSelfTest.java | 20 ++++++++++++++++----
 1 file changed, 16 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[42/50] incubator-ignite git commit: #IGNITE-389 - Readme.

Posted by sb...@apache.org.
#IGNITE-389 - Readme.


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

Branch: refs/heads/ignite-745
Commit: 9926fb89001e9115c1ea5105c4733208d426b08d
Parents: 71f29e9
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Jun 10 15:56:26 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Jun 10 15:56:26 2015 -0700

----------------------------------------------------------------------
 modules/spark/README.txt | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9926fb89/modules/spark/README.txt
----------------------------------------------------------------------
diff --git a/modules/spark/README.txt b/modules/spark/README.txt
index 5678441..589a050 100644
--- a/modules/spark/README.txt
+++ b/modules/spark/README.txt
@@ -1,4 +1,8 @@
 Apache Ignite Spark Module
 ---------------------------
 
-Apache Ignite Spark module.
+Apache Ignite provides an implementation of Spark RDD abstraction which enables easy access to Ignite caches.
+Ignite RDD does not keep it's state in the memory of the Spark application and provides a view of the corresponding
+Ignite cache. Depending on the chosen deployment mode this state may exist only during the lifespan of the Spark
+application (embedded mode) or may exist outside of the Spark application (standalone mode), allowing seamless
+sharing of the state between multiple Spark jobs.
\ No newline at end of file


[04/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-389' into ignite-389

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


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

Branch: refs/heads/ignite-745
Commit: 41afb3727844e1ccd3f5a2fd16ac33f9c884a0b9
Parents: 37a7679 29dc722
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri May 29 08:47:07 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri May 29 08:47:07 2015 -0700

----------------------------------------------------------------------
 ...CacheScanPartitionQueryFallbackSelfTest.java | 335 ++++++++++++++-----
 1 file changed, 259 insertions(+), 76 deletions(-)
----------------------------------------------------------------------



[06/50] incubator-ignite git commit: #IGNITE-389 - More functions on API.

Posted by sb...@apache.org.
#IGNITE-389 - More functions on API.


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

Branch: refs/heads/ignite-745
Commit: c527a04471bd4d263a346ee27a5dbef6c98a5894
Parents: 8503dec
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri May 29 16:29:33 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri May 29 16:29:33 2015 -0700

----------------------------------------------------------------------
 .../org/apache/ignite/spark/IgniteContext.scala     |  6 +++---
 .../scala/org/apache/ignite/spark/IgniteRDD.scala   | 16 ++++++++--------
 .../ignite/spark/examples/ColocationTest.scala      |  2 +-
 .../ignite/spark/examples/IgniteStoreExample.scala  |  2 +-
 .../ignite/spark/impl/IgniteQueryIterator.scala     |  2 +-
 .../org/apache/ignite/spark/impl/IgniteSqlRDD.scala |  2 +-
 .../org/apache/ignite/spark/IgniteRddSpec.scala     | 10 +++++-----
 7 files changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c527a044/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
index 5b649db..6259665 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
@@ -33,7 +33,7 @@ import org.apache.spark.SparkContext
  */
 class IgniteContext[K, V](
     @scala.transient val sparkContext: SparkContext,
-    cfgF: () => IgniteConfiguration
+    cfgF: () ⇒ IgniteConfiguration
 ) extends Serializable {
     def this(
         sc: SparkContext,
@@ -57,14 +57,14 @@ class IgniteContext[K, V](
             Ignition.ignite(igniteCfg.getGridName)
         }
         catch {
-            case e: Exception =>
+            case e: Exception ⇒
                 try {
                     igniteCfg.setClientMode(true)
 
                     Ignition.start(igniteCfg)
                 }
                 catch {
-                    case e: Exception => Ignition.ignite(igniteCfg.getGridName)
+                    case e: Exception ⇒ Ignition.ignite(igniteCfg.getGridName)
                 }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c527a044/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index 358fcd4..f286b58 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -59,7 +59,7 @@ class IgniteRDD[K, V] (
 
         val it: java.util.Iterator[Cache.Entry[K, V]] = cache.query(qry).iterator()
 
-        new IgniteQueryIterator[Cache.Entry[K, V], (K, V)](it, entry => {
+        new IgniteQueryIterator[Cache.Entry[K, V], (K, V)](it, entry ⇒ {
             (entry.getKey, entry.getValue)
         })
     }
@@ -95,7 +95,7 @@ class IgniteRDD[K, V] (
 
         qry.setArgs(args.map(_.asInstanceOf[Object]):_*)
 
-        new IgniteSqlRDD[(K, V), Cache.Entry[K, V], K, V](ic, cacheName, cacheCfg, qry, entry => (entry.getKey, entry.getValue))
+        new IgniteSqlRDD[(K, V), Cache.Entry[K, V], K, V](ic, cacheName, cacheCfg, qry, entry ⇒ (entry.getKey, entry.getValue))
     }
 
     def sql(sql: String, args: Any*): RDD[Seq[Any]] = {
@@ -103,11 +103,11 @@ class IgniteRDD[K, V] (
 
         qry.setArgs(args.map(_.asInstanceOf[Object]):_*)
 
-        new IgniteSqlRDD[Seq[Any], java.util.List[_], K, V](ic, cacheName, cacheCfg, qry, list => list)
+        new IgniteSqlRDD[Seq[Any], java.util.List[_], K, V](ic, cacheName, cacheCfg, qry, list ⇒ list)
     }
 
     def saveValues(rdd: RDD[V]) = {
-        rdd.foreachPartition(it => {
+        rdd.foreachPartition(it ⇒ {
             val ig = ic.ignite()
 
             ensureCache()
@@ -119,7 +119,7 @@ class IgniteRDD[K, V] (
             val streamer = ig.dataStreamer[Object, V](cacheName)
 
             try {
-                it.foreach(value => {
+                it.foreach(value ⇒ {
                     val key = affinityKeyFunc(value, node.orNull)
 
                     streamer.addData(key, value)
@@ -131,8 +131,8 @@ class IgniteRDD[K, V] (
         })
     }
 
-    def saveTuples(rdd: RDD[(K, V)]) = {
-        rdd.foreachPartition(it => {
+    def savePairs(rdd: RDD[(K, V)]) = {
+        rdd.foreachPartition(it ⇒ {
             val ig = ic.ignite()
 
             // Make sure to deploy the cache
@@ -145,7 +145,7 @@ class IgniteRDD[K, V] (
             val streamer = ig.dataStreamer[K, V](cacheName)
 
             try {
-                it.foreach(tup => {
+                it.foreach(tup ⇒ {
                     streamer.addData(tup._1, tup._2)
                 })
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c527a044/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
index a0814fa..e1d3d8e 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
@@ -31,7 +31,7 @@ object ColocationTest {
         // Search for lines containing "Ignite".
         val cache = ignite.fromCache("partitioned")
 
-        cache.saveTuples(sc.parallelize((1 to 100000).toSeq, 48).map(i => (i, i)))
+        cache.savePairs(sc.parallelize((1 to 100000).toSeq, 48).map(i => (i, i)))
 
         // Execute parallel sum.
         println("Local sum: " + (1 to 100000).sum)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c527a044/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteStoreExample.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteStoreExample.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteStoreExample.scala
index 24be795..ad6b7e6 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteStoreExample.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteStoreExample.scala
@@ -36,6 +36,6 @@ object IgniteStoreExample {
         })
 
         ignite.fromCache("partitioned").saveValues(lines)
-        ignite.fromCache("").saveTuples(lines.map(l ⇒ (l, l)))
+        ignite.fromCache("partitioned").savePairs(lines.map(l ⇒ (l, l)))
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c527a044/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteQueryIterator.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteQueryIterator.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteQueryIterator.scala
index b24ba50..4165fd3 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteQueryIterator.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteQueryIterator.scala
@@ -19,7 +19,7 @@ package org.apache.ignite.spark.impl
 
 class IgniteQueryIterator[T, R] (
     cur: java.util.Iterator[T],
-    conv: (T) => R
+    conv: (T) ⇒ R
 ) extends Iterator[R] {
     override def hasNext: Boolean = cur.hasNext
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c527a044/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
index 7cf9f3a..762a6ed 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
@@ -29,7 +29,7 @@ class IgniteSqlRDD[R: ClassTag, T, K, V](
     cacheName: String,
     cacheCfg: CacheConfiguration[K, V],
     qry: Query[T],
-    conv: (T) => R
+    conv: (T) ⇒ R
 ) extends IgniteAbstractRDD[R, K, V](ic, cacheName, cacheCfg) {
     override def compute(split: Partition, context: TaskContext): Iterator[R] = {
         new IgniteQueryIterator[T, R](ensureCache().query(qry).iterator(), conv)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c527a044/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
index 7af25de..68273da 100644
--- a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
+++ b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
@@ -42,7 +42,7 @@ class IgniteRddSpec extends FunSpec with Matchers with BeforeAndAfterAll with Be
                     () ⇒ configuration("client", client = true))
 
                 // Save pairs ("0", "val0"), ("1", "val1"), ... to Ignite cache.
-                ic.fromCache(PARTITIONED_CACHE_NAME).saveTuples(sc.parallelize(0 to 10000, 2).map(i ⇒ (String.valueOf(i), "val" + i)))
+                ic.fromCache(PARTITIONED_CACHE_NAME).savePairs(sc.parallelize(0 to 10000, 2).map(i ⇒ (String.valueOf(i), "val" + i)))
 
                 // Check cache contents.
                 val ignite = Ignition.ignite("grid-0")
@@ -92,9 +92,9 @@ class IgniteRddSpec extends FunSpec with Matchers with BeforeAndAfterAll with Be
 
                 val cache: IgniteRDD[String, Entity] = ic.fromCache(PARTITIONED_CACHE_NAME)
 
-                cache.saveTuples(sc.parallelize(0 to 1000, 2).map(i ⇒ (String.valueOf(i), new Entity(i, "name" + i, i * 100))))
+                cache.savePairs(sc.parallelize(0 to 1000, 2).map(i ⇒ (String.valueOf(i), new Entity(i, "name" + i, i * 100))))
 
-                val res = cache.objectSql("Entity", "name = ? and salary = ?", "name50", 5000).map(_._2).collect()
+                val res: Array[Entity] = cache.objectSql("Entity", "name = ? and salary = ?", "name50", 5000).map(_._2).collect()
 
                 assert(res.length == 1, "Invalid result length")
                 assert(50 == res(0).id, "Invalid result")
@@ -117,7 +117,7 @@ class IgniteRddSpec extends FunSpec with Matchers with BeforeAndAfterAll with Be
 
                 val cache: IgniteRDD[String, Entity] = ic.fromCache(PARTITIONED_CACHE_NAME)
 
-                cache.saveTuples(sc.parallelize(0 to 1000, 2).map(i ⇒ (String.valueOf(i), new Entity(i, "name" + i, i * 100))))
+                cache.savePairs(sc.parallelize(0 to 1000, 2).map(i ⇒ (String.valueOf(i), new Entity(i, "name" + i, i * 100))))
 
                 val res = cache.sql("select id, name, salary from Entity where name = ? and salary = ?", "name50", 5000).collect()
 
@@ -142,7 +142,7 @@ class IgniteRddSpec extends FunSpec with Matchers with BeforeAndAfterAll with Be
 
                 val cache: IgniteRDD[String, Entity] = ic.fromCache(PARTITIONED_CACHE_NAME)
 
-                cache.saveTuples(sc.parallelize(0 to 1000, 2).map(i ⇒ (String.valueOf(i), new Entity(i, "name" + i, i * 100))))
+                cache.savePairs(sc.parallelize(0 to 1000, 2).map(i ⇒ (String.valueOf(i), new Entity(i, "name" + i, i * 100))))
 
                 val res = cache.sql("select id, name, salary from Entity where name = ? and salary = ?", "name50", 5000).collect()
 


[38/50] incubator-ignite git commit: ignite-998

Posted by sb...@apache.org.
ignite-998


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

Branch: refs/heads/ignite-745
Commit: 46e6c48ee59a3837c06564d2431caabc55cf9a63
Parents: c47438e
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 10 15:33:01 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 10 15:33:01 2015 +0300

----------------------------------------------------------------------
 .../discovery/tcp/TcpClientDiscoverySpiSelfTest.java    | 12 ++++++++++++
 1 file changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/46e6c48e/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index ece898d..55a14e4 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -1136,6 +1136,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
      * @param clientCnt Number of client nodes.
      */
     private void checkNodes(int srvCnt, int clientCnt) {
+        long topVer = -1;
+
         for (int i = 0; i < srvCnt; i++) {
             Ignite g = G.ignite("server-" + i);
 
@@ -1144,6 +1146,11 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
             assertFalse(g.cluster().localNode().isClient());
 
             checkRemoteNodes(g, srvCnt + clientCnt - 1);
+
+            if (topVer < 0)
+                topVer = g.cluster().topologyVersion();
+            else
+                assertEquals(topVer, g.cluster().topologyVersion());
         }
 
         for (int i = 0; i < clientCnt; i++) {
@@ -1156,6 +1163,11 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
             assertTrue(g.cluster().localNode().isClient());
 
             checkRemoteNodes(g, srvCnt + clientCnt - 1);
+
+            if (topVer < 0)
+                topVer = g.cluster().topologyVersion();
+            else
+                assertEquals(topVer, g.cluster().topologyVersion());
         }
     }
 


[36/50] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-389

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-389


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

Branch: refs/heads/ignite-745
Commit: 980bf759e96954577e7a0b8662d7e6a63b6a4d2f
Parents: 079bcc6 928be42
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Jun 10 00:05:47 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Jun 10 00:05:47 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../managers/indexing/GridIndexingManager.java  |   4 -
 .../affinity/GridAffinityAssignment.java        |  12 ++
 .../affinity/GridAffinityAssignmentCache.java   |   4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   6 +-
 .../GridDhtPartitionsExchangeFuture.java        |  14 ++-
 .../continuous/GridContinuousProcessor.java     |   2 +
 .../util/nio/GridNioDelimitedBuffer.java        |   2 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |   9 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |  17 +--
 .../node/VisorNodeSuppressedErrorsTask.java     |  12 +-
 .../internal/visor/query/VisorQueryJob.java     |  11 +-
 .../internal/visor/query/VisorQueryTask.java    |   3 +-
 .../visor/util/VisorExceptionWrapper.java       |  81 ++++++++++++++
 .../internal/visor/util/VisorTaskUtils.java     |  10 ++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  10 +-
 .../continuous/GridEventConsumeSelfTest.java    |   7 +-
 .../nio/GridNioDelimitedBufferSelfTest.java     | 112 +++++++++++++++++++
 .../util/nio/GridNioDelimitedBufferTest.java    | 112 -------------------
 .../stream/socket/SocketStreamerSelfTest.java   |  29 ++---
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteStreamSelfTestSuite.java   |  39 +++++++
 .../testsuites/IgniteStreamTestSuite.java       |  39 -------
 .../testsuites/IgniteUtilSelfTestSuite.java     |   2 +-
 .../cache/GridCacheOffheapIndexGetSelfTest.java |  62 +++++++++-
 25 files changed, 395 insertions(+), 207 deletions(-)
----------------------------------------------------------------------



[23/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-389' into ignite-389

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


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

Branch: refs/heads/ignite-745
Commit: 0c2d381c830443b169bb7a1e4745356eebf78ef8
Parents: 7ee51ba 4d36d12
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jun 4 19:21:43 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jun 4 19:21:43 2015 -0700

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   6 +
 assembly/dependencies-fabric.xml                |   1 +
 examples/config/example-ignite.xml              |   4 +-
 examples/pom.xml                                |  34 +++
 modules/scalar-2.10/README.txt                  |   4 +
 modules/scalar-2.10/licenses/apache-2.0.txt     | 202 +++++++++++++
 .../scalar-2.10/licenses/scala-bsd-license.txt  |  18 ++
 modules/scalar-2.10/pom.xml                     | 197 ++++++++++++
 modules/spark-2.10/README.txt                   |   4 +
 modules/spark-2.10/licenses/apache-2.0.txt      | 202 +++++++++++++
 .../spark-2.10/licenses/scala-bsd-license.txt   |  18 ++
 modules/spark-2.10/pom.xml                      | 120 ++++++++
 modules/spark/licenses/apache-2.0.txt           | 202 +++++++++++++
 modules/spark/licenses/scala-bsd-license.txt    |  18 ++
 modules/spark/pom.xml                           |  22 +-
 .../spark/examples/java/ColocationTest.java     |  89 ++++++
 .../examples/java/ExampleConfiguration.java     |  31 ++
 .../examples/java/IgniteProcessExample.java     |  80 +++++
 .../spark/examples/java/IgniteStoreExample.java |  68 +++++
 .../spark/examples/java/package-info.java       |  21 ++
 .../org/apache/ignite/spark/IgniteRDD.scala     |  10 +-
 .../apache/ignite/spark/JavaIgniteContext.scala |  63 ++++
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |  99 ++++++
 .../ignite/spark/examples/ColocationTest.scala  |   5 +-
 .../spark/impl/JavaIgniteAbstractRDD.scala      |  34 +++
 .../ignite/spark/JavaIgniteRDDSelfTest.java     | 298 +++++++++++++++++++
 modules/visor-console-2.10/README.txt           |   4 +
 modules/visor-console-2.10/pom.xml              | 174 +++++++++++
 parent/pom.xml                                  |   4 +
 pom.xml                                         |  21 +-
 30 files changed, 2037 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[48/50] incubator-ignite git commit: Idea code style excluded from licence validator

Posted by sb...@apache.org.
Idea code style excluded from licence validator


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

Branch: refs/heads/ignite-745
Commit: b52e47ae53e905cb67bd3c644bb246e393349652
Parents: 4f500fd
Author: agura <ag...@gridgain.com>
Authored: Thu Jun 11 14:56:16 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Jun 11 14:56:16 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b52e47ae/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index f5b73df..b167932 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -626,6 +626,7 @@
                                         <exclude>**/keystore/*.pem</exclude><!--auto generated files-->
                                         <exclude>**/keystore/*.pfx</exclude><!--bin-files-->
                                         <!--special excludes-->
+                                        <exclude>idea/ignite_codeStyle.xml</exclude>
                                         <exclude>DEVNOTES.txt</exclude>
                                         <exclude>src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffHeapSnapTreeMap.java</exclude><!--BSD license-->
                                         <exclude>src/main/java/org/apache/ignite/internal/util/snaptree/*.java</exclude><!--BSD license-->


[49/50] incubator-ignite git commit: ignite-745 Fixed query metrics for partitioned cache

Posted by sb...@apache.org.
ignite-745 Fixed query metrics for partitioned cache


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

Branch: refs/heads/ignite-745
Commit: 01bcfd8a70cb1a3f2fa0938207fe7d7353b2168d
Parents: 4375529
Author: agura <ag...@gridgain.com>
Authored: Wed Jun 10 19:23:28 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Jun 11 16:04:00 2015 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 102 ++++++---
 .../CacheAbstractQueryMetricsSelfTest.java      | 205 ++++++++++++++++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 +++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 +++
 .../cache/GridCacheQueryMetricsSelfTest.java    | 206 -------------------
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   2 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +-
 7 files changed, 344 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01bcfd8a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e187713..3b0549e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -513,8 +513,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> query(String space, String clause,
-        Collection<Object> params, String resType, IndexingQueryFilter filters)
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> query(final String space, final String clause,
+        final Collection<Object> params, final String resType, final IndexingQueryFilter filters)
         throws IgniteCheckedException {
         checkEnabled();
 
@@ -522,12 +522,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            TypeDescriptor type = typesByName.get(new TypeName(space, resType));
+            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
 
-            if (type == null || !type.registered())
-                throw new CacheException("Failed to find SQL table for type: " + resType);
+            return executeQuery(cctx, new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
+                @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
+                    TypeDescriptor type = typesByName.get(new TypeName(space, resType));
 
-            return idx.query(space, clause, params, type, filters);
+                    if (type == null || !type.registered())
+                        throw new CacheException("Failed to find SQL table for type: " + resType);
+
+                    return idx.query(space, clause, params, type, filters);
+                }
+            });
         }
         finally {
             busyLock.leaveBusy();
@@ -539,19 +545,26 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Cursor.
      */
-    public Iterable<List<?>> queryTwoStep(String space, GridCacheTwoStepQuery qry) {
+    public Iterable<List<?>> queryTwoStep(String space, final GridCacheTwoStepQuery qry) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            GridCacheContext<Object, Object> cacheCtx = ctx.cache().internalCache(space).context();
-
-            return idx.queryTwoStep(
-                cacheCtx,
-                qry,
-                cacheCtx.keepPortable());
+            final GridCacheContext<Object, Object> cctx = ctx.cache().internalCache(space).context();
+
+            return executeQuery(cctx, new IgniteOutClosureX<Iterable<List<?>>>() {
+                @Override public Iterable<List<?>> applyx() throws IgniteCheckedException {
+                    return idx.queryTwoStep(
+                        cctx,
+                        qry,
+                        cctx.keepPortable());
+                }
+            });
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
         finally {
             busyLock.leaveBusy();
@@ -563,14 +576,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Cursor.
      */
-    public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
+    public QueryCursor<List<?>> queryTwoStep(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            return idx.queryTwoStep(cctx, qry);
+            return executeQuery(cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
+                @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                    return idx.queryTwoStep(cctx, qry);
+                }
+            });
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
         finally {
             busyLock.leaveBusy();
@@ -582,14 +602,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Cursor.
      */
-    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(GridCacheContext<?,?> cctx, SqlQuery qry) {
+    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(final GridCacheContext<?,?> cctx, final SqlQuery qry) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            return idx.queryTwoStep(cctx, qry);
+            return executeQuery(cctx, new IgniteOutClosureX<QueryCursor<Cache.Entry<K, V>>>() {
+                @Override public QueryCursor<Cache.Entry<K, V>> applyx() throws IgniteCheckedException {
+                    return idx.queryTwoStep(cctx, qry);
+                }
+            });
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
         finally {
             busyLock.leaveBusy();
@@ -698,7 +725,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * Closeable iterator.
      */
-    private static interface ClIter<X> extends AutoCloseable, Iterator<X> {
+    private interface ClIter<X> extends AutoCloseable, Iterator<X> {
         // No-op.
     }
 
@@ -810,24 +837,30 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(String space, String clause, String resType,
-        IndexingQueryFilter filters) throws IgniteCheckedException {
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(final String space, final String clause,
+        final String resType, final IndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            TypeDescriptor type = typesByName.get(new TypeName(space, resType));
+            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
 
-            if (type == null || !type.registered())
-                throw new CacheException("Failed to find SQL table for type: " + resType);
+            return executeQuery(cctx, new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
+                @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
+                    TypeDescriptor type = typesByName.get(new TypeName(space, resType));
 
-            return idx.queryText(
-                space,
-                clause,
-                type,
-                filters);
+                    if (type == null || !type.registered())
+                        throw new CacheException("Failed to find SQL table for type: " + resType);
+
+                    return idx.queryText(
+                        space,
+                        clause,
+                        type,
+                        filters);
+                }
+            });
         }
         finally {
             busyLock.leaveBusy();
@@ -842,15 +875,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Field rows.
      * @throws IgniteCheckedException If failed.
      */
-    public GridQueryFieldsResult queryFields(@Nullable String space, String clause, Collection<Object> params,
-        IndexingQueryFilter filters) throws IgniteCheckedException {
+    public GridQueryFieldsResult queryFields(@Nullable final String space, final String clause,
+        final Collection<Object> params, final IndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            return idx.queryFields(space, clause, params, filters);
+            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
+
+            return executeQuery(cctx, new IgniteOutClosureX<GridQueryFieldsResult>() {
+                @Override public GridQueryFieldsResult applyx() throws IgniteCheckedException {
+                    return idx.queryFields(space, clause, params, filters);
+                }
+            });
         }
         finally {
             busyLock.leaveBusy();
@@ -1718,6 +1757,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
         @Override public <T> T value(String field, Object key, Object val) throws IgniteCheckedException {
             assert field != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01bcfd8a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
new file mode 100644
index 0000000..63912bf
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Tests for cache query metrics.
+ */
+public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstractTest {
+    /** Grid count. */
+    private static final int GRID_CNT = 2;
+
+    /** Cache mode. */
+    protected CacheMode cacheMode;
+
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGridsMultiThreaded(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        CacheConfiguration<String, Integer> cacheCfg1 = defaultCacheConfiguration();
+
+        cacheCfg1.setName("A");
+        cacheCfg1.setCacheMode(cacheMode);
+        cacheCfg1.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg1.setIndexedTypes(String.class, Integer.class);
+
+        CacheConfiguration<String, Integer> cacheCfg2 = defaultCacheConfiguration();
+
+        cacheCfg2.setName("B");
+        cacheCfg2.setCacheMode(cacheMode);
+        cacheCfg2.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg2.setIndexedTypes(String.class, Integer.class);
+
+        cfg.setCacheConfiguration(cacheCfg1, cacheCfg2);
+
+        return cfg;
+    }
+
+    /**
+     * Test metrics for SQL queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlFieldsQueryMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from Integer");
+
+        cache.query(qry).getAll();
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        cache.query(qry).getAll();
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+
+    /**
+     * Test metrics for Scan queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testScanQueryMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        ScanQuery<String, Integer> qry = new ScanQuery<>();
+
+        cache.query(qry).getAll();
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        cache.query(qry).getAll();
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+
+    /**
+     * Test metrics for SQL cross cache queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlCrossCacheQueryMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".Integer");
+
+        cache.query(qry).getAll();
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        cache.query(qry).getAll();
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01bcfd8a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
new file mode 100644
index 0000000..666acfb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for partitioned cache query metrics.
+ */
+public class CachePartitionedQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        cacheMode = PARTITIONED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01bcfd8a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
new file mode 100644
index 0000000..2be6dd3
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for replicated cache query metrics.
+ */
+public class CacheReplicatedQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        cacheMode = REPLICATED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01bcfd8a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
deleted file mode 100644
index 24011b4..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.query.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-
-/**
- * Tests for cache query metrics.
- */
-public class GridCacheQueryMetricsSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final int GRID_CNT = 2;
-
-    /** */
-    private static final CacheMode CACHE_MODE = REPLICATED;
-
-    /** */
-    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        startGridsMultiThreaded(GRID_CNT);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(disco);
-
-        CacheConfiguration<String, Integer> cacheCfg1 = defaultCacheConfiguration();
-
-        cacheCfg1.setName("A");
-        cacheCfg1.setCacheMode(CACHE_MODE);
-        cacheCfg1.setWriteSynchronizationMode(FULL_SYNC);
-        cacheCfg1.setIndexedTypes(String.class, Integer.class);
-
-        CacheConfiguration<String, Integer> cacheCfg2 = defaultCacheConfiguration();
-
-        cacheCfg2.setName("B");
-        cacheCfg2.setCacheMode(CACHE_MODE);
-        cacheCfg2.setWriteSynchronizationMode(FULL_SYNC);
-        cacheCfg2.setIndexedTypes(String.class, Integer.class);
-
-        cfg.setCacheConfiguration(cacheCfg1, cacheCfg2);
-
-        return cfg;
-    }
-
-    /**
-     * Test metrics for SQL queries.
-     *
-     * @throws Exception In case of error.
-     */
-    public void testSqlFieldsQueryMetrics() throws Exception {
-        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
-
-        // Execute query.
-        SqlFieldsQuery qry = new SqlFieldsQuery("select * from Integer");
-
-        cache.query(qry).getAll();
-
-        QueryMetrics m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(1, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-
-        // Execute again with the same parameters.
-        cache.query(qry).getAll();
-
-        m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(2, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-    }
-
-    /**
-     * Test metrics for Scan queries.
-     *
-     * @throws Exception In case of error.
-     */
-    public void testScanQueryMetrics() throws Exception {
-        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
-
-        // Execute query.
-        ScanQuery<String, Integer> qry = new ScanQuery<>();
-
-        cache.query(qry).getAll();
-
-        QueryMetrics m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(1, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-
-        // Execute again with the same parameters.
-        cache.query(qry).getAll();
-
-        m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(2, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-    }
-
-    /**
-     * Test metrics for SQL cross cache queries.
-     *
-     * @throws Exception In case of error.
-     */
-    public void testSqlCrossCacheQueryMetrics() throws Exception {
-        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
-
-        // Execute query.
-        SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".Integer");
-
-        cache.query(qry).getAll();
-
-        QueryMetrics m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(1, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-
-        // Execute again with the same parameters.
-        cache.query(qry).getAll();
-
-        m = cache.queryMetrics();
-
-        assert m != null;
-
-        info("Metrics: " + m);
-
-        assertEquals(2, m.executions());
-        assertEquals(0, m.fails());
-        assertTrue(m.averageTime() >= 0);
-        assertTrue(m.maximumTime() >= 0);
-        assertTrue(m.minimumTime() >= 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01bcfd8a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java
index 99366f0..937419c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/BaseH2CompareQueryTest.java
@@ -198,8 +198,8 @@ public class BaseH2CompareQueryTest extends AbstractH2CompareQueryTest {
     /**
      * @throws Exception
      */
-    // TODO: IGNITE-705
     public void testAllExamples() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-705");
 //        compareQueryRes0("select ? limit ? offset ?");
 
 //        compareQueryRes0("select cool1()");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01bcfd8a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 7e20690..64873bc 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -112,7 +112,9 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(BaseH2CompareQueryTest.class);
         suite.addTestSuite(H2CompareBigQueryTest.class);
 
-        suite.addTestSuite(GridCacheQueryMetricsSelfTest.class);
+        // Cache query metrics.
+        suite.addTestSuite(CachePartitionedQueryMetricsSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsSelfTest.class);
 
         //Unmarshallig query test.
         suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);


[27/50] incubator-ignite git commit: #IGNITE-389 - Javadoc and API cleanup.

Posted by sb...@apache.org.
#IGNITE-389 - Javadoc and API cleanup.


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

Branch: refs/heads/ignite-745
Commit: 3d1e5342f32f56d2479ec7971e8fe2f4adfbf468
Parents: 1552a4b
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 8 16:03:34 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 8 16:03:34 2015 -0700

----------------------------------------------------------------------
 .../spark/examples/java/ColocationTest.java     | 89 --------------------
 .../examples/java/ExampleConfiguration.java     | 31 -------
 .../examples/java/IgniteProcessExample.java     | 80 ------------------
 .../spark/examples/java/IgniteStoreExample.java | 68 ---------------
 .../spark/examples/java/package-info.java       | 21 -----
 .../org/apache/ignite/spark/IgniteContext.scala | 30 ++++++-
 .../org/apache/ignite/spark/IgniteRDD.scala     | 41 +++++++--
 .../ignite/spark/examples/ColocationTest.scala  | 39 ---------
 .../spark/examples/ExampleConfiguration.scala   | 41 ---------
 .../spark/examples/IgniteProcessExample.scala   | 52 ------------
 .../spark/examples/IgniteStoreExample.scala     | 41 ---------
 11 files changed, 62 insertions(+), 471 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ColocationTest.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ColocationTest.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ColocationTest.java
deleted file mode 100644
index 20d6e88..0000000
--- a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ColocationTest.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spark.examples.java;
-
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spark.*;
-import org.apache.spark.*;
-import org.apache.spark.api.java.*;
-import org.apache.spark.api.java.function.*;
-
-import scala.Tuple2;
-
-import java.util.*;
-
-/**
- * Colocation test example.
- */
-public class ColocationTest {
-    /** Keys count. */
-    private static final int KEYS_CNT = 10000;
-
-    /** To pair function. */
-    private static final IgniteClosure<Integer, Tuple2<Integer, Integer>> TO_PAIR_F =
-        new IgniteClosure<Integer, Tuple2<Integer, Integer>>() {
-            @Override public Tuple2<Integer, Integer> apply(Integer i) {
-                return new Tuple2<>(i, i);
-            }
-        };
-
-    /** To value function. */
-    private static final Function<Tuple2<Integer, Integer>, Integer> TO_VALUE_F =
-        new Function<Tuple2<Integer, Integer>, Integer>() {
-            /** {@inheritDoc} */
-            @Override public Integer call(Tuple2<Integer, Integer> t) throws Exception {
-                return t._2();
-            }
-        };
-
-    /** Sum function. */
-    private static final Function2<Integer, Integer, Integer> SUM_F = new Function2<Integer, Integer, Integer>() {
-        public Integer call(Integer x, Integer y) {
-            return x + y;
-        }
-    };
-
-    /**
-     * @param args Args.
-     */
-    public static void main(String[] args) {
-        SparkConf conf = new SparkConf();
-
-        conf.setAppName("Colocation test");
-
-        JavaSparkContext sc = new JavaSparkContext(conf);
-
-        JavaIgniteContext<Integer, Integer> ignite = new JavaIgniteContext<>(sc, new ExampleConfiguration());
-
-        JavaIgniteRDD<Integer, Integer> cache = ignite.fromCache("partitioned");
-
-        List<Integer> seq = F.range(0, KEYS_CNT + 1);
-
-        JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(F.transformList(seq, TO_PAIR_F), 48);
-
-        cache.savePairs(rdd);
-
-        int sum = (KEYS_CNT * KEYS_CNT - KEYS_CNT) / 2;
-
-        // Execute parallel sum.
-        System.out.println("Local sum: " + sum);
-
-        System.out.println("Distributed sum: " + cache.map(TO_VALUE_F).fold(0, SUM_F));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ExampleConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ExampleConfiguration.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ExampleConfiguration.java
deleted file mode 100644
index 5d769f2..0000000
--- a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ExampleConfiguration.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spark.examples.java;
-
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
-
-/**
- * Ignite example configuration provider.
- */
-public class ExampleConfiguration implements IgniteOutClosure<IgniteConfiguration> {
-    /** {@inheritDoc} */
-    @Override public IgniteConfiguration apply() {
-        return org.apache.ignite.spark.examples.ExampleConfiguration.configuration();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteProcessExample.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteProcessExample.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteProcessExample.java
deleted file mode 100644
index 8994355..0000000
--- a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteProcessExample.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spark.examples.java;
-
-import org.apache.ignite.spark.*;
-import org.apache.spark.*;
-import org.apache.spark.api.java.*;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.sql.*;
-
-import scala.*;
-
-import java.lang.Boolean;
-
-/**
- * Ignite process example.
- */
-public class IgniteProcessExample {
-    /** Filter function. */
-    private static final Function<Tuple2<Object, String>, Boolean> FILTER_F =
-        new Function<Tuple2<Object, String>, Boolean>() {
-            @Override public Boolean call(Tuple2<Object, String> t) throws Exception {
-                System.out.println("Analyzing line: " + t._2());
-
-                return t._2().contains("Ignite");
-            }
-        };
-
-    /** To value function. */
-    private static final Function<Tuple2<Object, String>, String> TO_VALUE_F =
-        new Function<Tuple2<Object, String>, String>() {
-            @Override public String call(Tuple2<Object, String> t) throws Exception {
-                return t._2();
-            }
-        };
-
-    /**
-     * @param args Args.
-     */
-    public static void main(String[] args) {
-        SparkConf conf = new SparkConf();
-
-        conf.setAppName("Ignite processing example");
-
-        JavaSparkContext sc = new JavaSparkContext(conf);
-
-        JavaIgniteContext<Object, String> ignite = new JavaIgniteContext<>(sc, new ExampleConfiguration());
-
-        // Search for lines containing "Ignite".
-        JavaIgniteRDD<Object, String> scanRdd = ignite.fromCache("partitioned");
-
-        JavaRDD<String> processedRdd = scanRdd.filter(FILTER_F).map(TO_VALUE_F);
-
-        // Create a new cache for results.
-        JavaIgniteRDD<Object, String> results = ignite.fromCache("results");
-
-        results.saveValues(processedRdd);
-
-        // SQL query
-        ignite.fromCache("indexed").objectSql("Person", "age > ? and organizationId = ?", 20, 12).collect();
-
-        // SQL fields query
-        DataFrame df = ignite.fromCache("indexed").sql("select name, age from Person where age > ?", 20);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteStoreExample.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteStoreExample.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteStoreExample.java
deleted file mode 100644
index 24ae77f..0000000
--- a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteStoreExample.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spark.examples.java;
-
-import org.apache.ignite.spark.*;
-import org.apache.spark.*;
-import org.apache.spark.api.java.*;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.api.java.function.*;
-
-import scala.*;
-
-import java.lang.Boolean;
-
-/**
- * Ignite store example.
- */
-public class IgniteStoreExample {
-    /** Predicate. */
-    private static final Function<String, Boolean> PREDICATE = new Function<String, Boolean>() {
-        @Override public Boolean call(String s) throws Exception {
-            System.out.println("Read line: " + s);
-
-            return s.contains("Ignite");
-        }
-    };
-
-    /** To pair function. */
-    private static final PairFunction<String, String, String> TO_PAIR_F = new PairFunction<String, String, String>() {
-        @Override public Tuple2<String, String> call(String s) throws Exception {
-            return new Tuple2<>(s, s);
-        }
-    };
-
-    /**
-     * @param args Args.
-     */
-    public static void main(String[] args) {
-        SparkConf conf = new SparkConf();
-
-        conf.setAppName("Ignite processing example");
-
-        JavaSparkContext sc = new JavaSparkContext(conf);
-
-        JavaIgniteContext<String, String> ignite = new JavaIgniteContext<>(sc, new ExampleConfiguration());
-
-        JavaRDD<String> lines = sc.textFile(args[0]).filter(PREDICATE);
-
-        ignite.fromCache("partitioned").saveValues(lines);
-
-        ignite.fromCache("partitioned").savePairs(lines.mapToPair(TO_PAIR_F));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/package-info.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/package-info.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/package-info.java
deleted file mode 100644
index e3243bf..0000000
--- a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Demonstrates usage of Ignite and Spark from Java.
- */
-package org.apache.ignite.spark.examples.java;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
index 2cfebd6..e52555a 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
@@ -21,7 +21,7 @@ package org.apache.ignite.spark
 import org.apache.ignite.internal.IgnitionEx
 import org.apache.ignite.{Ignition, Ignite}
 import org.apache.ignite.configuration.{CacheConfiguration, IgniteConfiguration}
-import org.apache.spark.SparkContext
+import org.apache.spark.{Logging, SparkContext}
 import org.apache.spark.sql.SQLContext
 
 /**
@@ -36,7 +36,7 @@ class IgniteContext[K, V](
     @scala.transient val sparkContext: SparkContext,
     cfgF: () ⇒ IgniteConfiguration,
     client: Boolean = true
-) extends Serializable {
+) extends Serializable with Logging {
     @scala.transient private val driver = true
 
     if (!client) {
@@ -45,7 +45,7 @@ class IgniteContext[K, V](
         if (workers <= 0)
             throw new IllegalStateException("No Spark executors found to start Ignite nodes.")
 
-        println("Will start Ignite nodes on " + workers + " workers")
+        logInfo("Will start Ignite nodes on " + workers + " workers")
 
         // Start ignite server node on each worker in server mode.
         sparkContext.parallelize(1 to workers, workers).foreach(it ⇒ ignite())
@@ -60,14 +60,34 @@ class IgniteContext[K, V](
 
     val sqlContext = new SQLContext(sparkContext)
 
+    /**
+     * Creates an `IgniteRDD` instance from the given cache name. If the cache does not exist, it will be
+     * automatically started from template on the first invoked RDD action.
+     *
+     * @param cacheName Cache name.
+     * @return `IgniteRDD` instance.
+     */
     def fromCache(cacheName: String): IgniteRDD[K, V] = {
         new IgniteRDD[K, V](this, cacheName, null)
     }
 
+    /**
+     * Creates an `IgniteRDD` instance from the given cache configuration. If the cache does not exist, it will be
+     * automatically started using the configuration provided on the first invoked RDD action.
+     *
+     * @param cacheCfg Cache configuration to use.
+     * @return `IgniteRDD` instance.
+     */
     def fromCache(cacheCfg: CacheConfiguration[K, V]) = {
         new IgniteRDD[K, V](this, cacheCfg.getName, cacheCfg)
     }
 
+    /**
+     * Gets an Ignite instance supporting this context. Ignite instance will be started
+     * if it has not been started yet.
+     *
+     * @return Ignite instance.
+     */
     def ignite(): Ignite = {
         val igniteCfg = cfgF()
 
@@ -87,6 +107,10 @@ class IgniteContext[K, V](
         }
     }
 
+    /**
+     * Stops supporting ignite instance. If ignite instance has been already stopped, this operation will be
+     * a no-op.
+     */
     def close() = {
         val igniteCfg = cfgF()
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index 5fc457f..2146acb 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -82,7 +82,7 @@ class IgniteRDD[K, V] (
     }
 
     /**
-     * Gets prefferred locations for the given partition.
+     * Gets preferred locations for the given partition.
      *
      * @param split Split partition.
      * @return
@@ -94,6 +94,14 @@ class IgniteRDD[K, V] (
             .map(_.asInstanceOf[TcpDiscoveryNode].socketAddresses()).flatten.map(_.getHostName).toList
     }
 
+    /**
+     * Runs an object SQL on corresponding Ignite cache.
+     *
+     * @param typeName Type name to run SQL against.
+     * @param sql SQL query to run.
+     * @param args Optional SQL query arguments.
+     * @return RDD with query results.
+     */
     def objectSql(typeName: String, sql: String, args: Any*): RDD[(K, V)] = {
         val qry: SqlQuery[K, V] = new SqlQuery[K, V](typeName, sql)
 
@@ -102,6 +110,13 @@ class IgniteRDD[K, V] (
         new IgniteSqlRDD[(K, V), Cache.Entry[K, V], K, V](ic, cacheName, cacheCfg, qry, entry ⇒ (entry.getKey, entry.getValue))
     }
 
+    /**
+     * Runs an SQL fields query.
+     *
+     * @param sql SQL statement to run.
+     * @param args Optional SQL query arguments.
+     * @return `DataFrame` instance with the query results.
+     */
     def sql(sql: String, args: Any*): DataFrame = {
         val qry = new SqlFieldsQuery(sql)
 
@@ -114,7 +129,12 @@ class IgniteRDD[K, V] (
         ic.sqlContext.createDataFrame(rowRdd, schema)
     }
 
-    def saveValues(rdd: RDD[V], overwrite: Boolean = false) = {
+    /**
+     * Saves values from given RDD into Ignite. A unique key will be generated for each value of the given RDD.
+     *
+     * @param rdd RDD instance to save values from.
+     */
+    def saveValues(rdd: RDD[V]) = {
         rdd.foreachPartition(it ⇒ {
             val ig = ic.ignite()
 
@@ -127,8 +147,6 @@ class IgniteRDD[K, V] (
             val streamer = ig.dataStreamer[Object, V](cacheName)
 
             try {
-                streamer.allowOverwrite(overwrite)
-
                 it.foreach(value ⇒ {
                     val key = affinityKeyFunc(value, node.orNull)
 
@@ -141,6 +159,13 @@ class IgniteRDD[K, V] (
         })
     }
 
+    /**
+     * Saves values from the given key-value RDD into Ignite.
+     *
+     * @param rdd RDD instance to save values from.
+     * @param overwrite Boolean flag indicating whether the call on this method should overwrite existing
+     *      values in Ignite cache.
+     */
     def savePairs(rdd: RDD[(K, V)], overwrite: Boolean = false) = {
         rdd.foreachPartition(it ⇒ {
             val ig = ic.ignite()
@@ -163,6 +188,9 @@ class IgniteRDD[K, V] (
         })
     }
 
+    /**
+     * Removes all values from the underlying Ignite cache.
+     */
     def clear(): Unit = {
         ensureCache().removeAll()
     }
@@ -197,7 +225,7 @@ class IgniteRDD[K, V] (
         case "java.sql.Timestamp" ⇒ TimestampType
         case "[B" ⇒ BinaryType
 
-        case _ ⇒ StructType(new Array[StructField](0)) // TODO Do we need to fill user types?
+        case _ ⇒ StructType(new Array[StructField](0))
     }
 
     /**
@@ -210,6 +238,7 @@ class IgniteRDD[K, V] (
     private def affinityKeyFunc(value: V, node: ClusterNode): IgniteUuid = {
         val aff = ic.ignite().affinity[IgniteUuid](cacheName)
 
-        Stream.continually(IgniteUuid.randomUuid()).find(node == null || aff.mapKeyToNode(_).eq(node)).get
+        Stream.from(1, 1000).map(_ ⇒ IgniteUuid.randomUuid()).find(node == null || aff.mapKeyToNode(_).eq(node))
+            .getOrElse(IgniteUuid.randomUuid())
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
deleted file mode 100644
index 29587e4..0000000
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spark.examples
-
-import org.apache.ignite.spark.IgniteContext
-import org.apache.spark.{SparkConf, SparkContext}
-
-object ColocationTest {
-    def main(args: Array[String]) {
-        val conf = new SparkConf().setAppName("Colocation test")
-        val sc = new SparkContext(conf)
-
-        val ignite = new IgniteContext[Int, Int](sc, ExampleConfiguration.configuration _)
-
-        // Search for lines containing "Ignite".
-        val cache = ignite.fromCache("partitioned")
-
-        cache.savePairs(sc.parallelize((1 to 100000).toSeq, 48).map(i => (i, i)))
-
-        // Execute parallel sum.
-        println("Local sum: " + (1 to 100000).sum)
-        println("Distributed sum: " + cache.map(_._2).sum())
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ExampleConfiguration.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ExampleConfiguration.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ExampleConfiguration.scala
deleted file mode 100644
index 3b0dac7..0000000
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ExampleConfiguration.scala
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spark.examples
-
-import org.apache.ignite.configuration.IgniteConfiguration
-import org.apache.ignite.internal.util.lang.{GridFunc => F}
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder
-
-object ExampleConfiguration {
-    def configuration(): IgniteConfiguration = {
-        val cfg = new IgniteConfiguration()
-
-        val discoSpi = new TcpDiscoverySpi()
-
-        val ipFinder = new TcpDiscoveryVmIpFinder()
-
-        ipFinder.setAddresses(F.asList("127.0.0.1:47500", "127.0.0.1:47501"))
-
-        discoSpi.setIpFinder(ipFinder)
-
-        cfg.setDiscoverySpi(discoSpi)
-
-        cfg
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
deleted file mode 100644
index ab91c62..0000000
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spark.examples
-
-import org.apache.ignite.spark.IgniteContext
-import org.apache.spark.rdd.RDD
-import org.apache.spark.{SparkContext, SparkConf}
-
-object IgniteProcessExample {
-    def main(args: Array[String]) {
-        val conf = new SparkConf().setAppName("Ignite processing example")
-        val sc = new SparkContext(conf)
-
-        val ignite = new IgniteContext[Object, String](sc, ExampleConfiguration.configuration _)
-
-        // Search for lines containing "Ignite".
-        val scanRdd = ignite.fromCache("partitioned")
-
-        val processedRdd = scanRdd.filter(line => {
-            println("Analyzing line: " + line)
-            line._2.contains("Ignite")
-
-            true
-        }).map(_._2)
-
-        // Create a new cache for results.
-        val results = ignite.fromCache("results")
-
-        results.saveValues(processedRdd)
-
-        // SQL query
-        ignite.fromCache("indexed").objectSql("Person", "age > ? and organizationId = ?", 20, 12).collect()
-
-        // SQL fields query
-        val df = ignite.fromCache("indexed").sql("select name, age from Person where age > ?", 20)
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3d1e5342/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteStoreExample.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteStoreExample.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteStoreExample.scala
deleted file mode 100644
index ad6b7e6..0000000
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteStoreExample.scala
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spark.examples
-
-import org.apache.ignite.spark.IgniteContext
-import org.apache.spark.SparkContext
-import org.apache.spark.SparkConf
-import org.apache.spark.rdd.RDD
-
-object IgniteStoreExample {
-    def main(args: Array[String]) {
-        val conf = new SparkConf().setAppName("Ignite store example")
-        val sc = new SparkContext(conf)
-
-        val ignite = new IgniteContext[String, String](sc, () ⇒ ExampleConfiguration.configuration())
-
-        val lines: RDD[String] = sc.textFile(args(0)).filter(line ⇒ {
-            println("Read line: " + line)
-
-            line.contains("IGNITE")
-        })
-
-        ignite.fromCache("partitioned").saveValues(lines)
-        ignite.fromCache("partitioned").savePairs(lines.map(l ⇒ (l, l)))
-    }
-}
\ No newline at end of file


[21/50] incubator-ignite git commit: ignite-948 Add Java API for Ignite RDD

Posted by sb...@apache.org.
ignite-948 Add Java API for Ignite RDD


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

Branch: refs/heads/ignite-745
Commit: 4d36d12361b78aa79517addce2a33fd772a0201e
Parents: ac9dd30
Author: agura <ag...@gridgain.com>
Authored: Tue Jun 2 01:09:17 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Jun 4 22:15:42 2015 +0300

----------------------------------------------------------------------
 examples/config/example-ignite.xml              |   4 +-
 modules/spark/pom.xml                           |  14 +
 .../spark/examples/java/ColocationTest.java     |  89 ++++++
 .../examples/java/ExampleConfiguration.java     |  31 ++
 .../examples/java/IgniteProcessExample.java     |  80 +++++
 .../spark/examples/java/IgniteStoreExample.java |  68 +++++
 .../spark/examples/java/package-info.java       |  21 ++
 .../org/apache/ignite/spark/IgniteRDD.scala     |  10 +-
 .../apache/ignite/spark/JavaIgniteContext.scala |  63 ++++
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |  99 ++++++
 .../ignite/spark/examples/ColocationTest.scala  |   5 +-
 .../spark/impl/JavaIgniteAbstractRDD.scala      |  34 +++
 .../ignite/spark/JavaIgniteRDDSelfTest.java     | 298 +++++++++++++++++++
 parent/pom.xml                                  |   4 +
 14 files changed, 811 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/examples/config/example-ignite.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-ignite.xml b/examples/config/example-ignite.xml
index e746e59..dcb2ba8 100644
--- a/examples/config/example-ignite.xml
+++ b/examples/config/example-ignite.xml
@@ -30,14 +30,16 @@
         http://www.springframework.org/schema/util/spring-util.xsd">
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <!-- Set to true to enable distributed class loading for examples, default is false. -->
+<!--
         <property name="peerClassLoadingEnabled" value="true"/>
 
         <property name="marshaller">
             <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
-                <!-- Set to false to allow non-serializable objects in examples, default is true. -->
+                &lt;!&ndash; Set to false to allow non-serializable objects in examples, default is true. &ndash;&gt;
                 <property name="requireSerializable" value="false"/>
             </bean>
         </property>
+-->
 
         <!-- Enable task execution events for examples. -->
         <property name="includeEventTypes">

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index c22a52b..8900a10 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -87,6 +87,20 @@
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+            <version>${spring.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+            <version>${spring.version}</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ColocationTest.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ColocationTest.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ColocationTest.java
new file mode 100644
index 0000000..20d6e88
--- /dev/null
+++ b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ColocationTest.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spark.examples.java;
+
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spark.*;
+import org.apache.spark.*;
+import org.apache.spark.api.java.*;
+import org.apache.spark.api.java.function.*;
+
+import scala.Tuple2;
+
+import java.util.*;
+
+/**
+ * Colocation test example.
+ */
+public class ColocationTest {
+    /** Keys count. */
+    private static final int KEYS_CNT = 10000;
+
+    /** To pair function. */
+    private static final IgniteClosure<Integer, Tuple2<Integer, Integer>> TO_PAIR_F =
+        new IgniteClosure<Integer, Tuple2<Integer, Integer>>() {
+            @Override public Tuple2<Integer, Integer> apply(Integer i) {
+                return new Tuple2<>(i, i);
+            }
+        };
+
+    /** To value function. */
+    private static final Function<Tuple2<Integer, Integer>, Integer> TO_VALUE_F =
+        new Function<Tuple2<Integer, Integer>, Integer>() {
+            /** {@inheritDoc} */
+            @Override public Integer call(Tuple2<Integer, Integer> t) throws Exception {
+                return t._2();
+            }
+        };
+
+    /** Sum function. */
+    private static final Function2<Integer, Integer, Integer> SUM_F = new Function2<Integer, Integer, Integer>() {
+        public Integer call(Integer x, Integer y) {
+            return x + y;
+        }
+    };
+
+    /**
+     * @param args Args.
+     */
+    public static void main(String[] args) {
+        SparkConf conf = new SparkConf();
+
+        conf.setAppName("Colocation test");
+
+        JavaSparkContext sc = new JavaSparkContext(conf);
+
+        JavaIgniteContext<Integer, Integer> ignite = new JavaIgniteContext<>(sc, new ExampleConfiguration());
+
+        JavaIgniteRDD<Integer, Integer> cache = ignite.fromCache("partitioned");
+
+        List<Integer> seq = F.range(0, KEYS_CNT + 1);
+
+        JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(F.transformList(seq, TO_PAIR_F), 48);
+
+        cache.savePairs(rdd);
+
+        int sum = (KEYS_CNT * KEYS_CNT - KEYS_CNT) / 2;
+
+        // Execute parallel sum.
+        System.out.println("Local sum: " + sum);
+
+        System.out.println("Distributed sum: " + cache.map(TO_VALUE_F).fold(0, SUM_F));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ExampleConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ExampleConfiguration.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ExampleConfiguration.java
new file mode 100644
index 0000000..5d769f2
--- /dev/null
+++ b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/ExampleConfiguration.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spark.examples.java;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.lang.*;
+
+/**
+ * Ignite example configuration provider.
+ */
+public class ExampleConfiguration implements IgniteOutClosure<IgniteConfiguration> {
+    /** {@inheritDoc} */
+    @Override public IgniteConfiguration apply() {
+        return org.apache.ignite.spark.examples.ExampleConfiguration.configuration();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteProcessExample.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteProcessExample.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteProcessExample.java
new file mode 100644
index 0000000..8994355
--- /dev/null
+++ b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteProcessExample.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spark.examples.java;
+
+import org.apache.ignite.spark.*;
+import org.apache.spark.*;
+import org.apache.spark.api.java.*;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.sql.*;
+
+import scala.*;
+
+import java.lang.Boolean;
+
+/**
+ * Ignite process example.
+ */
+public class IgniteProcessExample {
+    /** Filter function. */
+    private static final Function<Tuple2<Object, String>, Boolean> FILTER_F =
+        new Function<Tuple2<Object, String>, Boolean>() {
+            @Override public Boolean call(Tuple2<Object, String> t) throws Exception {
+                System.out.println("Analyzing line: " + t._2());
+
+                return t._2().contains("Ignite");
+            }
+        };
+
+    /** To value function. */
+    private static final Function<Tuple2<Object, String>, String> TO_VALUE_F =
+        new Function<Tuple2<Object, String>, String>() {
+            @Override public String call(Tuple2<Object, String> t) throws Exception {
+                return t._2();
+            }
+        };
+
+    /**
+     * @param args Args.
+     */
+    public static void main(String[] args) {
+        SparkConf conf = new SparkConf();
+
+        conf.setAppName("Ignite processing example");
+
+        JavaSparkContext sc = new JavaSparkContext(conf);
+
+        JavaIgniteContext<Object, String> ignite = new JavaIgniteContext<>(sc, new ExampleConfiguration());
+
+        // Search for lines containing "Ignite".
+        JavaIgniteRDD<Object, String> scanRdd = ignite.fromCache("partitioned");
+
+        JavaRDD<String> processedRdd = scanRdd.filter(FILTER_F).map(TO_VALUE_F);
+
+        // Create a new cache for results.
+        JavaIgniteRDD<Object, String> results = ignite.fromCache("results");
+
+        results.saveValues(processedRdd);
+
+        // SQL query
+        ignite.fromCache("indexed").objectSql("Person", "age > ? and organizationId = ?", 20, 12).collect();
+
+        // SQL fields query
+        DataFrame df = ignite.fromCache("indexed").sql("select name, age from Person where age > ?", 20);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteStoreExample.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteStoreExample.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteStoreExample.java
new file mode 100644
index 0000000..24ae77f
--- /dev/null
+++ b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/IgniteStoreExample.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spark.examples.java;
+
+import org.apache.ignite.spark.*;
+import org.apache.spark.*;
+import org.apache.spark.api.java.*;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.*;
+
+import scala.*;
+
+import java.lang.Boolean;
+
+/**
+ * Ignite store example.
+ */
+public class IgniteStoreExample {
+    /** Predicate. */
+    private static final Function<String, Boolean> PREDICATE = new Function<String, Boolean>() {
+        @Override public Boolean call(String s) throws Exception {
+            System.out.println("Read line: " + s);
+
+            return s.contains("Ignite");
+        }
+    };
+
+    /** To pair function. */
+    private static final PairFunction<String, String, String> TO_PAIR_F = new PairFunction<String, String, String>() {
+        @Override public Tuple2<String, String> call(String s) throws Exception {
+            return new Tuple2<>(s, s);
+        }
+    };
+
+    /**
+     * @param args Args.
+     */
+    public static void main(String[] args) {
+        SparkConf conf = new SparkConf();
+
+        conf.setAppName("Ignite processing example");
+
+        JavaSparkContext sc = new JavaSparkContext(conf);
+
+        JavaIgniteContext<String, String> ignite = new JavaIgniteContext<>(sc, new ExampleConfiguration());
+
+        JavaRDD<String> lines = sc.textFile(args[0]).filter(PREDICATE);
+
+        ignite.fromCache("partitioned").saveValues(lines);
+
+        ignite.fromCache("partitioned").savePairs(lines.mapToPair(TO_PAIR_F));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/package-info.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/package-info.java b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/package-info.java
new file mode 100644
index 0000000..e3243bf
--- /dev/null
+++ b/modules/spark/src/main/java/org/apache/ignite/spark/examples/java/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Demonstrates usage of Ignite and Spark from Java.
+ */
+package org.apache.ignite.spark.examples.java;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index 0b8e845..742d7ee 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -43,9 +43,9 @@ import scala.collection.JavaConversions._
  * @tparam V Value type.
  */
 class IgniteRDD[K, V] (
-    ic: IgniteContext[K, V],
-    cacheName: String,
-    cacheCfg: CacheConfiguration[K, V]
+    val ic: IgniteContext[K, V],
+    val cacheName: String,
+    val cacheCfg: CacheConfiguration[K, V]
 ) extends IgniteAbstractRDD[(K, V), K, V] (ic, cacheName, cacheCfg) {
     /**
      * Computes iterator based on given partition.
@@ -73,7 +73,7 @@ class IgniteRDD[K, V] (
      *
      * @return Partitions.
      */
-    override protected def getPartitions: Array[Partition] = {
+    override protected[spark] def getPartitions: Array[Partition] = {
         ensureCache()
 
         val parts = ic.ignite().affinity(cacheName).partitions()
@@ -87,7 +87,7 @@ class IgniteRDD[K, V] (
      * @param split Split partition.
      * @return
      */
-    override protected def getPreferredLocations(split: Partition): Seq[String] = {
+    override protected[spark] def getPreferredLocations(split: Partition): Seq[String] = {
         ensureCache()
 
         ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
new file mode 100644
index 0000000..e2d57bf
--- /dev/null
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spark
+
+import org.apache.ignite.Ignite
+import org.apache.ignite.configuration.{CacheConfiguration, IgniteConfiguration}
+import org.apache.ignite.internal.IgnitionEx
+import org.apache.ignite.lang.IgniteOutClosure
+import org.apache.spark.api.java.JavaSparkContext
+
+import scala.reflect.ClassTag
+
+/**
+ * Java-friendly Ignite context wrapper.
+ *
+ * @param sc Java Spark context.
+ * @param cfgF Configuration factory.
+ * @tparam K Key type.
+ * @tparam V Value type.
+ */
+class JavaIgniteContext[K, V](
+    @scala.transient val sc: JavaSparkContext,
+    val cfgF: IgniteOutClosure[IgniteConfiguration]) extends Serializable {
+
+    @transient val ic: IgniteContext[K, V] = new IgniteContext[K, V](sc.sc, () => cfgF.apply())
+
+    def this(sc: JavaSparkContext, springUrl: String) {
+        this(sc, new IgniteOutClosure[IgniteConfiguration] {
+            override def apply() = IgnitionEx.loadConfiguration(springUrl).get1()
+        })
+    }
+
+    def fromCache(cacheName: String): JavaIgniteRDD[K, V] =
+        JavaIgniteRDD.fromIgniteRDD(new IgniteRDD[K, V](ic, cacheName, null))
+
+    def fromCache(cacheCfg: CacheConfiguration[K, V]) =
+        JavaIgniteRDD.fromIgniteRDD(new IgniteRDD[K, V](ic, cacheCfg.getName, cacheCfg))
+
+    def ignite(): Ignite = ic.ignite()
+
+    def close() = ic.close()
+
+    private[spark] def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]]
+
+    implicit val ktag: ClassTag[K] = fakeClassTag
+
+    implicit val vtag: ClassTag[V] = fakeClassTag
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
new file mode 100644
index 0000000..2e8702e
--- /dev/null
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spark
+
+import java.util
+
+import org.apache.spark.api.java.{JavaPairRDD, JavaRDD}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.{Partition, TaskContext}
+
+import scala.annotation.varargs
+import scala.collection.JavaConversions._
+import scala.language.implicitConversions
+import scala.reflect.ClassTag
+
+/**
+ * Java-friendly Ignite RDD wrapper. Represents Ignite cache as Java Spark RDD abstraction.
+ *
+ * @param rdd Ignite RDD instance.
+ * @tparam K Key type.
+ * @tparam V Value type.
+ */
+class JavaIgniteRDD[K, V](override val rdd: IgniteRDD[K, V])
+    extends JavaPairRDD[K, V](rdd)(JavaIgniteRDD.fakeClassTag, JavaIgniteRDD.fakeClassTag) {
+
+    override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd)
+
+    override val classTag: ClassTag[(K, V)] = JavaIgniteRDD.fakeClassTag
+
+    /**
+     * Computes iterator based on given partition.
+     *
+     * @param part Partition to use.
+     * @param context Task context.
+     * @return Partition iterator.
+     */
+    def compute(part: Partition, context: TaskContext): Iterator[(K, V)] = {
+        rdd.compute(part, context)
+    }
+
+    /**
+     * Gets partitions for the given cache RDD.
+     *
+     * @return Partitions.
+     */
+    protected def getPartitions: java.util.List[Partition] = {
+        new util.ArrayList[Partition](rdd.getPartitions.toSeq)
+    }
+
+    /**
+     * Gets preferred locations for the given partition.
+     *
+     * @param split Split partition.
+     * @return
+     */
+    protected def getPreferredLocations(split: Partition): Seq[String] = {
+        rdd.getPreferredLocations(split)
+    }
+
+    @varargs def objectSql(typeName: String, sql: String, args: Any*): JavaPairRDD[K, V] =
+        JavaPairRDD.fromRDD(rdd.objectSql(typeName, sql, args:_*))
+
+    @varargs def sql(sql: String, args: Any*): DataFrame = rdd.sql(sql, args:_*)
+
+    def saveValues(jrdd: JavaRDD[V]) = rdd.saveValues(JavaRDD.toRDD(jrdd))
+
+    def savePairs(jrdd: JavaPairRDD[K, V]) = {
+        val rrdd: RDD[(K, V)] = JavaPairRDD.toRDD(jrdd)
+
+        rdd.savePairs(rrdd)
+    }
+
+    def clear(): Unit = rdd.clear()
+}
+
+object JavaIgniteRDD {
+    implicit def fromIgniteRDD[K: ClassTag, V: ClassTag](rdd: IgniteRDD[K, V]): JavaIgniteRDD[K, V] =
+        new JavaIgniteRDD[K, V](rdd)
+
+    implicit def toIgniteRDD[K, V](rdd: JavaIgniteRDD[K, V]): IgniteRDD[K, V] = rdd.rdd
+
+    def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]]
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
index e1d3d8e..29587e4 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/ColocationTest.scala
@@ -17,16 +17,15 @@
 
 package org.apache.ignite.spark.examples
 
-import org.apache.ignite.configuration.IgniteConfiguration
 import org.apache.ignite.spark.IgniteContext
-import org.apache.spark.{SparkContext, SparkConf}
+import org.apache.spark.{SparkConf, SparkContext}
 
 object ColocationTest {
     def main(args: Array[String]) {
         val conf = new SparkConf().setAppName("Colocation test")
         val sc = new SparkContext(conf)
 
-        val ignite = new IgniteContext[Int, Int](sc, () ⇒ new IgniteConfiguration())
+        val ignite = new IgniteContext[Int, Int](sc, ExampleConfiguration.configuration _)
 
         // Search for lines containing "Ignite".
         val cache = ignite.fromCache("partitioned")

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/main/scala/org/apache/ignite/spark/impl/JavaIgniteAbstractRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/JavaIgniteAbstractRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/JavaIgniteAbstractRDD.scala
new file mode 100644
index 0000000..13bd3e8
--- /dev/null
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/JavaIgniteAbstractRDD.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spark.impl
+
+import org.apache.ignite.IgniteCache
+import org.apache.ignite.spark.IgniteRDD
+import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike}
+
+abstract class JavaIgniteAbstractRDD[K, V](val rdd: IgniteRDD[K, V])
+    extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] {
+
+    protected def ensureCache(): IgniteCache[K, V] = {
+        // Make sure to deploy the cache
+        if (rdd.cacheCfg != null)
+            rdd.ic.ignite().getOrCreateCache(rdd.cacheCfg)
+        else
+            rdd.ic.ignite().getOrCreateCache(rdd.cacheName)
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/modules/spark/src/test/java/org/apache/ignite/spark/JavaIgniteRDDSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/java/org/apache/ignite/spark/JavaIgniteRDDSelfTest.java b/modules/spark/src/test/java/org/apache/ignite/spark/JavaIgniteRDDSelfTest.java
new file mode 100644
index 0000000..e14abfc
--- /dev/null
+++ b/modules/spark/src/test/java/org/apache/ignite/spark/JavaIgniteRDDSelfTest.java
@@ -0,0 +1,298 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spark;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.spark.api.java.*;
+import org.apache.spark.api.java.function.*;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.sql.*;
+
+import scala.*;
+
+import java.util.*;
+
+/**
+ * Tests for {@link JavaIgniteRDD}.
+ */
+public class JavaIgniteRDDSelfTest extends GridCommonAbstractTest {
+    /** Grid count. */
+    private static final int GRID_CNT = 3;
+
+    /** Keys count. */
+    private static final int KEYS_CNT = 10000;
+
+    /** Cache name. */
+    private static final String PARTITIONED_CACHE_NAME = "partitioned";
+
+    /** Ip finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Sum function. */
+    private static final Function2<Integer, Integer, Integer> SUM_F = new Function2<Integer, Integer, Integer>() {
+        public Integer call(Integer x, Integer y) {
+            return x + y;
+        }
+    };
+
+    /** To pair function. */
+    private static final PairFunction<Integer, String, String> TO_PAIR_F = new PairFunction<Integer, String, String>() {
+        /** {@inheritDoc} */
+        @Override public Tuple2<String, String> call(Integer i) {
+            return new Tuple2<>(String.valueOf(i), "val" + i);
+        }
+    };
+
+    /** (String, Integer); pair to Integer value function. */
+    private static final Function<Tuple2<String, Integer>, Integer> STR_INT_PAIR_TO_INT_F = new PairToValueFunction<>();
+
+    /** (String, Entity) pair to Entity value function. */
+    private static final Function<Tuple2<String, Entity>, Entity> STR_ENTITY_PAIR_TO_ENTITY_F =
+        new PairToValueFunction<>();
+
+    /** Integer to entity function. */
+    private static final PairFunction<Integer, String, Entity> INT_TO_ENTITY_F =
+        new PairFunction<Integer, String, Entity>() {
+            @Override public Tuple2<String, Entity> call(Integer i) throws Exception {
+                return new Tuple2<>(String.valueOf(i), new Entity(i, "name" + i, i * 100));
+            }
+        };
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        Ignition.ignite("grid-0").cache(PARTITIONED_CACHE_NAME).removeAll();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        Ignition.stop("client", false);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        for (int i = 0; i < GRID_CNT; i++)
+            Ignition.start(getConfiguration("grid-" + i, false));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        for (int i = 0; i < GRID_CNT; i++)
+            Ignition.stop("grid-" + i, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStoreDataToIgnite() throws Exception {
+        JavaSparkContext sc = new JavaSparkContext("local[*]", "test");
+
+        try {
+            JavaIgniteContext<String, String> ic = new JavaIgniteContext<>(sc, new IgniteConfigProvider());
+
+            ic.fromCache(PARTITIONED_CACHE_NAME)
+                .savePairs(sc.parallelize(F.range(0, KEYS_CNT), 2).mapToPair(TO_PAIR_F));
+
+            Ignite ignite = Ignition.ignite("grid-0");
+
+            IgniteCache<String, String> cache = ignite.cache(PARTITIONED_CACHE_NAME);
+
+            for (int i = 0; i < KEYS_CNT; i++) {
+                String val = cache.get(String.valueOf(i));
+
+                assertNotNull("Value was not put to cache for key: " + i, val);
+                assertEquals("Invalid value stored for key: " + i, "val" + i, val);
+            }
+        }
+        finally {
+            sc.stop();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadDataFromIgnite() throws Exception {
+        JavaSparkContext sc = new JavaSparkContext("local[*]", "test");
+
+        try {
+            JavaIgniteContext<String, Integer> ic = new JavaIgniteContext<>(sc, new IgniteConfigProvider());
+
+            Ignite ignite = Ignition.ignite("grid-0");
+
+            IgniteCache<String, Integer> cache = ignite.cache(PARTITIONED_CACHE_NAME);
+
+            for (int i = 0; i < KEYS_CNT; i++)
+                cache.put(String.valueOf(i), i);
+
+            JavaRDD<Integer> values = ic.fromCache(PARTITIONED_CACHE_NAME).map(STR_INT_PAIR_TO_INT_F);
+
+            int sum = values.fold(0, SUM_F);
+
+            int expSum = (KEYS_CNT * KEYS_CNT + KEYS_CNT) / 2 - KEYS_CNT;
+
+            assertEquals(expSum, sum);
+        }
+        finally {
+            sc.stop();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryObjectsFromIgnite() throws Exception {
+        JavaSparkContext sc = new JavaSparkContext("local[*]", "test");
+
+        try {
+            JavaIgniteContext<String, Entity> ic = new JavaIgniteContext<>(sc, new IgniteConfigProvider());
+
+            JavaIgniteRDD<String, Entity> cache = ic.fromCache(PARTITIONED_CACHE_NAME);
+
+            cache.savePairs(sc.parallelize(F.range(0, 1001), 2).mapToPair(INT_TO_ENTITY_F));
+
+            List<Entity> res = cache.objectSql("Entity", "name = ? and salary = ?", "name50", 5000)
+                .map(STR_ENTITY_PAIR_TO_ENTITY_F).collect();
+
+            assertEquals("Invalid result length", 1, res.size());
+            assertEquals("Invalid result", 50, res.get(0).id());
+            assertEquals("Invalid result", "name50", res.get(0).name());
+            assertEquals("Invalid result", 5000, res.get(0).salary());
+            assertEquals("Invalid count", 500, cache.objectSql("Entity", "id > 500").count());
+        }
+        finally {
+            sc.stop();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryFieldsFromIgnite() throws Exception {
+        JavaSparkContext sc = new JavaSparkContext("local[*]", "test");
+
+        try {
+            JavaIgniteContext<String, Entity> ic = new JavaIgniteContext<>(sc, new IgniteConfigProvider());
+
+            JavaIgniteRDD<String, Entity> cache = ic.fromCache(PARTITIONED_CACHE_NAME);
+
+            cache.savePairs(sc.parallelize(F.range(0, 1001), 2).mapToPair(INT_TO_ENTITY_F));
+
+            DataFrame df =
+                cache.sql("select id, name, salary from Entity where name = ? and salary = ?", "name50", 5000);
+
+            df.printSchema();
+
+            Row[] res = df.collect();
+
+            assertEquals("Invalid result length", 1, res.length);
+            assertEquals("Invalid result", 50, res[0].get(0));
+            assertEquals("Invalid result", "name50", res[0].get(1));
+            assertEquals("Invalid result", 5000, res[0].get(2));
+
+            Column exp = new Column("NAME").equalTo("name50").and(new Column("SALARY").equalTo(5000));
+
+            DataFrame df0 = cache.sql("select id, name, salary from Entity").where(exp);
+
+            df.printSchema();
+
+            Row[] res0 = df0.collect();
+
+            assertEquals("Invalid result length", 1, res0.length);
+            assertEquals("Invalid result", 50, res0[0].get(0));
+            assertEquals("Invalid result", "name50", res0[0].get(1));
+            assertEquals("Invalid result", 5000, res0[0].get(2));
+
+            assertEquals("Invalid count", 500, cache.sql("select id from Entity where id > 500").count());
+        }
+        finally {
+            sc.stop();
+        }
+
+    }
+
+    /**
+     * @param gridName Grid name.
+     * @param client Client.
+     */
+    private static IgniteConfiguration getConfiguration(String gridName, boolean client) throws Exception {
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        cfg.setCacheConfiguration(cacheConfiguration());
+
+        cfg.setClientMode(client);
+
+        cfg.setGridName(gridName);
+
+        return cfg;
+    }
+
+    /**
+     * Creates cache configuration.
+     */
+    private static CacheConfiguration<Object, Object> cacheConfiguration() {
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setBackups(1);
+
+        ccfg.setName(PARTITIONED_CACHE_NAME);
+
+        ccfg.setIndexedTypes(String.class, Entity.class);
+
+        return ccfg;
+    }
+
+    /**
+     * Ignite configiration provider.
+     */
+    static class IgniteConfigProvider implements IgniteOutClosure<IgniteConfiguration> {
+        /** {@inheritDoc} */
+        @Override public IgniteConfiguration apply() {
+            try {
+                return getConfiguration("client", true);
+            }
+            catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    /**
+     * @param <K>
+     * @param <V>
+     */
+    static class PairToValueFunction<K, V> implements Function<Tuple2<K, V>, V> {
+        /** {@inheritDoc} */
+        @Override public V call(Tuple2<K, V> t) throws Exception {
+            return t._2();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d36d123/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index a514e35..f5b73df 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -321,6 +321,10 @@
                                 <title>Mesos Framework</title>
                                 <packages>org.apache.ignite.mesos*</packages>
                             </group>
+                            <group>
+                                <title>Spark Integration</title>
+                                <packages>org.apache.ignite.spark.examples.java</packages>
+                            </group>
                         </groups>
                         <header>
                             <![CDATA[



[08/50] incubator-ignite git commit: IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389

Posted by sb...@apache.org.
IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389


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

Branch: refs/heads/ignite-745
Commit: d0157d4ef6091105245aee8d2e6698028ce3b95f
Parents: f8910f6
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 1 15:18:40 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 1 15:18:40 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/processors/service/GridServiceProcessor.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0157d4e/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 2e31b69..aeb48f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -932,7 +932,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
             GridCacheQueryManager qryMgr = cache.context().queries();
 
-            CacheQuery<Map.Entry<Object, Object>> qry = qryMgr.createScanQuery(p, false);
+            CacheQuery<Map.Entry<Object, Object>> qry = qryMgr.createScanQuery(p, null, false);
 
             qry.keepAll(false);
 


[20/50] incubator-ignite git commit: IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389-ipc

Posted by sb...@apache.org.
IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389-ipc


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

Branch: refs/heads/ignite-745
Commit: 1d8643c0b93786f7eeff82bb56b64e6df53b3697
Parents: a329e90 c9f7291
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jun 4 11:09:30 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jun 4 11:09:30 2015 -0700

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  13 +-
 .../apache/ignite/internal/IgniteKernal.java    |   3 +
 .../managers/communication/GridIoManager.java   | 117 ++++----
 .../processors/cache/GridCacheContext.java      |   3 -
 .../dht/GridClientPartitionTopology.java        |   2 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  16 +-
 .../GridDhtPartitionsExchangeFuture.java        |  29 +-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../processors/hadoop/HadoopTaskContext.java    |  14 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   2 +-
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   3 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  31 --
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  56 +++-
 .../tcp/ipfinder/TcpDiscoveryIpFinder.java      |  10 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  47 +++-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  62 ++++
 ...niteDynamicCacheWithConfigStartSelfTest.java |  35 +--
 .../igfs/IgfsClientCacheSelfTest.java           |   9 +-
 .../IgniteMessagingWithClientTest.java          | 164 +++++++++++
 .../tcp/TcpClientDiscoverySpiMulticastTest.java | 129 +++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   1 +
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |  43 +--
 .../fs/IgniteHadoopFileSystemCounterWriter.java |  14 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  70 ++---
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   2 +-
 .../internal/processors/hadoop/HadoopUtils.java | 282 ++++++++++++++++++-
 .../hadoop/SecondaryFileSystemProvider.java     |   4 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  48 +++-
 .../hadoop/HadoopClientProtocolSelfTest.java    |   6 +-
 .../hadoop/HadoopAbstractSelfTest.java          |  14 +-
 .../hadoop/HadoopCommandLineTest.java           |  14 +-
 .../processors/hadoop/HadoopMapReduceTest.java  | 176 +++++++++++-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |  15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |  12 +
 ...acheConfigurationPrimitiveTypesSelfTest.java | 104 +++++++
 .../IgniteCacheWithIndexingTestSuite.java       |   2 +
 .../commands/cache/VisorCacheScanCommand.scala  |   2 +-
 48 files changed, 1358 insertions(+), 299 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1d8643c0/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1d8643c0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1d8643c0/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
----------------------------------------------------------------------


[29/50] incubator-ignite git commit: IGNITE-389 - Fixing tests.

Posted by sb...@apache.org.
IGNITE-389 - Fixing tests.


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

Branch: refs/heads/ignite-745
Commit: 224cbcb1fbd283a3015b73fecfbf364cc2670ff1
Parents: 2c3acf0
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 8 17:41:33 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 8 17:41:33 2015 -0700

----------------------------------------------------------------------
 examples/config/example-ignite.xml                               | 4 +---
 .../internal/processors/cache/query/GridCacheQueryAdapter.java   | 1 +
 .../internal/processors/datastructures/GridCacheSetImpl.java     | 2 +-
 3 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/224cbcb1/examples/config/example-ignite.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-ignite.xml b/examples/config/example-ignite.xml
index dcb2ba8..e746e59 100644
--- a/examples/config/example-ignite.xml
+++ b/examples/config/example-ignite.xml
@@ -30,16 +30,14 @@
         http://www.springframework.org/schema/util/spring-util.xsd">
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <!-- Set to true to enable distributed class loading for examples, default is false. -->
-<!--
         <property name="peerClassLoadingEnabled" value="true"/>
 
         <property name="marshaller">
             <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
-                &lt;!&ndash; Set to false to allow non-serializable objects in examples, default is true. &ndash;&gt;
+                <!-- Set to false to allow non-serializable objects in examples, default is true. -->
                 <property name="requireSerializable" value="false"/>
             </bean>
         </property>
--->
 
         <!-- Enable task execution events for examples. -->
         <property name="includeEventTypes">

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/224cbcb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index eaf7515..5b82c34 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -123,6 +123,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         boolean keepPortable) {
         assert cctx != null;
         assert type != null;
+        assert part == null || part >= 0;
 
         this.cctx = cctx;
         this.type = type;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/224cbcb1/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
index c0e763f..f74fe95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
@@ -114,7 +114,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
             }
 
             CacheQuery qry = new GridCacheQueryAdapter<>(ctx, SET, null, null,
-                new GridSetQueryPredicate<>(id, collocated), -1, false, false);
+                new GridSetQueryPredicate<>(id, collocated), null, false, false);
 
             Collection<ClusterNode> nodes = dataNodes(ctx.affinity().affinityTopologyVersion());
 


[24/50] incubator-ignite git commit: Merge branch ignite-sprint-5 into ignite-389

Posted by sb...@apache.org.
Merge branch ignite-sprint-5 into ignite-389


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

Branch: refs/heads/ignite-745
Commit: 5872b7f20064fb19d682ae2204ce83c4c9a58183
Parents: 0c2d381 c9f7291
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jun 4 19:22:20 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jun 4 19:22:20 2015 -0700

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  13 +-
 .../apache/ignite/internal/IgniteKernal.java    |   3 +
 .../managers/communication/GridIoManager.java   | 117 ++++----
 .../processors/cache/GridCacheContext.java      |   3 -
 .../dht/GridClientPartitionTopology.java        |   2 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  16 +-
 .../GridDhtPartitionsExchangeFuture.java        |  29 +-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../processors/hadoop/HadoopTaskContext.java    |  14 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   2 +-
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   3 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  31 --
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  56 +++-
 .../tcp/ipfinder/TcpDiscoveryIpFinder.java      |  10 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  47 +++-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  62 ++++
 ...niteDynamicCacheWithConfigStartSelfTest.java |  35 +--
 .../igfs/IgfsClientCacheSelfTest.java           |   9 +-
 .../IgniteMessagingWithClientTest.java          | 164 +++++++++++
 .../tcp/TcpClientDiscoverySpiMulticastTest.java | 129 +++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   1 +
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |  43 +--
 .../fs/IgniteHadoopFileSystemCounterWriter.java |  14 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  70 ++---
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   2 +-
 .../internal/processors/hadoop/HadoopUtils.java | 282 ++++++++++++++++++-
 .../hadoop/SecondaryFileSystemProvider.java     |   4 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  48 +++-
 .../hadoop/HadoopClientProtocolSelfTest.java    |   6 +-
 .../hadoop/HadoopAbstractSelfTest.java          |  14 +-
 .../hadoop/HadoopCommandLineTest.java           |  14 +-
 .../processors/hadoop/HadoopMapReduceTest.java  | 176 +++++++++++-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |  15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |  12 +
 ...acheConfigurationPrimitiveTypesSelfTest.java | 104 +++++++
 .../IgniteCacheWithIndexingTestSuite.java       |   2 +
 .../commands/cache/VisorCacheScanCommand.scala  |   2 +-
 48 files changed, 1358 insertions(+), 299 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5872b7f2/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5872b7f2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------


[19/50] incubator-ignite git commit: Merge branch 'ignite-970' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-389-ipc

Posted by sb...@apache.org.
Merge branch 'ignite-970' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-389-ipc


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

Branch: refs/heads/ignite-745
Commit: a329e901d47419d5ab5e1db55dee6d2001f9d66e
Parents: 7ee51ba 7158fb6
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Jun 3 15:22:36 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Jun 3 15:22:36 2015 -0700

----------------------------------------------------------------------
 modules/core/pom.xml                            |   1 -
 .../util/nio/GridShmemCommunicationClient.java  | 151 +++++++
 .../communication/tcp/TcpCommunicationSpi.java  | 414 ++++++++++++++++++-
 .../tcp/TcpCommunicationSpiMBean.java           |   8 +
 .../IgniteCacheMessageRecoveryAbstractTest.java |   1 +
 .../communication/GridIoManagerBenchmark0.java  |   1 +
 .../spi/GridTcpSpiForwardingSelfTest.java       |   1 +
 .../GridTcpCommunicationSpiAbstractTest.java    |  13 +
 ...mmunicationSpiConcurrentConnectSelfTest.java |   4 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |  21 +-
 ...pCommunicationSpiMultithreadedShmemTest.java |  28 ++
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |   1 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   1 +
 .../GridTcpCommunicationSpiShmemSelfTest.java   |  38 ++
 .../tcp/GridTcpCommunicationSpiTcpSelfTest.java |   7 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |   2 +
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  13 +
 ...oopSecondaryFileSystemConfigurationTest.java |  14 +
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |   7 +
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   7 +
 .../hadoop/HadoopAbstractSelfTest.java          |   6 +
 21 files changed, 718 insertions(+), 21 deletions(-)
----------------------------------------------------------------------



[28/50] incubator-ignite git commit: IGNITE-389 - Fixing shmem tests.

Posted by sb...@apache.org.
IGNITE-389 - Fixing shmem tests.


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

Branch: refs/heads/ignite-745
Commit: 2c3acf0e7747fee9bc565b74670e43d9858c5387
Parents: 3d1e534
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 8 16:27:31 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 8 16:27:31 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/internal/util/GridJavaProcess.java   | 2 +-
 .../ignite/internal/util/nio/GridShmemCommunicationClient.java  | 5 -----
 .../internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java     | 2 ++
 3 files changed, 3 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3acf0e/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
index 42fe089..4946eb2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
@@ -138,7 +138,7 @@ public final class GridJavaProcess {
         procCommands.add(javaBin);
         procCommands.addAll(jvmArgs == null ? U.jvmArgs() : jvmArgs);
 
-        if (!jvmArgs.contains("-cp") && !jvmArgs.contains("-classpath")) {
+        if (jvmArgs == null || (!jvmArgs.contains("-cp") && !jvmArgs.contains("-classpath"))) {
             String classpath = System.getProperty("java.class.path");
 
             String sfcp = System.getProperty("surefire.test.class.path");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3acf0e/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
index f3dc46f..e05c37a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
@@ -140,11 +140,6 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
     }
 
     /** {@inheritDoc} */
-    @Override public void flushIfNeeded(long timeout) throws IOException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridShmemCommunicationClient.class, this, super.toString());
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3acf0e/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java
index 1a8fd10..e220031 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IgfsSharedMemoryTestServer.java
@@ -49,6 +49,8 @@ public class IgfsSharedMemoryTestServer {
 
             srv.start();
 
+            System.out.println("IPC shared memory server endpoint started");
+
             IpcEndpoint clientEndpoint = srv.accept();
 
             is = clientEndpoint.inputStream();


[50/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-998' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-998' into ignite-sprint-5


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

Branch: refs/heads/ignite-745
Commit: 4375529fa929e650f7b68d750318d67a8609ee10
Parents: b52e47a 4ab301d
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 11 16:06:03 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 11 16:06:03 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  38 +------
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 114 ++++++++++++++++++-
 2 files changed, 115 insertions(+), 37 deletions(-)
----------------------------------------------------------------------



[32/50] incubator-ignite git commit: ignite-389 Partition scan fallback test fixed

Posted by sb...@apache.org.
ignite-389 Partition scan fallback test fixed


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

Branch: refs/heads/ignite-745
Commit: 9fca6b5005a6f5ddd16af936c6445748b398ed39
Parents: 7e8f648
Author: agura <ag...@gridgain.com>
Authored: Tue Jun 9 14:42:49 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jun 9 14:42:49 2015 +0300

----------------------------------------------------------------------
 ...CacheScanPartitionQueryFallbackSelfTest.java | 20 ++++++++++++++++----
 1 file changed, 16 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fca6b50/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
index dfa7296..b7f5fa8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
@@ -31,6 +31,8 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
@@ -47,6 +49,9 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     /** Keys count. */
     private static final int KEYS_CNT = 5000;
 
+    /** Ip finder. */
+    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
     /** Backups. */
     private int backups;
 
@@ -75,6 +80,13 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        cfg.setClientMode(clientMode);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+        discoSpi.setIpFinder(IP_FINDER);
+        discoSpi.setForceServerMode(true);
+        cfg.setDiscoverySpi(discoSpi);
+
         cfg.setCommunicationSpi(commSpiFactory.create());
 
         CacheConfiguration ccfg = defaultCacheConfiguration();
@@ -85,8 +97,6 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
         cfg.setCacheConfiguration(ccfg);
 
-        cfg.setClientMode(clientMode);
-
         return cfg;
     }
 
@@ -183,6 +193,7 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
                 if (!test.get()) {
                     candidates.addAll(localPartitions(ignite1));
+
                     candidates.retainAll(localPartitions(ignite2));
                 }
 
@@ -195,8 +206,9 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
                             awaitPartitionMapExchange();
 
                             if (!test.get()) {
-                                Set<Integer> parts = localPartitions(ignite1);
-                                candidates.removeAll(parts);
+                                candidates.removeAll(localPartitions(ignite1));
+
+                                F.retain(candidates, false, localPartitions(ignite2));
                             }
 
                             latch.countDown();


[47/50] incubator-ignite git commit: ScanQuery setPartition chaining style fixed

Posted by sb...@apache.org.
ScanQuery setPartition chaining style fixed


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

Branch: refs/heads/ignite-745
Commit: 4f500fdf65275ef066578578493212433075a201
Parents: d36610f
Author: agura <ag...@gridgain.com>
Authored: Thu Jun 11 13:33:15 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Jun 11 13:33:15 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/cache/query/ScanQuery.java  | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f500fdf/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
index e6b69bc..11a8c84 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
@@ -111,9 +111,12 @@ public final class ScanQuery<K, V> extends Query<Cache.Entry<K, V>> {
      * all partitions in the cache. Must be in the range [0, N) where N is partition number in the cache.
      *
      * @param part Partition number over which this query should iterate.
+     * @return {@code this} for chaining.
      */
-    public void setPartition(@Nullable Integer part) {
+    public ScanQuery<K, V> setPartition(@Nullable Integer part) {
         this.part = part;
+
+        return this;
     }
 
     /** {@inheritDoc} */


[22/50] incubator-ignite git commit: IGNITE-389 - IPC checked and API improvements.

Posted by sb...@apache.org.
IGNITE-389 - IPC checked and API improvements.


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

Branch: refs/heads/ignite-745
Commit: 6b51f99e72eb11af25403f8ec50087c03b1f1fb7
Parents: 1d8643c
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jun 4 19:19:36 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jun 4 19:19:36 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       |   4 +-
 .../shmem/IpcSharedMemoryClientEndpoint.java    |   2 +-
 .../ipc/shmem/IpcSharedMemoryNativeLoader.java  | 150 +++++++++++++++++--
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   2 +-
 .../IpcSharedMemoryCrashDetectionSelfTest.java  |   2 +-
 .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java |   2 +-
 .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java |   2 +-
 .../LoadWithCorruptedLibFileTestRunner.java     |   2 +-
 .../IpcSharedMemoryBenchmarkReader.java         |   2 +-
 .../IpcSharedMemoryBenchmarkWriter.java         |   2 +-
 .../hadoop/HadoopAbstractSelfTest.java          |   1 +
 .../org/apache/ignite/spark/IgniteContext.scala |  19 ++-
 .../org/apache/ignite/spark/IgniteRDD.scala     |   8 +-
 13 files changed, 171 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 0932212..9016b10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -9025,11 +9025,11 @@ public abstract class IgniteUtils {
                 hasShmem = false;
             else {
                 try {
-                    IpcSharedMemoryNativeLoader.load();
+                    IpcSharedMemoryNativeLoader.load(null);
 
                     hasShmem = true;
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteCheckedException ignore) {
                     hasShmem = false;
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java
index 27a234f..c935c4a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java
@@ -112,7 +112,7 @@ public class IpcSharedMemoryClientEndpoint implements IpcEndpoint {
         boolean clear = true;
 
         try {
-            IpcSharedMemoryNativeLoader.load();
+            IpcSharedMemoryNativeLoader.load(log);
 
             sock.connect(new InetSocketAddress("127.0.0.1", port), timeout);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
index dc00ca6..8c345f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.ipc.shmem;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
 import java.io.*;
@@ -25,6 +26,8 @@ import java.net.*;
 import java.nio.channels.*;
 import java.security.*;
 import java.util.*;
+import java.util.jar.*;
+import java.util.zip.*;
 
 import static org.apache.ignite.internal.IgniteVersionUtils.*;
 
@@ -36,6 +39,9 @@ public class IpcSharedMemoryNativeLoader {
     /** Library name base. */
     private static final String LIB_NAME_BASE = "igniteshmem";
 
+    /** Library jar name base. */
+    private static final String JAR_NAME_BASE = "shmem";
+
     /** Library name. */
     static final String LIB_NAME = LIB_NAME_BASE + "-" + VER_STR;
 
@@ -84,9 +90,10 @@ public class IpcSharedMemoryNativeLoader {
     }
 
     /**
+     * @param log Logger, if available. If null, warnings will be printed out to console.
      * @throws IgniteCheckedException If failed.
      */
-    public static void load() throws IgniteCheckedException {
+    public static void load(IgniteLogger log) throws IgniteCheckedException {
         if (loaded)
             return;
 
@@ -94,7 +101,7 @@ public class IpcSharedMemoryNativeLoader {
             if (loaded)
                 return;
 
-            doLoad();
+            doLoad(log);
 
             loaded = true;
         }
@@ -103,7 +110,7 @@ public class IpcSharedMemoryNativeLoader {
     /**
      * @throws IgniteCheckedException If failed.
      */
-    private static void doLoad() throws IgniteCheckedException {
+    private static void doLoad(IgniteLogger log) throws IgniteCheckedException {
         assert Thread.holdsLock(IpcSharedMemoryNativeLoader.class);
 
         Collection<Throwable> errs = new ArrayList<>();
@@ -124,7 +131,7 @@ public class IpcSharedMemoryNativeLoader {
 
         // Obtain lock on file to prevent concurrent extracts.
         try (RandomAccessFile randomAccessFile = new RandomAccessFile(lockFile, "rws");
-             FileLock ignored = randomAccessFile.getChannel().lock()) {
+            FileLock ignored = randomAccessFile.getChannel().lock()) {
             if (extractAndLoad(errs, tmpDir, platformSpecificResourcePath()))
                 return;
 
@@ -134,6 +141,30 @@ public class IpcSharedMemoryNativeLoader {
             if (extractAndLoad(errs, tmpDir, resourcePath()))
                 return;
 
+            try {
+                U.quietAndWarn(log, "Failed to load 'igniteshmem' library from classpath. Will try to load it from IGNITE_HOME.");
+
+                String igniteHome = X.resolveIgniteHome();
+
+                File shmemJar = findShmemJar(errs, igniteHome);
+
+                if (shmemJar != null) {
+                    try (JarFile jar = new JarFile(shmemJar, false, JarFile.OPEN_READ)) {
+                        if (extractAndLoad(errs, jar, tmpDir, platformSpecificResourcePath()))
+                            return;
+
+                        if (extractAndLoad(errs, jar, tmpDir, osSpecificResourcePath()))
+                            return;
+
+                        if (extractAndLoad(errs, jar, tmpDir, resourcePath()))
+                            return;
+                    }
+                }
+            }
+            catch (IgniteCheckedException ignore) {
+
+            }
+
             // Failed to find the library.
             assert !errs.isEmpty();
 
@@ -145,6 +176,32 @@ public class IpcSharedMemoryNativeLoader {
     }
 
     /**
+     * Tries to find shmem jar in IGNITE_HOME/libs folder.
+     *
+     * @param errs Collection of errors to add readable exception to.
+     * @param igniteHome Resolver IGNITE_HOME variable.
+     * @return File, if found.
+     */
+    private static File findShmemJar(Collection<Throwable> errs, String igniteHome) {
+        File libs = new File(igniteHome, "libs");
+
+        if (!libs.exists() || libs.isFile()) {
+            errs.add(new IllegalStateException("Failed to find libs folder in resolved IGNITE_HOME: " + igniteHome));
+
+            return null;
+        }
+
+        for (File lib : libs.listFiles()) {
+            if (lib.getName().endsWith(".jar") && lib.getName().contains(JAR_NAME_BASE))
+                return lib;
+        }
+
+        errs.add(new IllegalStateException("Failed to find shmem jar in resolved IGNITE_HOME: " + igniteHome));
+
+        return null;
+    }
+
+    /**
      * Gets temporary directory unique for each OS user.
      * The directory guaranteed to exist, though may not be empty.
      */
@@ -220,6 +277,24 @@ public class IpcSharedMemoryNativeLoader {
 
     /**
      * @param errs Errors collection.
+     * @param rsrcPath Path.
+     * @return {@code True} if library was found and loaded.
+     */
+    private static boolean extractAndLoad(Collection<Throwable> errs, JarFile jar, File tmpDir, String rsrcPath) {
+        ZipEntry rsrc = jar.getEntry(rsrcPath);
+
+        if (rsrc != null)
+            return extract(errs, rsrc, jar, new File(tmpDir, mapLibraryName(LIB_NAME)));
+        else {
+            errs.add(new IllegalStateException("Failed to find resource within specified jar file " +
+                "[rsrc=" + rsrcPath + ", jar=" + jar.getName() + ']'));
+
+            return false;
+        }
+    }
+
+    /**
+     * @param errs Errors collection.
      * @param src Source.
      * @param target Target.
      * @return {@code True} if resource was found and loaded.
@@ -230,7 +305,7 @@ public class IpcSharedMemoryNativeLoader {
         InputStream is = null;
 
         try {
-            if (!target.exists() || !haveEqualMD5(target, src)) {
+            if (!target.exists() || !haveEqualMD5(target, src.openStream())) {
                 is = src.openStream();
 
                 if (is != null) {
@@ -265,20 +340,69 @@ public class IpcSharedMemoryNativeLoader {
     }
 
     /**
-     * @param target Target.
+     * @param errs Errors collection.
      * @param src Source.
+     * @param target Target.
+     * @return {@code True} if resource was found and loaded.
+     */
+    @SuppressWarnings("ResultOfMethodCallIgnored")
+    private static boolean extract(Collection<Throwable> errs, ZipEntry src, JarFile jar, File target) {
+        FileOutputStream os = null;
+        InputStream is = null;
+
+        try {
+            if (!target.exists() || !haveEqualMD5(target, jar.getInputStream(src))) {
+                is = jar.getInputStream(src);
+
+                if (is != null) {
+                    os = new FileOutputStream(target);
+
+                    int read;
+
+                    byte[] buf = new byte[4096];
+
+                    while ((read = is.read(buf)) != -1)
+                        os.write(buf, 0, read);
+                }
+            }
+
+            // chmod 775.
+            if (!U.isWindows())
+                Runtime.getRuntime().exec(new String[] {"chmod", "775", target.getCanonicalPath()}).waitFor();
+
+            System.load(target.getPath());
+
+            return true;
+        }
+        catch (IOException | UnsatisfiedLinkError | InterruptedException | NoSuchAlgorithmException e) {
+            errs.add(e);
+        }
+        finally {
+            U.closeQuiet(os);
+            U.closeQuiet(is);
+        }
+
+        return false;
+    }
+
+    /**
+     * @param target Target.
+     * @param srcIS Source input stream.
      * @return {@code True} if target md5-sum equal to source md5-sum.
      * @throws NoSuchAlgorithmException If md5 algorithm was not found.
      * @throws IOException If an I/O exception occurs.
      */
-    private static boolean haveEqualMD5(File target, URL src) throws NoSuchAlgorithmException, IOException {
-        try (InputStream targetIS = new FileInputStream(target);
-             InputStream srcIS = src.openStream()) {
-
-            String targetMD5 = U.calculateMD5(targetIS);
-            String srcMD5 = U.calculateMD5(srcIS);
+    private static boolean haveEqualMD5(File target, InputStream srcIS) throws NoSuchAlgorithmException, IOException {
+        try {
+            try (InputStream targetIS = new FileInputStream(target)) {
+                String targetMD5 = U.calculateMD5(targetIS);
+                String srcMD5 = U.calculateMD5(srcIS);
 
-            return targetMD5.equals(srcMD5);
+                return targetMD5.equals(srcMD5);
+            }
+        }
+        finally {
+            srcIS.close();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
index 5185856..102c5b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
@@ -146,7 +146,7 @@ public class IpcSharedMemoryServerEndpoint implements IpcServerEndpoint {
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
-        IpcSharedMemoryNativeLoader.load();
+        IpcSharedMemoryNativeLoader.load(log);
 
         pid = IpcSharedMemoryUtils.pid();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryCrashDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryCrashDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryCrashDetectionSelfTest.java
index 2ddf6f3..c6f590e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryCrashDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryCrashDetectionSelfTest.java
@@ -42,7 +42,7 @@ public class IpcSharedMemoryCrashDetectionSelfTest extends GridCommonAbstractTes
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        IpcSharedMemoryNativeLoader.load();
+        IpcSharedMemoryNativeLoader.load(log());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemorySpaceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemorySpaceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemorySpaceSelfTest.java
index 7dc0870..4afb64b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemorySpaceSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemorySpaceSelfTest.java
@@ -51,7 +51,7 @@ public class IpcSharedMemorySpaceSelfTest extends GridCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        IpcSharedMemoryNativeLoader.load();
+        IpcSharedMemoryNativeLoader.load(log());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryUtilsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryUtilsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryUtilsSelfTest.java
index 4c5413c..176429e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryUtilsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryUtilsSelfTest.java
@@ -31,7 +31,7 @@ public class IpcSharedMemoryUtilsSelfTest extends GridCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        IpcSharedMemoryNativeLoader.load();
+        IpcSharedMemoryNativeLoader.load(log());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/LoadWithCorruptedLibFileTestRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/LoadWithCorruptedLibFileTestRunner.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/LoadWithCorruptedLibFileTestRunner.java
index 8ff827b..8fee239 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/LoadWithCorruptedLibFileTestRunner.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/LoadWithCorruptedLibFileTestRunner.java
@@ -37,7 +37,7 @@ public class LoadWithCorruptedLibFileTestRunner {
 
         createCorruptedLibFile();
 
-        IpcSharedMemoryNativeLoader.load();
+        IpcSharedMemoryNativeLoader.load(null);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkReader.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkReader.java
index 28495af..89eeda1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkReader.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkReader.java
@@ -43,7 +43,7 @@ public class IpcSharedMemoryBenchmarkReader implements IpcSharedMemoryBenchmarkP
      * @throws IgniteCheckedException If failed.
      */
     public static void main(String[] args) throws IgniteCheckedException {
-        IpcSharedMemoryNativeLoader.load();
+        IpcSharedMemoryNativeLoader.load(null);
 
         int nThreads = (args.length > 0 ? Integer.parseInt(args[0]) : 1);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkWriter.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkWriter.java
index 2ade145..e8a8402 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkWriter.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/benchmark/IpcSharedMemoryBenchmarkWriter.java
@@ -42,7 +42,7 @@ public class IpcSharedMemoryBenchmarkWriter implements IpcSharedMemoryBenchmarkP
      * @throws IgniteCheckedException If failed.
      */
     public static void main(String[] args) throws IgniteCheckedException {
-        IpcSharedMemoryNativeLoader.load();
+        IpcSharedMemoryNativeLoader.load(null);
 
         int nThreads = args.length > 0 ? Integer.parseInt(args[0]) : 1;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
index 517a587..a3c9bde 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem;
 import org.apache.ignite.internal.processors.hadoop.fs.*;
+import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
index 5cdbad0..2cfebd6 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
@@ -34,8 +34,23 @@ import org.apache.spark.sql.SQLContext
  */
 class IgniteContext[K, V](
     @scala.transient val sparkContext: SparkContext,
-    cfgF: () ⇒ IgniteConfiguration
+    cfgF: () ⇒ IgniteConfiguration,
+    client: Boolean = true
 ) extends Serializable {
+    @scala.transient private val driver = true
+
+    if (!client) {
+        val workers = sparkContext.getExecutorStorageStatus.length - 1
+
+        if (workers <= 0)
+            throw new IllegalStateException("No Spark executors found to start Ignite nodes.")
+
+        println("Will start Ignite nodes on " + workers + " workers")
+
+        // Start ignite server node on each worker in server mode.
+        sparkContext.parallelize(1 to workers, workers).foreach(it ⇒ ignite())
+    }
+
     def this(
         sc: SparkContext,
         springUrl: String
@@ -62,7 +77,7 @@ class IgniteContext[K, V](
         catch {
             case e: Exception ⇒
                 try {
-                    igniteCfg.setClientMode(true)
+                    igniteCfg.setClientMode(client || driver)
 
                     Ignition.start(igniteCfg)
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b51f99e/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index 0b8e845..0d1a3be 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -114,7 +114,7 @@ class IgniteRDD[K, V] (
         ic.sqlContext.createDataFrame(rowRdd, schema)
     }
 
-    def saveValues(rdd: RDD[V]) = {
+    def saveValues(rdd: RDD[V], overwrite: Boolean = false) = {
         rdd.foreachPartition(it ⇒ {
             val ig = ic.ignite()
 
@@ -127,6 +127,8 @@ class IgniteRDD[K, V] (
             val streamer = ig.dataStreamer[Object, V](cacheName)
 
             try {
+                streamer.allowOverwrite(overwrite)
+
                 it.foreach(value ⇒ {
                     val key = affinityKeyFunc(value, node.orNull)
 
@@ -139,7 +141,7 @@ class IgniteRDD[K, V] (
         })
     }
 
-    def savePairs(rdd: RDD[(K, V)]) = {
+    def savePairs(rdd: RDD[(K, V)], overwrite: Boolean = false) = {
         rdd.foreachPartition(it ⇒ {
             val ig = ic.ignite()
 
@@ -149,6 +151,8 @@ class IgniteRDD[K, V] (
             val streamer = ig.dataStreamer[K, V](cacheName)
 
             try {
+                streamer.allowOverwrite(overwrite)
+
                 it.foreach(tup ⇒ {
                     streamer.addData(tup._1, tup._2)
                 })


[12/50] incubator-ignite git commit: # ignite-970

Posted by sb...@apache.org.
# ignite-970


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

Branch: refs/heads/ignite-745
Commit: 7158fb6a4ff7b9db3afda73f75376ad3285c556c
Parents: d6f9b64
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 15:57:16 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 15:57:16 2015 +0300

----------------------------------------------------------------------
 .../GridTcpCommunicationSpiMultithreadedSelfTest.java   | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7158fb6a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index 5d25299..dc7f344 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -62,8 +62,7 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
     private static final Collection<IgniteTestResources> spiRsrcs = new ArrayList<>();
 
     /** SPIs */
-    private static final Map<UUID, CommunicationSpi<Message>> spis =
-        new ConcurrentHashMap<>();
+    private static final Map<UUID, CommunicationSpi<Message>> spis = new ConcurrentHashMap<>();
 
     /** Listeners. */
     private static final Map<UUID, MessageListener> lsnrs = new HashMap<>();
@@ -85,13 +84,20 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
     /**
      * @param useShmem Use shared mem.
      */
-    public GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
+    protected GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
         super(false);
 
         this.useShmem = useShmem;
     }
 
     /**
+     *
+     */
+    public GridTcpCommunicationSpiMultithreadedSelfTest() {
+        this(false);
+    }
+
+    /**
      * Accumulating listener.
      */
     @SuppressWarnings({"deprecation"})


[02/50] incubator-ignite git commit: ignite-389 Partition scan query fallback test

Posted by sb...@apache.org.
ignite-389 Partition scan query fallback test


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

Branch: refs/heads/ignite-745
Commit: 29dc7221c12db1e39a17de4471a8c5ebed4b8709
Parents: 5d6bb53
Author: agura <ag...@gridgain.com>
Authored: Fri May 29 16:28:34 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Fri May 29 16:28:34 2015 +0300

----------------------------------------------------------------------
 ...CacheScanPartitionQueryFallbackSelfTest.java | 335 ++++++++++++++-----
 1 file changed, 259 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/29dc7221/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
index 31336e6..dfa7296 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheScanPartitionQueryFallbackSelfTest.java
@@ -21,8 +21,10 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
@@ -32,15 +34,17 @@ import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
 
 /**
  * Tests partition scan query fallback.
  */
 public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractTest {
     /** Grid count. */
-    private static final int GRID_CNT = 5;
+    private static final int GRID_CNT = 3;
 
-    /** Kys count. */
+    /** Keys count. */
     private static final int KEYS_CNT = 5000;
 
     /** Backups. */
@@ -49,20 +53,29 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
     /** Cache mode. */
     private CacheMode cacheMode;
 
-    /** Fallback. */
-    private boolean fallback;
+    /** Client mode. */
+    private volatile boolean clientMode;
 
-    /** Primary node id. */
-    private static volatile UUID expNodeId;
+    /** Expected first node ID. */
+    private static UUID expNodeId;
 
-    /** Fail node id. */
-    private static volatile UUID failNodeId;
+    /** Expected fallback node ID. */
+    private static UUID expFallbackNodeId;
+
+    /** Communication SPI factory. */
+    private CommunicationSpiFactory commSpiFactory;
+
+    /** Latch. */
+    private static CountDownLatch latch;
+
+    /** Test entries. */
+    private Map<Integer, Map<Integer, Integer>> entries = new HashMap<>();
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setCommunicationSpi(new TestCommunicationSpi());
+        cfg.setCommunicationSpi(commSpiFactory.create());
 
         CacheConfiguration ccfg = defaultCacheConfiguration();
         ccfg.setCacheMode(cacheMode);
@@ -72,142 +85,312 @@ public class CacheScanPartitionQueryFallbackSelfTest extends GridCommonAbstractT
 
         cfg.setCacheConfiguration(ccfg);
 
+        cfg.setClientMode(clientMode);
+
         return cfg;
     }
 
     /**
+     * Scan should perform on the local node.
+     *
      * @throws Exception If failed.
      */
-    public void testPrimary() throws Exception {
+    public void testScanLocal() throws Exception {
         cacheMode = CacheMode.PARTITIONED;
         backups = 0;
-        failNodeId = null;
-        fallback = false;
+        commSpiFactory = new TestLocalCommunicationSpiFactory();
 
-        doTestScanPartition();
+        try {
+            Ignite ignite = startGrids(GRID_CNT);
+
+            IgniteCacheProxy<Integer, Integer> cache = fillCache(ignite);
+
+            int part = anyLocalPartition(cache.context());
+
+            CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(null, part, false);
+
+            doTestScanQuery(qry);
+        }
+        finally {
+            stopAllGrids();
+        }
     }
 
     /**
+     * Scan should perform on the remote node.
+     *
      * @throws Exception If failed.
      */
-    public void testFallbackToBackup() throws Exception {
+    public void testScanRemote() throws Exception {
         cacheMode = CacheMode.PARTITIONED;
-        backups = 1;
-        failNodeId = null;
-        fallback = true;
+        backups = 0;
+        commSpiFactory = new TestRemoteCommunicationSpiFactory();
 
-        doTestScanPartition();
+        try {
+            Ignite ignite = startGrids(GRID_CNT);
+
+            IgniteCacheProxy<Integer, Integer> cache = fillCache(ignite);
+
+            IgniteBiTuple<Integer, UUID> tup = remotePartition(cache.context());
+
+            int part = tup.get1();
+
+            expNodeId = tup.get2();
+
+            CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(null, part, false);
+
+            doTestScanQuery(qry);
+        }
+        finally {
+            stopAllGrids();
+        }
     }
 
     /**
+     * Scan should try first remote node and fallbacks to second remote node.
+     *
      * @throws Exception If failed.
      */
-    protected void doTestScanPartition() throws Exception {
-        try {
-            Ignite ignite = startGrids(GRID_CNT);
+    public void testScanFallback() throws Exception {
+        cacheMode = CacheMode.PARTITIONED;
+        backups = 1;
+        commSpiFactory = new TestFallbackCommunicationSpiFactory();
 
-            IgniteCacheProxy<Integer, Integer> cache =
-                (IgniteCacheProxy<Integer, Integer>)ignite.<Integer, Integer>cache(null);
+        final Set<Integer> candidates = new TreeSet<>();
 
-            Map<Integer, Map<Integer, Integer>> entries = new HashMap<>();
+        final AtomicBoolean test = new AtomicBoolean(false);
 
-            for (int i = 0; i < KEYS_CNT; i++) {
-                cache.put(i, i);
+        for(int j = 0; j < 2; j++) {
+            clientMode = true;
 
-                int part = cache.context().affinity().partition(i);
+            latch = new CountDownLatch(1);
 
-                Map<Integer, Integer> partEntries = entries.get(part);
+            try {
+                final Ignite ignite0 = startGrid(0);
 
-                if (partEntries == null)
-                    entries.put(part, partEntries = new HashMap<>());
+                clientMode = false;
 
-                partEntries.put(i, i);
-            }
+                final IgniteEx ignite1 = startGrid(1);
+                final IgniteEx ignite2 = startGrid(2);
+                startGrid(3);
 
-            IgniteBiTuple<Integer, UUID> tup = remotePartition(cache.context(), true);
+                if (test.get()) {
+                    expNodeId = ignite1.localNode().id();
+                    expFallbackNodeId = ignite2.localNode().id();
+                }
 
-            int part = tup.get1();
+                final IgniteCacheProxy<Integer, Integer> cache = fillCache(ignite0);
 
-            if (fallback)
-                failNodeId = tup.get2();
-            else
-                expNodeId = tup.get2();
+                if (!test.get()) {
+                    candidates.addAll(localPartitions(ignite1));
+                    candidates.retainAll(localPartitions(ignite2));
+                }
 
-            if (fallback)
-                expNodeId = remoteBackup(part, cache.context());
+                Runnable run = new Runnable() {
+                    @Override public void run() {
+                        try {
+                            startGrid(4);
+                            startGrid(5);
 
-            CacheQuery<Map.Entry<Integer, Integer>> qry = cache.context().queries().createScanQuery(null, part, false);
+                            awaitPartitionMapExchange();
+
+                            if (!test.get()) {
+                                Set<Integer> parts = localPartitions(ignite1);
+                                candidates.removeAll(parts);
+                            }
 
-            CacheQueryFuture<Map.Entry<Integer, Integer>> fut = qry.execute();
+                            latch.countDown();
+                        }
+                        catch (Exception e) {
+                            e.printStackTrace();
+                        }
 
-            Collection<Map.Entry<Integer, Integer>> expEntries = fut.get();
+                    }
+                };
 
-            for (Map.Entry<Integer, Integer> e : expEntries) {
-                Map<Integer, Integer> map = entries.get(part);
+                int part;
+                CacheQuery<Map.Entry<Integer, Integer>> qry = null;
 
-                if(map == null)
-                    assertTrue(expEntries.isEmpty());
+                if (test.get()) {
+                    part = F.first(candidates);
+
+                    qry = cache.context().queries().createScanQuery(null, part, false);
+                }
+
+                new Thread(run).start();
+
+                if (test.get())
+                    doTestScanQuery(qry);
                 else
-                    assertEquals(map.get(e.getKey()), e.getValue());
+                    latch.await();
+            }
+            finally {
+                test.set(true);
+
+                stopAllGrids();
             }
         }
-        finally {
-            stopAllGrids();
+    }
+
+    /**
+     * @param ignite Ignite.
+     */
+    protected IgniteCacheProxy<Integer, Integer> fillCache(Ignite ignite) {
+        IgniteCacheProxy<Integer, Integer> cache =
+            (IgniteCacheProxy<Integer, Integer>)ignite.<Integer, Integer>cache(null);
+
+        for (int i = 0; i < KEYS_CNT; i++) {
+            cache.put(i, i);
+
+            int part = cache.context().affinity().partition(i);
+
+            Map<Integer, Integer> partEntries = entries.get(part);
+
+            if (partEntries == null)
+                entries.put(part, partEntries = new HashMap<>());
+
+            partEntries.put(i, i);
         }
+
+        return cache;
+    }
+
+    /**
+     * @param qry Query.
+     */
+    protected void doTestScanQuery(
+        CacheQuery<Map.Entry<Integer, Integer>> qry) throws IgniteCheckedException {
+        CacheQueryFuture<Map.Entry<Integer, Integer>> fut = qry.execute();
+
+        Collection<Map.Entry<Integer, Integer>> expEntries = fut.get();
+
+        for (Map.Entry<Integer, Integer> e : expEntries) {
+            Map<Integer, Integer> map = entries.get(((GridCacheQueryAdapter)qry).partition());
+
+            if (map == null)
+                assertTrue(expEntries.isEmpty());
+            else
+                assertEquals(map.get(e.getKey()), e.getValue());
+        }
+    }
+
+    /**
+     * @param cctx Cctx.
+     */
+    private static int anyLocalPartition(GridCacheContext<?, ?> cctx) {
+        return F.first(cctx.topology().localPartitions()).id();
     }
 
     /**
      * @param cctx Cctx.
-     * @param primary Primary.
      */
-    private IgniteBiTuple<Integer, UUID> remotePartition(final GridCacheContext cctx, boolean primary) {
+    private IgniteBiTuple<Integer, UUID> remotePartition(final GridCacheContext cctx) {
         ClusterNode node = F.first(cctx.kernalContext().grid().cluster().forRemotes().nodes());
 
         GridCacheAffinityManager affMgr = cctx.affinity();
 
         AffinityTopologyVersion topVer = affMgr.affinityTopologyVersion();
 
-        Set<Integer> parts = primary ?
-            affMgr.primaryPartitions(node.id(), topVer) : affMgr.backupPartitions(node.id(), topVer);
+        Set<Integer> parts = affMgr.primaryPartitions(node.id(), topVer);
 
         return new IgniteBiTuple<>(F.first(parts), node.id());
     }
 
     /**
-     * @param part Partition.
-     * @param cctx Cctx.
+     * @param ignite Ignite.
      */
-    private UUID remoteBackup(int part, final GridCacheContext cctx) {
-        final UUID locUuid = cctx.localNodeId();
+    private Set<Integer> localPartitions(Ignite ignite) {
+        GridCacheContext cctx = ((IgniteCacheProxy)ignite.cache(null)).context();
+
+        Collection<GridDhtLocalPartition> owningParts = F.view(cctx.topology().localPartitions(),
+            new IgnitePredicate<GridDhtLocalPartition>() {
+                @Override public boolean apply(GridDhtLocalPartition part) {
+                    return part.state() == GridDhtPartitionState.OWNING;
+                }
+            });
+
+        return new HashSet<>(F.transform(owningParts, new IgniteClosure<GridDhtLocalPartition, Integer>() {
+            @Override public Integer apply(GridDhtLocalPartition part) {
+                return part.id();
+            }
+        }));
+    }
 
-        GridCacheAffinityManager affMgr = cctx.affinity();
+    /**
+     * Factory for tests specific communication SPI.
+     */
+    private interface CommunicationSpiFactory {
+        /**
+         * Creates communication SPI instance.
+         */
+        TcpCommunicationSpi create();
+    }
 
-        AffinityTopologyVersion topVer = affMgr.affinityTopologyVersion();
+    /**
+     *
+     */
+    private static class TestLocalCommunicationSpiFactory implements CommunicationSpiFactory {
+        /** {@inheritDoc} */
+        @Override public TcpCommunicationSpi create() {
+            return new TcpCommunicationSpi() {
+                @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+                    Object origMsg = ((GridIoMessage)msg).message();
 
-        return F.first(F.view(affMgr.backups(part, topVer), new IgnitePredicate<ClusterNode>() {
-            @Override public boolean apply(ClusterNode node) {
-                return !node.id().equals(locUuid);
-            }
-        })).id();
+                    if (origMsg instanceof GridCacheQueryRequest)
+                        fail(); //should use local node
+
+                    super.sendMessage(node, msg);
+                }
+            };
+        }
     }
 
     /**
      *
      */
-    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+    private static class TestRemoteCommunicationSpiFactory implements CommunicationSpiFactory {
         /** {@inheritDoc} */
-        @Override public void sendMessage(ClusterNode node, Message msg)
-            throws IgniteSpiException {
-            Object origMsg = ((GridIoMessage)msg).message();
+        @Override public TcpCommunicationSpi create() {
+            return new TcpCommunicationSpi() {
+                @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+                    Object origMsg = ((GridIoMessage)msg).message();
 
-            if (origMsg instanceof GridCacheQueryRequest) {
-                if (node.id().equals(failNodeId))
-                    throw new IgniteSpiException("");
-                else
-                    assertEquals(expNodeId, node.id());
-            }
+                    if (origMsg instanceof GridCacheQueryRequest)
+                        assertEquals(expNodeId, node.id());
+
+                    super.sendMessage(node, msg);
+                }
+            };
+        }
+    }
 
-            super.sendMessage(node, msg);
+    /**
+     *
+     */
+    private static class TestFallbackCommunicationSpiFactory implements CommunicationSpiFactory {
+        /** {@inheritDoc} */
+        @Override public TcpCommunicationSpi create() {
+            return new TcpCommunicationSpi() {
+                @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+                    Object origMsg = ((GridIoMessage)msg).message();
+
+                    if (origMsg instanceof GridCacheQueryRequest) {
+                        if (latch.getCount() > 0)
+                            assertEquals(expNodeId, node.id());
+                        else
+                            assertEquals(expFallbackNodeId, node.id());
+
+                        try {
+                            latch.await();
+                        }
+                        catch (InterruptedException e) {
+                            throw new IgniteSpiException(e);
+                        }
+                    }
+
+                    super.sendMessage(node, msg);
+                }
+            };
         }
     }
 }


[41/50] incubator-ignite git commit: IGNITE-389 - Fixing tests.

Posted by sb...@apache.org.
IGNITE-389 - Fixing tests.


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

Branch: refs/heads/ignite-745
Commit: 71f29e98e2ea571e437206a3712b7261e086e1db
Parents: 3417215
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Jun 10 13:22:15 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Jun 10 13:22:15 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/processors/query/GridQueryIndexing.java   | 4 +++-
 .../ignite/internal/processors/query/GridQueryProcessor.java  | 7 +++++--
 .../ignite/internal/processors/query/h2/IgniteH2Indexing.java | 7 ++++---
 .../processors/query/h2/twostep/GridReduceQueryExecutor.java  | 4 ++--
 4 files changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/71f29e98/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index cc0916a..7fcc284 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -58,9 +58,11 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepCacheObjects If {@code true}, cache objects representation will be preserved.
      * @return Cursor.
      */
-    public Iterable<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry);
+    public Iterable<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry,
+        boolean keepCacheObjects);
 
     /**
      * Parses SQL query into two step query and executes it.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/71f29e98/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 1be2a36..e187713 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -546,9 +546,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
+            GridCacheContext<Object, Object> cacheCtx = ctx.cache().internalCache(space).context();
+
             return idx.queryTwoStep(
-                ctx.cache().internalCache(space).context(),
-                qry);
+                cacheCtx,
+                qry,
+                cacheCtx.keepPortable());
         }
         finally {
             busyLock.leaveBusy();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/71f29e98/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 6ec329f..5e27c24 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -771,10 +771,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public Iterable<List<?>> queryTwoStep(final GridCacheContext<?,?> cctx, final GridCacheTwoStepQuery qry) {
+    @Override public Iterable<List<?>> queryTwoStep(final GridCacheContext<?,?> cctx, final GridCacheTwoStepQuery qry,
+        final boolean keepCacheObj) {
         return new Iterable<List<?>>() {
             @Override public Iterator<List<?>> iterator() {
-                return rdcQryExec.query(cctx, qry);
+                return rdcQryExec.query(cctx, qry, keepCacheObj);
             }
         };
     }
@@ -872,7 +873,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         twoStepQry.pageSize(qry.getPageSize());
 
-        QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(queryTwoStep(cctx, twoStepQry));
+        QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(queryTwoStep(cctx, twoStepQry, cctx.keepPortable()));
 
         cursor.fieldsMeta(meta);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/71f29e98/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index cfacfcf..11054b7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -269,7 +269,7 @@ public class GridReduceQueryExecutor {
      * @param qry Query.
      * @return Cursor.
      */
-    public Iterator<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry) {
+    public Iterator<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry, boolean keepPortable) {
         long qryReqId = reqIdGen.incrementAndGet();
 
         QueryRun r = new QueryRun();
@@ -356,7 +356,7 @@ public class GridReduceQueryExecutor {
 //                dropTable(r.conn, tbl.getName()); TODO
             }
 
-            return new GridQueryCacheObjectsIterator(new Iter(res), cctx, cctx.keepPortable());
+            return new GridQueryCacheObjectsIterator(new Iter(res), cctx, keepPortable);
         }
         catch (IgniteCheckedException | InterruptedException | RuntimeException e) {
             U.closeQuiet(r.conn);


[03/50] incubator-ignite git commit: #IGNITE-389 - More functions on API.

Posted by sb...@apache.org.
#IGNITE-389 - More functions on API.


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

Branch: refs/heads/ignite-745
Commit: 37a7679df3fd05473840482d0e5c2c9483d02b2a
Parents: 5d6bb53
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri May 29 08:46:52 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri May 29 08:46:52 2015 -0700

----------------------------------------------------------------------
 .../org/apache/ignite/spark/IgniteContext.scala | 13 ++++++++
 .../org/apache/ignite/spark/IgniteRDD.scala     | 32 +++++++++++++++++++-
 2 files changed, 44 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37a7679d/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
index a73405b..5b649db 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
@@ -23,6 +23,14 @@ import org.apache.ignite.{Ignition, Ignite}
 import org.apache.ignite.configuration.{CacheConfiguration, IgniteConfiguration}
 import org.apache.spark.SparkContext
 
+/**
+ * Ignite context.
+ *
+ * @param sparkContext Spark context.
+ * @param cfgF Configuration factory.
+ * @tparam K Key type.
+ * @tparam V Value type.
+ */
 class IgniteContext[K, V](
     @scala.transient val sparkContext: SparkContext,
     cfgF: () => IgniteConfiguration
@@ -61,4 +69,9 @@ class IgniteContext[K, V](
         }
     }
 
+    def close() = {
+        val igniteCfg = cfgF()
+
+        Ignition.stop(igniteCfg.getGridName, false)
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37a7679d/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index 30efa7a..358fcd4 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -29,12 +29,27 @@ import org.apache.spark.{TaskContext, Partition}
 
 import scala.collection.JavaConversions._
 
+/**
+ * Ignite RDD. Represents Ignite cache as Spark RDD abstraction.
+ *
+ * @param ic Ignite context to use.
+ * @param cacheName Cache name.
+ * @param cacheCfg Cache configuration.
+ * @tparam K Key type.
+ * @tparam V Value type.
+ */
 class IgniteRDD[K, V] (
     ic: IgniteContext[K, V],
     cacheName: String,
     cacheCfg: CacheConfiguration[K, V]
 ) extends IgniteAbstractRDD[(K, V), K, V] (ic, cacheName, cacheCfg) {
-
+    /**
+     * Computes iterator based on given partition.
+     *
+     * @param part Partition to use.
+     * @param context Task context.
+     * @return Partition iterator.
+     */
     override def compute(part: Partition, context: TaskContext): Iterator[(K, V)] = {
         val cache = ensureCache()
 
@@ -49,6 +64,11 @@ class IgniteRDD[K, V] (
         })
     }
 
+    /**
+     * Gets partitions for the given cache RDD.
+     *
+     * @return Partitions.
+     */
     override protected def getPartitions: Array[Partition] = {
         ensureCache()
 
@@ -57,6 +77,12 @@ class IgniteRDD[K, V] (
         (0 until parts).map(new IgnitePartition(_)).toArray
     }
 
+    /**
+     * Gets prefferred locations for the given partition.
+     *
+     * @param split Split partition.
+     * @return
+     */
     override protected def getPreferredLocations(split: Partition): Seq[String] = {
         ensureCache()
 
@@ -129,6 +155,10 @@ class IgniteRDD[K, V] (
         })
     }
 
+    def clear(): Unit = {
+        ensureCache().removeAll()
+    }
+
     private def affinityKeyFunc(value: V, node: ClusterNode): Object = {
         IgniteUuid.randomUuid()
     }


[15/50] incubator-ignite git commit: # IGNITE-956: Added support for scala-2.10.

Posted by sb...@apache.org.
# IGNITE-956: Added support for scala-2.10.


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

Branch: refs/heads/ignite-745
Commit: bae1804ece3eb677c7f3e2c01c8566486af83c7d
Parents: ccd6825
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Jun 3 16:29:06 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Jun 3 16:29:06 2015 +0700

----------------------------------------------------------------------
 modules/scalar-2.10/pom.xml        | 202 ++++++++++++++++----------------
 modules/spark-2.10/pom.xml         |  18 +++
 modules/visor-console-2.10/pom.xml |  96 +++++++++------
 3 files changed, 176 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bae1804e/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index b51c1b3..23eb13e 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -83,106 +83,106 @@
     </dependencies>
 
     <build>
-        <plugins>
-            <plugin>
-                <groupId>net.alchim31.maven</groupId>
-                <artifactId>scala-maven-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>scaladoc</id>
-                        <phase>prepare-package</phase>
-                        <goals>
-                            <goal>doc</goal>
-                        </goals>
-                        <configuration>
-                            <doctitle>Ignite Scalar</doctitle>
-                            <windowtitle>Ignite Scalar</windowtitle>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-antrun-plugin</artifactId>
-                <version>1.7</version>
-                <dependencies>
-                    <dependency>
-                        <groupId>org.apache.ignite</groupId>
-                        <artifactId>ignite-tools</artifactId>
-                        <version>${project.version}</version>
-                    </dependency>
-                </dependencies>
-                <executions>
-                    <execution>
-                        <id>scaladoc-postprocessing</id>
-                        <goals>
-                            <goal>run</goal>
-                        </goals>
-                        <phase>prepare-package</phase>
-                        <configuration>
-                            <target>
-                                <copy todir="target/site/scaladocs">
-                                    <fileset dir="../../assembly/docfiles">
-                                        <include name="img/**" />
-                                        <include name="*.js" />
-                                    </fileset>
-                                </copy>
-
-                                <taskdef name="doctask" classname="org.apache.ignite.tools.ant.beautifier.GridJavadocAntTask" />
-
-                                <doctask css="dotted" dir="target/site/scaladocs" verify="false">
-                                    <include name="**/*.html" />
-                                </doctask>
-
-                                <property name="footer">
-                                    <![CDATA[
-                                        <table style="padding: 0px 10px 10px 10px; width: 100%" border="0">
-                                        <tr>
-                                            <td>
-                                                <nobr>Ignite&#153; - Scalar DSL, ver. <strong>${project.version}</strong></nobr>
-                                                <br>
-                                                <a target=_blank href="https://incubator.apache.org/projects/ignite.html"><nobr>2015 Copyright &#169; Apache Software Foundation</nobr></a>
-                                            </td>
-                                        </tr>
-                                        </table>
-                                    ]]>
-                                </property>
-
-                                <replace dir="target/site/scaladocs" token="&lt;!--FOOTER--&gt;" value="${footer}">
-                                    <include name="**/*.html" />
-                                </replace>
-
-                                <replace dir="target/site/scaladocs">
-                                    <replacetoken>src="package.html"</replacetoken>
-                                    <replacevalue>src=org/apache/ignite/scalar/scalar$.html</replacevalue>
-                                    <include name="**/index.html" />
-                                </replace>
-
-                                <replace dir="target/site/scaladocs">
-                                    <replacetoken>location.replace("package.html")</replacetoken>
-                                    <replacevalue>location.replace("org/apache/ignite/scalar/scalar$.html")</replacevalue>
-                                    <include name="**/index.js" />
-                                </replace>
-
-                                <replace dir="target/site/scaladocs">
-                                    <replacetoken>docs.scala-lang.org/overviews/scaladoc/usage.html#members</replacetoken>
-                                    <replacevalue>docs.scala-lang.org/overviews/scaladoc/interface.html</replacevalue>
-                                    <fileset dir="target/site/scaladocs" />
-                                </replace>
-
-                                <replaceregexp byline="true" flags="im" encoding="UTF-8">
-                                    <regexp pattern="(&lt;dt&gt;Attributes&lt;/dt&gt;&lt;dd&gt;protected\[)&lt;a.+&gt;(.+)&lt;/a&gt;(\]\s+&lt;/dd&gt;)" />
-                                    <substitution expression="\1\2\3" />
-                                    <fileset dir="target/site/scaladocs" />
-                                </replaceregexp>
-
-                                <zip destfile="target/ignite-scalar-${project.version}-javadoc.jar" basedir="target/site/scaladocs" encoding="UTF-8" />
-                            </target>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
+        <resources>
+            <resource>
+                <directory>../scalar/src/main/scala</directory>
+                <excludes>
+                    <exclude>**/*.scala</exclude>
+                </excludes>
+            </resource>
+        </resources>
+
+        <testResources>
+            <testResource>
+                <directory>../scalar/src/test/scala</directory>
+                <excludes>
+                    <exclude>**/*.scala</exclude>
+                </excludes>
+            </testResource>
+        </testResources>
+
+        <!-- TODO IGNITE-956 FIX scaladocs plugins-->
+            <!--<plugin>-->
+                <!--<groupId>org.apache.maven.plugins</groupId>-->
+                <!--<artifactId>maven-antrun-plugin</artifactId>-->
+                <!--<version>1.7</version>-->
+                <!--<dependencies>-->
+                    <!--<dependency>-->
+                        <!--<groupId>org.apache.ignite</groupId>-->
+                        <!--<artifactId>ignite-tools</artifactId>-->
+                        <!--<version>${project.version}</version>-->
+                    <!--</dependency>-->
+                <!--</dependencies>-->
+                <!--<executions>-->
+                    <!--<execution>-->
+                        <!--<id>scaladoc-postprocessing</id>-->
+                        <!--<goals>-->
+                            <!--<goal>run</goal>-->
+                        <!--</goals>-->
+                        <!--<phase>prepare-package</phase>-->
+                        <!--<configuration>-->
+                            <!--<target>-->
+                                <!--<copy todir="target/site/scaladocs">-->
+                                    <!--<fileset dir="../../assembly/docfiles">-->
+                                        <!--<include name="img/**" />-->
+                                        <!--<include name="*.js" />-->
+                                    <!--</fileset>-->
+                                <!--</copy>-->
+
+                                <!--<taskdef name="doctask" classname="org.apache.ignite.tools.ant.beautifier.GridJavadocAntTask" />-->
+
+                                <!--<doctask css="dotted" dir="target/site/scaladocs" verify="false">-->
+                                    <!--<include name="**/*.html" />-->
+                                <!--</doctask>-->
+
+                                <!--<property name="footer">-->
+                                    <!--<![CDATA[-->
+                                        <!--<table style="padding: 0px 10px 10px 10px; width: 100%" border="0">-->
+                                        <!--<tr>-->
+                                            <!--<td>-->
+                                                <!--<nobr>Ignite&#153; - Scalar DSL, ver. <strong>${project.version}</strong></nobr>-->
+                                                <!--<br>-->
+                                                <!--<a target=_blank href="https://incubator.apache.org/projects/ignite.html"><nobr>2015 Copyright &#169; Apache Software Foundation</nobr></a>-->
+                                            <!--</td>-->
+                                        <!--</tr>-->
+                                        <!--</table>-->
+                                    <!--]]>-->
+                                <!--</property>-->
+
+                                <!--<replace dir="target/site/scaladocs" token="&lt;!&#45;&#45;FOOTER&#45;&#45;&gt;" value="${footer}">-->
+                                    <!--<include name="**/*.html" />-->
+                                <!--</replace>-->
+
+                                <!--<replace dir="target/site/scaladocs">-->
+                                    <!--<replacetoken>src="package.html"</replacetoken>-->
+                                    <!--<replacevalue>src=org/apache/ignite/scalar/scalar$.html</replacevalue>-->
+                                    <!--<include name="**/index.html" />-->
+                                <!--</replace>-->
+
+                                <!--<replace dir="target/site/scaladocs">-->
+                                    <!--<replacetoken>location.replace("package.html")</replacetoken>-->
+                                    <!--<replacevalue>location.replace("org/apache/ignite/scalar/scalar$.html")</replacevalue>-->
+                                    <!--<include name="**/index.js" />-->
+                                <!--</replace>-->
+
+                                <!--<replace dir="target/site/scaladocs">-->
+                                    <!--<replacetoken>docs.scala-lang.org/overviews/scaladoc/usage.html#members</replacetoken>-->
+                                    <!--<replacevalue>docs.scala-lang.org/overviews/scaladoc/interface.html</replacevalue>-->
+                                    <!--<fileset dir="target/site/scaladocs" />-->
+                                <!--</replace>-->
+
+                                <!--<replaceregexp byline="true" flags="im" encoding="UTF-8">-->
+                                    <!--<regexp pattern="(&lt;dt&gt;Attributes&lt;/dt&gt;&lt;dd&gt;protected\[)&lt;a.+&gt;(.+)&lt;/a&gt;(\]\s+&lt;/dd&gt;)" />-->
+                                    <!--<substitution expression="\1\2\3" />-->
+                                    <!--<fileset dir="target/site/scaladocs" />-->
+                                <!--</replaceregexp>-->
+
+                                <!--<zip destfile="target/ignite-scalar-${project.version}-javadoc.jar" basedir="target/site/scaladocs" encoding="UTF-8" />-->
+                            <!--</target>-->
+                        <!--</configuration>-->
+                    <!--</execution>-->
+                <!--</executions>-->
+            <!--</plugin>-->
+        <!--</plugins>-->
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bae1804e/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 75d265d..2cf0af5 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -90,6 +90,24 @@
     </dependencies>
 
     <build>
+        <resources>
+            <resource>
+                <directory>../spark/src/main/scala</directory>
+                <excludes>
+                    <exclude>**/*.scala</exclude>
+                </excludes>
+            </resource>
+        </resources>
+
+        <testResources>
+            <testResource>
+                <directory>../spark/src/test/scala</directory>
+                <excludes>
+                    <exclude>**/*.scala</exclude>
+                </excludes>
+            </testResource>
+        </testResources>
+
         <plugins>
             <plugin>
                 <groupId>net.alchim31.maven</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bae1804e/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 3df3287..f0df657 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -113,44 +113,62 @@
     </dependencies>
 
     <build>
-        <plugins>
-            <plugin>
-                <groupId>net.alchim31.maven</groupId>
-                <artifactId>scala-maven-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>scaladoc</id>
-                        <phase>prepare-package</phase>
-                        <goals>
-                            <goal>doc</goal>
-                        </goals>
-                        <configuration>
-                            <doctitle>Ignite Console Visor</doctitle>
-                            <windowtitle>Ignite Console Visor</windowtitle>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-
-            <plugin>
-                 <groupId>org.apache.maven.plugins</groupId>
-                 <artifactId>maven-antrun-plugin</artifactId>
-                 <version>1.7</version>
-                 <executions>
-                     <execution>
-                         <id>javadoc-zip</id>
-                         <goals>
-                             <goal>run</goal>
-                         </goals>
-                         <phase>prepare-package</phase>
-                         <configuration>
-                             <target>
-                                 <zip destfile="target/ignite-visor-console-2.10-${project.version}-javadoc.jar" basedir="target/site/scaladocs" encoding="UTF-8" />
-                             </target>
-                         </configuration>
-                     </execution>
-                 </executions>
-             </plugin>
-        </plugins>
+        <resources>
+            <resource>
+                <directory>../visor-console/src/main/scala</directory>
+                <excludes>
+                    <exclude>**/*.scala</exclude>
+                </excludes>
+            </resource>
+        </resources>
+
+        <testResources>
+            <testResource>
+                <directory>../visor-console/src/test/scala</directory>
+                <excludes>
+                    <exclude>**/*.scala</exclude>
+                </excludes>
+            </testResource>
+        </testResources>
+
+        <!-- TODO IGNITE-956 FIX scaladocs plugins-->
+            <!--<plugin>-->
+                <!--<groupId>net.alchim31.maven</groupId>-->
+                <!--<artifactId>scala-maven-plugin</artifactId>-->
+                <!--<executions>-->
+                    <!--<execution>-->
+                        <!--<id>scaladoc</id>-->
+                        <!--<phase>prepare-package</phase>-->
+                        <!--<goals>-->
+                            <!--<goal>doc</goal>-->
+                        <!--</goals>-->
+                        <!--<configuration>-->
+                            <!--<doctitle>Ignite Console Visor</doctitle>-->
+                            <!--<windowtitle>Ignite Console Visor</windowtitle>-->
+                        <!--</configuration>-->
+                    <!--</execution>-->
+                <!--</executions>-->
+            <!--</plugin>-->
+
+            <!--<plugin>-->
+                 <!--<groupId>org.apache.maven.plugins</groupId>-->
+                 <!--<artifactId>maven-antrun-plugin</artifactId>-->
+                 <!--<version>1.7</version>-->
+                 <!--<executions>-->
+                     <!--<execution>-->
+                         <!--<id>javadoc-zip</id>-->
+                         <!--<goals>-->
+                             <!--<goal>run</goal>-->
+                         <!--</goals>-->
+                         <!--<phase>prepare-package</phase>-->
+                         <!--<configuration>-->
+                             <!--<target>-->
+                                 <!--<zip destfile="target/ignite-visor-console-${project.version}-javadoc.jar" basedir="target/site/scaladocs" encoding="UTF-8" />-->
+                             <!--</target>-->
+                         <!--</configuration>-->
+                     <!--</execution>-->
+                 <!--</executions>-->
+             <!--</plugin>-->
+        <!--/plugins-->
     </build>
 </project>


[14/50] incubator-ignite git commit: # IGNITE-956: Added support for scala-2.10.

Posted by sb...@apache.org.
# IGNITE-956: Added support for scala-2.10.


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

Branch: refs/heads/ignite-745
Commit: ccd682536ce6cde4a7e947a1ffc000b50a2c2f63
Parents: 2aa1ace
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Jun 3 16:06:19 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Jun 3 16:06:19 2015 +0700

----------------------------------------------------------------------
 DEVNOTES.txt                          |   6 +
 examples/pom.xml                      |  34 ++++++
 modules/scalar-2.10/README.txt        |   4 +
 modules/scalar-2.10/pom.xml           | 188 +++++++++++++++++++++++++++++
 modules/spark-2.10/README.txt         |   4 +
 modules/spark-2.10/pom.xml            | 100 +++++++++++++++
 modules/spark/pom.xml                 |   8 +-
 modules/visor-console-2.10/README.txt |   4 +
 modules/visor-console-2.10/pom.xml    | 156 ++++++++++++++++++++++++
 pom.xml                               |  21 +++-
 10 files changed, 518 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index 1562dc4..8cc759c 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -3,9 +3,15 @@ Ignite Fabric Maven Build Instructions
 Without LGPL dependencies (default):
   mvn clean package -DskipTests
 
+Without LGPL dependencies and Scala 2.10:
+  mvn clean package -DskipTests -Dscala-2.10
+
 With LGPL dependencies:
   mvn clean package -DskipTests -Prelease,lgpl
 
+With LGPL dependencies and Scala 2.10:
+  mvn clean package -DskipTests -Prelease,lgpl -Dscala-2.10
+
 Look for incubator-ignite-<version>-bin.zip in ./target/bin directory.
 
 NOTE: JDK version should be 1.7.0-* or >= 1.8.0-u40.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 2b2bfeb..960df9c 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -173,6 +173,40 @@
         </profile>
 
         <profile>
+            <id>scala-2.10</id>
+
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.ignite</groupId>
+                    <artifactId>ignite-scalar_2.10</artifactId>
+                    <version>${project.version}</version>
+                </dependency>
+
+                <dependency>
+                    <groupId>org.scalatest</groupId>
+                    <artifactId>scalatest_2.10</artifactId>
+                    <version>2.2.2</version>
+                    <scope>test</scope>
+                    <exclusions>
+                        <exclusion>
+                            <groupId>org.scala-lang</groupId>
+                            <artifactId>scala-library</artifactId>
+                        </exclusion>
+                    </exclusions>
+                </dependency>
+            </dependencies>
+
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>net.alchim31.maven</groupId>
+                        <artifactId>scala-maven-plugin</artifactId>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+
+        <profile>
             <id>java8-examples</id>
 
             <activation>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/modules/scalar-2.10/README.txt
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/README.txt b/modules/scalar-2.10/README.txt
new file mode 100644
index 0000000..535a193
--- /dev/null
+++ b/modules/scalar-2.10/README.txt
@@ -0,0 +1,4 @@
+Apache Ignite Scalar Module
+---------------------------
+
+Apache Ignite Scalar module to be build with Scala 2.10.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
new file mode 100644
index 0000000..b51c1b3
--- /dev/null
+++ b/modules/scalar-2.10/pom.xml
@@ -0,0 +1,188 @@
+<?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.
+-->
+
+<!--
+    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">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <artifactId>ignite-scalar_2.10</artifactId>
+    <version>1.1.1-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+            <version>2.10.4</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.scalatest</groupId>
+            <artifactId>scalatest_2.10</artifactId>
+            <version>2.2.2</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.scala-lang</groupId>
+                    <artifactId>scala-library</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>net.alchim31.maven</groupId>
+                <artifactId>scala-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>scaladoc</id>
+                        <phase>prepare-package</phase>
+                        <goals>
+                            <goal>doc</goal>
+                        </goals>
+                        <configuration>
+                            <doctitle>Ignite Scalar</doctitle>
+                            <windowtitle>Ignite Scalar</windowtitle>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-antrun-plugin</artifactId>
+                <version>1.7</version>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.apache.ignite</groupId>
+                        <artifactId>ignite-tools</artifactId>
+                        <version>${project.version}</version>
+                    </dependency>
+                </dependencies>
+                <executions>
+                    <execution>
+                        <id>scaladoc-postprocessing</id>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                        <phase>prepare-package</phase>
+                        <configuration>
+                            <target>
+                                <copy todir="target/site/scaladocs">
+                                    <fileset dir="../../assembly/docfiles">
+                                        <include name="img/**" />
+                                        <include name="*.js" />
+                                    </fileset>
+                                </copy>
+
+                                <taskdef name="doctask" classname="org.apache.ignite.tools.ant.beautifier.GridJavadocAntTask" />
+
+                                <doctask css="dotted" dir="target/site/scaladocs" verify="false">
+                                    <include name="**/*.html" />
+                                </doctask>
+
+                                <property name="footer">
+                                    <![CDATA[
+                                        <table style="padding: 0px 10px 10px 10px; width: 100%" border="0">
+                                        <tr>
+                                            <td>
+                                                <nobr>Ignite&#153; - Scalar DSL, ver. <strong>${project.version}</strong></nobr>
+                                                <br>
+                                                <a target=_blank href="https://incubator.apache.org/projects/ignite.html"><nobr>2015 Copyright &#169; Apache Software Foundation</nobr></a>
+                                            </td>
+                                        </tr>
+                                        </table>
+                                    ]]>
+                                </property>
+
+                                <replace dir="target/site/scaladocs" token="&lt;!--FOOTER--&gt;" value="${footer}">
+                                    <include name="**/*.html" />
+                                </replace>
+
+                                <replace dir="target/site/scaladocs">
+                                    <replacetoken>src="package.html"</replacetoken>
+                                    <replacevalue>src=org/apache/ignite/scalar/scalar$.html</replacevalue>
+                                    <include name="**/index.html" />
+                                </replace>
+
+                                <replace dir="target/site/scaladocs">
+                                    <replacetoken>location.replace("package.html")</replacetoken>
+                                    <replacevalue>location.replace("org/apache/ignite/scalar/scalar$.html")</replacevalue>
+                                    <include name="**/index.js" />
+                                </replace>
+
+                                <replace dir="target/site/scaladocs">
+                                    <replacetoken>docs.scala-lang.org/overviews/scaladoc/usage.html#members</replacetoken>
+                                    <replacevalue>docs.scala-lang.org/overviews/scaladoc/interface.html</replacevalue>
+                                    <fileset dir="target/site/scaladocs" />
+                                </replace>
+
+                                <replaceregexp byline="true" flags="im" encoding="UTF-8">
+                                    <regexp pattern="(&lt;dt&gt;Attributes&lt;/dt&gt;&lt;dd&gt;protected\[)&lt;a.+&gt;(.+)&lt;/a&gt;(\]\s+&lt;/dd&gt;)" />
+                                    <substitution expression="\1\2\3" />
+                                    <fileset dir="target/site/scaladocs" />
+                                </replaceregexp>
+
+                                <zip destfile="target/ignite-scalar-${project.version}-javadoc.jar" basedir="target/site/scaladocs" encoding="UTF-8" />
+                            </target>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/modules/spark-2.10/README.txt
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/README.txt b/modules/spark-2.10/README.txt
new file mode 100644
index 0000000..29d3930
--- /dev/null
+++ b/modules/spark-2.10/README.txt
@@ -0,0 +1,4 @@
+Apache Ignite Spark Module
+---------------------------
+
+Apache Ignite Spark module to be build with Scala 2.10.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
new file mode 100644
index 0000000..75d265d
--- /dev/null
+++ b/modules/spark-2.10/pom.xml
@@ -0,0 +1,100 @@
+<?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.
+-->
+
+<!--
+    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">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <artifactId>ignite-spark_2.10</artifactId>
+    <version>1.1.1-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+            <version>2.10.4</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_2.10</artifactId>
+            <version>1.3.1</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_2.10</artifactId>
+            <version>1.3.1</version>
+        </dependency>
+
+        <!-- Test dependencies -->
+
+        <dependency>
+            <groupId>org.scalatest</groupId>
+            <artifactId>scalatest_2.10</artifactId>
+            <version>2.2.2</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.scala-lang</groupId>
+                    <artifactId>scala-library</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>net.alchim31.maven</groupId>
+                <artifactId>scala-maven-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index a4a25f5..c22a52b 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -51,18 +51,18 @@
         <dependency>
             <groupId>org.scala-lang</groupId>
             <artifactId>scala-library</artifactId>
-            <version>2.10.4</version>
+            <version>2.11.2</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.spark</groupId>
-            <artifactId>spark-core_2.10</artifactId>
+            <artifactId>spark-core_2.11</artifactId>
             <version>1.3.1</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.spark</groupId>
-            <artifactId>spark-sql_2.10</artifactId>
+            <artifactId>spark-sql_2.11</artifactId>
             <version>1.3.1</version>
         </dependency>
 
@@ -70,7 +70,7 @@
 
         <dependency>
             <groupId>org.scalatest</groupId>
-            <artifactId>scalatest_2.10</artifactId>
+            <artifactId>scalatest_2.11</artifactId>
             <version>2.2.2</version>
             <scope>test</scope>
             <exclusions>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/modules/visor-console-2.10/README.txt
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/README.txt b/modules/visor-console-2.10/README.txt
new file mode 100644
index 0000000..1a018b9
--- /dev/null
+++ b/modules/visor-console-2.10/README.txt
@@ -0,0 +1,4 @@
+Apache Ignite Visor Console Module
+---------------------------
+
+Apache Ignite Visor Console module to be build with Scala 2.10.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/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
new file mode 100644
index 0000000..3df3287
--- /dev/null
+++ b/modules/visor-console-2.10/pom.xml
@@ -0,0 +1,156 @@
+<?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.
+-->
+
+<!--
+    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">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <artifactId>ignite-visor-console_2.10</artifactId>
+    <version>1.1.1-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-ssh</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-core</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-expression</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+            <version>2.10.4</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>jline</artifactId>
+            <version>2.10.4</version>
+        </dependency>
+        <!-- Third party dependencies -->
+
+        <!-- Test dependencies -->
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.scalatest</groupId>
+            <artifactId>scalatest_2.10</artifactId>
+            <version>2.2.2</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.scala-lang</groupId>
+                    <artifactId>scala-library</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- Test dependencies -->
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>net.alchim31.maven</groupId>
+                <artifactId>scala-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>scaladoc</id>
+                        <phase>prepare-package</phase>
+                        <goals>
+                            <goal>doc</goal>
+                        </goals>
+                        <configuration>
+                            <doctitle>Ignite Console Visor</doctitle>
+                            <windowtitle>Ignite Console Visor</windowtitle>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                 <groupId>org.apache.maven.plugins</groupId>
+                 <artifactId>maven-antrun-plugin</artifactId>
+                 <version>1.7</version>
+                 <executions>
+                     <execution>
+                         <id>javadoc-zip</id>
+                         <goals>
+                             <goal>run</goal>
+                         </goals>
+                         <phase>prepare-package</phase>
+                         <configuration>
+                             <target>
+                                 <zip destfile="target/ignite-visor-console-2.10-${project.version}-javadoc.jar" basedir="target/site/scaladocs" encoding="UTF-8" />
+                             </target>
+                         </configuration>
+                     </execution>
+                 </executions>
+             </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccd68253/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 54c1e1b..b74476f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -51,7 +51,6 @@
         <module>modules/tools</module>
         <module>modules/core</module>
         <module>modules/hadoop</module>
-        <module>modules/spark</module>
         <module>modules/extdata/p2p</module>
         <module>modules/extdata/uri</module>
         <module>modules/clients</module>
@@ -410,18 +409,34 @@
             <id>scala</id>
 
             <activation>
-                <activeByDefault>true</activeByDefault>
-                <jdk>[1.7,)</jdk>
+                <property><name>!scala-2.10</name></property>
             </activation>
 
             <modules>
                 <module>modules/scalar</module>
+                <module>modules/spark</module>
                 <module>modules/visor-console</module>
                 <module>modules/visor-plugins</module>
             </modules>
         </profile>
 
         <profile>
+            <id>scala-2.10</id>
+
+            <activation>
+                <property><name>scala-2.10</name></property>
+            </activation>
+
+            <modules>
+                <module>modules/scalar-2.10</module>
+                <module>modules/spark-2.10</module>
+                <module>modules/visor-console-2.10</module>
+                <module>modules/visor-plugins</module>
+            </modules>
+        </profile>
+
+
+        <profile>
             <id>lgpl</id>
             <modules>
                 <module>modules/hibernate</module>


[34/50] incubator-ignite git commit: GG-10406

Posted by sb...@apache.org.
GG-10406


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

Branch: refs/heads/ignite-745
Commit: f129d08c6855ca6e720ebb3bb1ea76357f54aef6
Parents: 79ae323
Author: avinogradov <av...@gridgain.com>
Authored: Tue Jun 9 15:11:27 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Jun 9 15:11:27 2015 +0300

----------------------------------------------------------------------
 modules/spark-2.10/README.txt | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f129d08c/modules/spark-2.10/README.txt
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/README.txt b/modules/spark-2.10/README.txt
new file mode 100644
index 0000000..29d3930
--- /dev/null
+++ b/modules/spark-2.10/README.txt
@@ -0,0 +1,4 @@
+Apache Ignite Spark Module
+---------------------------
+
+Apache Ignite Spark module to be build with Scala 2.10.


[37/50] incubator-ignite git commit: ignite-998

Posted by sb...@apache.org.
ignite-998


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

Branch: refs/heads/ignite-745
Commit: c47438e8d2ea734218bcd9c14945ce43456269ce
Parents: af120a7
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 10 14:24:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 10 15:23:37 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  38 +------
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 102 ++++++++++++++++++-
 2 files changed, 103 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c47438e8/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 5aceaae..44374db 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -1752,6 +1752,9 @@ class ServerImpl extends TcpDiscoveryImpl {
         @Nullable Collection<TcpDiscoveryAbstractMessage> messages(IgniteUuid lastMsgId) {
             assert lastMsgId != null;
 
+            if (msgs.isEmpty())
+                return Collections.emptyList();
+
             Collection<TcpDiscoveryAbstractMessage> cp = new ArrayList<>(msgs.size());
 
             boolean skip = true;
@@ -1769,30 +1772,6 @@ class ServerImpl extends TcpDiscoveryImpl {
         }
 
         /**
-         * Resets pending messages.
-         *
-         * @param msgs Message.
-         * @param discardId Discarded message ID.
-         */
-        void reset(@Nullable Collection<TcpDiscoveryAbstractMessage> msgs, @Nullable IgniteUuid discardId) {
-            this.msgs.clear();
-
-            if (msgs != null)
-                this.msgs.addAll(msgs);
-
-            this.discardId = discardId;
-        }
-
-        /**
-         * Clears pending messages.
-         */
-        void clear() {
-            msgs.clear();
-
-            discardId = null;
-        }
-
-        /**
          * Discards message with provided ID and all before it.
          *
          * @param id Discarded message ID.
@@ -2921,8 +2900,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             topHist.clear();
                             topHist.putAll(msg.topologyHistory());
 
-                            // Restore pending messages.
-                            pendingMsgs.reset(msg.messages(), msg.discardedMessageId());
+                            pendingMsgs.discard(msg.discardedMessageId());
 
                             // Clear data to minimize message size.
                             msg.messages(null, null);
@@ -3180,10 +3158,6 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if (log.isDebugEnabled())
                     log.debug("Removed node from topology: " + leftNode);
 
-                // Clear pending messages map.
-                if (!ring.hasRemoteNodes())
-                    pendingMsgs.clear();
-
                 long topVer;
 
                 if (locNodeCoord) {
@@ -3347,10 +3321,6 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 assert node != null;
 
-                // Clear pending messages map.
-                if (!ring.hasRemoteNodes())
-                    pendingMsgs.clear();
-
                 long topVer;
 
                 if (locNodeCoord) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c47438e8/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index 7333020..ece898d 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -103,12 +103,17 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     /** */
     private boolean longSockTimeouts;
 
+    /** */
+    private int maxMissedClientHbs = TcpDiscoverySpi.DFLT_MAX_MISSED_CLIENT_HEARTBEATS;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         TcpDiscoverySpi disco = new TestTcpDiscoverySpi();
 
+        disco.setMaxMissedClientHeartbeats(maxMissedClientHbs);
+
         if (gridName.startsWith("server"))
             disco.setIpFinder(IP_FINDER);
         else if (gridName.startsWith("client")) {
@@ -494,6 +499,96 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testClientReconnectTopologyChange1() throws Exception {
+        maxMissedClientHbs = 100;
+
+        clientsPerSrv = 1;
+
+        startServerNodes(2);
+        startClientNodes(1);
+
+        checkNodes(2, 1);
+
+        srvLeftLatch = new CountDownLatch(3);
+        srvFailedLatch = new CountDownLatch(1);
+
+        attachListeners(2, 0);
+
+        Ignite ignite = G.ignite("client-0");
+
+        TestTcpDiscoverySpi spi = ((TestTcpDiscoverySpi)ignite.configuration().getDiscoverySpi());
+
+        spi.pauseAll();
+
+        try {
+            spi.brakeConnection();
+
+            Ignite g = startGrid("server-" + srvIdx.getAndIncrement());
+
+            g.close();
+
+            spi.resumeAll();
+
+            assertFalse(srvFailedLatch.await(2000, TimeUnit.MILLISECONDS));
+
+            assertEquals(1L, srvLeftLatch.getCount());
+
+            checkNodes(2, 1);
+        }
+        finally {
+            spi.resumeAll();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectTopologyChange2() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-998");
+
+        maxMissedClientHbs = 100;
+
+        clientsPerSrv = 1;
+
+        startServerNodes(1);
+        startClientNodes(1);
+
+        checkNodes(1, 1);
+
+        srvLeftLatch = new CountDownLatch(2);
+        srvFailedLatch = new CountDownLatch(1);
+
+        attachListeners(1, 0);
+
+        Ignite ignite = G.ignite("client-0");
+
+        TestTcpDiscoverySpi spi = ((TestTcpDiscoverySpi)ignite.configuration().getDiscoverySpi());
+
+        spi.pauseAll();
+
+        try {
+            spi.brakeConnection();
+
+            Ignite g = startGrid("server-" + srvIdx.getAndIncrement());
+
+            g.close();
+
+            spi.resumeAll();
+
+            assertFalse(srvFailedLatch.await(2000, TimeUnit.MILLISECONDS));
+
+            assertEquals(1L, srvLeftLatch.getCount());
+
+            checkNodes(1, 1);
+        }
+        finally {
+            spi.resumeAll();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testGetMissedMessagesOnReconnect() throws Exception {
         clientsPerSrv = 1;
 
@@ -731,8 +826,6 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * TODO: IGNITE-587.
-     *
      * @throws Exception If failed.
      */
     public void testDataExchangeFromClient() throws Exception {
@@ -740,6 +833,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param masterName Node name
      * @throws Exception If failed.
      */
     private void testDataExchange(String masterName) throws Exception {
@@ -890,7 +984,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @param clientIdx Index.
+     * @param clientIdx Client index.
+     * @param srvIdx Server index.
      * @throws Exception In case of error.
      */
     private void setClientRouter(int clientIdx, int srvIdx) throws Exception {
@@ -948,6 +1043,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     /**
      * @param srvCnt Number of server nodes.
      * @param clientCnt Number of client nodes.
+     * @throws Exception If failed.
      */
     private void attachListeners(int srvCnt, int clientCnt) throws Exception {
         if (srvJoinedLatch != null) {


[13/50] incubator-ignite git commit: IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389

Posted by sb...@apache.org.
IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389


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

Branch: refs/heads/ignite-745
Commit: 2aa1ace0cdbf0fbbbcd5893958bddb7869742ce0
Parents: d0157d4
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Jun 2 19:34:49 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Jun 2 19:34:49 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheProxy.java      |   8 +-
 .../processors/cache/QueryCursorImpl.java       |  23 ++--
 .../processors/cache/query/QueryCursorEx.java   |   8 ++
 .../processors/query/GridQueryIndexing.java     |   2 +-
 .../processors/query/GridQueryProcessor.java    |  13 ++-
 ...niteDynamicCacheWithConfigStartSelfTest.java | 108 +++++++++++++++++++
 .../processors/query/h2/IgniteH2Indexing.java   |  43 +++++---
 .../h2/twostep/GridReduceQueryExecutor.java     |   8 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |  12 ++-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  23 ++++
 modules/spark/pom.xml                           |  18 ++--
 .../org/apache/ignite/spark/IgniteContext.scala |   3 +
 .../org/apache/ignite/spark/IgniteRDD.scala     |  68 ++++++++++--
 .../spark/examples/IgniteProcessExample.scala   |   2 +-
 .../org/apache/ignite/spark/IgniteRddSpec.scala |  38 +++----
 15 files changed, 291 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 176543b..b3914e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -497,10 +497,14 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
                 return (QueryCursor<R>)queryContinuous((ContinuousQuery<K, V>)qry, qry.isLocal());
 
             if (qry instanceof SqlQuery) {
-                SqlQuery p = (SqlQuery)qry;
+                final SqlQuery p = (SqlQuery)qry;
 
                 if (isReplicatedDataNode() || ctx.isLocal() || qry.isLocal())
-                    return (QueryCursor<R>)new QueryCursorImpl<>(ctx.kernalContext().query().<K, V>queryLocal(ctx, p));
+                    return (QueryCursor<R>)new QueryCursorImpl<>(new Iterable<Cache.Entry<K, V>>() {
+                        @Override public Iterator<Cache.Entry<K, V>> iterator() {
+                            return ctx.kernalContext().query().<K, V>queryLocal(ctx, p);
+                        }
+                    });
 
                 return (QueryCursor<R>)ctx.kernalContext().query().queryTwoStep(ctx, p);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
index 7cb9efc..d68c377 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
@@ -27,6 +27,9 @@ import java.util.*;
  * Query cursor implementation.
  */
 public class QueryCursorImpl<T> implements QueryCursorEx<T> {
+    /** Query executor. */
+    private Iterable<T> iterExec;
+
     /** */
     private Iterator<T> iter;
 
@@ -34,18 +37,18 @@ public class QueryCursorImpl<T> implements QueryCursorEx<T> {
     private boolean iterTaken;
 
     /** */
-    private Collection<GridQueryFieldMetadata> fieldsMeta;
+    private List<GridQueryFieldMetadata> fieldsMeta;
 
     /**
-     * @param iter Iterator.
+     * @param iterExec Query executor.
      */
-    public QueryCursorImpl(Iterator<T> iter) {
-        this.iter = iter;
+    public QueryCursorImpl(Iterable<T> iterExec) {
+        this.iterExec = iterExec;
     }
 
     /** {@inheritDoc} */
     @Override public Iterator<T> iterator() {
-        if (iter == null)
+        if (iter == null && iterTaken)
             throw new IgniteException("Cursor is closed.");
 
         if (iterTaken)
@@ -53,12 +56,16 @@ public class QueryCursorImpl<T> implements QueryCursorEx<T> {
 
         iterTaken = true;
 
+        iter = iterExec.iterator();
+
+        assert iter != null;
+
         return iter;
     }
 
     /** {@inheritDoc} */
     @Override public List<T> getAll() {
-        ArrayList<T> all = new ArrayList<>();
+        List<T> all = new ArrayList<>();
 
         try {
             for (T t : this) // Implicitly calls iterator() to do all checks.
@@ -103,14 +110,14 @@ public class QueryCursorImpl<T> implements QueryCursorEx<T> {
     /**
      * @param fieldsMeta SQL Fields query result metadata.
      */
-    public void fieldsMeta(Collection<GridQueryFieldMetadata> fieldsMeta) {
+    public void fieldsMeta(List<GridQueryFieldMetadata> fieldsMeta) {
         this.fieldsMeta = fieldsMeta;
     }
 
     /**
      * @return SQL Fields query result metadata.
      */
-    public Collection<GridQueryFieldMetadata> fieldsMeta() {
+    @Override public List<GridQueryFieldMetadata> fieldsMeta() {
         return fieldsMeta;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryCursorEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryCursorEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryCursorEx.java
index bf1d4ea..5e19b99 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryCursorEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryCursorEx.java
@@ -19,6 +19,9 @@ package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.processors.query.*;
+
+import java.util.*;
 
 /**
  * Extended query cursor interface allowing for "getAll" to output data into destination other than Collection.
@@ -32,6 +35,11 @@ public interface QueryCursorEx<T> extends QueryCursor<T> {
     public void getAll(Consumer<T> c) throws IgniteCheckedException;
 
     /**
+     * @return Query metadata.
+     */
+    public List<GridQueryFieldMetadata> fieldsMeta();
+
+    /**
      * Query value consumer.
      */
     public static interface Consumer<T> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 0bb820d..cc0916a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -60,7 +60,7 @@ public interface GridQueryIndexing {
      * @param qry Query.
      * @return Cursor.
      */
-    public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry);
+    public Iterable<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry);
 
     /**
      * Parses SQL query into two step query and executes it.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index cd4d543..31337ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -532,7 +532,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Cursor.
      */
-    public QueryCursor<List<?>> queryTwoStep(String space, GridCacheTwoStepQuery qry) {
+    public Iterable<List<?>> queryTwoStep(String space, GridCacheTwoStepQuery qry) {
         checkxEnabled();
 
         if (!busyLock.enterBusy())
@@ -670,7 +670,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Iterator.
      */
-    public QueryCursor<List<?>> queryLocalFields(GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
+    public QueryCursor<List<?>> queryLocalFields(final GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
@@ -679,7 +679,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             String sql = qry.getSql();
             Object[] args = qry.getArgs();
 
-            GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
+            final GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
 
             if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
                 ctx.event().record(new CacheQueryExecutedEvent<>(
@@ -697,8 +697,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         null));
             }
 
-            QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
-                new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepPortable()));
+            QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(new Iterable<List<?>>() {
+                @Override public Iterator<List<?>> iterator() {
+                    return new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepPortable());
+                }
+            });
 
             cursor.fieldsMeta(res.metaData());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheWithConfigStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheWithConfigStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheWithConfigStartSelfTest.java
new file mode 100644
index 0000000..704cf26
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheWithConfigStartSelfTest.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class IgniteDynamicCacheWithConfigStartSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE_NAME = "partitioned";
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        if (!client)
+            cfg.setCacheConfiguration(cacheConfiguration(gridName));
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @return Cache configuration.
+     */
+    protected CacheConfiguration cacheConfiguration(String cacheName) {
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>(CACHE_NAME);
+
+        ccfg.setIndexedTypes(String.class, String.class);
+
+        return ccfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartCacheOnClient() throws Exception {
+        int srvCnt = 3;
+
+        startGrids(srvCnt);
+
+        try {
+            client = true;
+
+            int clientCnt = 12;
+
+            IgniteEx[] clients = new IgniteEx[clientCnt];
+
+            for (int i = 0; i < clients.length; i++)
+                clients[i] = startGrid(i + srvCnt);
+
+            final AtomicInteger idx = new AtomicInteger();
+
+            GridTestUtils.runMultiThreaded(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    final int idx0 = idx.getAndIncrement();
+
+                    ignite(idx0).cache(CACHE_NAME).get(1);
+
+                    return null;
+                }
+            }, clients.length, "runner");
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 200da77..6ec329f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -592,7 +592,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @throws SQLException If failed.
      */
     private static List<GridQueryFieldMetadata> meta(ResultSetMetaData rsMeta) throws SQLException {
-        ArrayList<GridQueryFieldMetadata> meta = new ArrayList<>(rsMeta.getColumnCount());
+        List<GridQueryFieldMetadata> meta = new ArrayList<>(rsMeta.getColumnCount());
 
         for (int i = 1; i <= rsMeta.getColumnCount(); i++) {
             String schemaName = rsMeta.getSchemaName(i);
@@ -771,8 +771,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry) {
-        return rdcQryExec.query(cctx, qry);
+    @Override public Iterable<List<?>> queryTwoStep(final GridCacheContext<?,?> cctx, final GridCacheTwoStepQuery qry) {
+        return new Iterable<List<?>>() {
+            @Override public Iterator<List<?>> iterator() {
+                return rdcQryExec.query(cctx, qry);
+            }
+        };
     }
 
     /** {@inheritDoc} */
@@ -802,25 +806,30 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         final QueryCursor<List<?>> res = queryTwoStep(cctx, fqry);
 
-        final Iterator<List<?>> iter0 = res.iterator();
+        final Iterable<Cache.Entry<K, V>> converted = new Iterable<Cache.Entry<K, V>>() {
+            @Override public Iterator<Cache.Entry<K, V>> iterator() {
+                final Iterator<List<?>> iter0 = res.iterator();
 
-        Iterator<Cache.Entry<K,V>> iter = new Iterator<Cache.Entry<K,V>>() {
-            @Override public boolean hasNext() {
-                return iter0.hasNext();
-            }
+                return new Iterator<Cache.Entry<K,V>>() {
+                    @Override public boolean hasNext() {
+                        return iter0.hasNext();
+                    }
 
-            @Override public Cache.Entry<K,V> next() {
-                List<?> l = iter0.next();
+                    @Override public Cache.Entry<K,V> next() {
+                        List<?> l = iter0.next();
 
-                return new CacheEntryImpl<>((K)l.get(0),(V)l.get(1));
-            }
+                        return new CacheEntryImpl<>((K)l.get(0),(V)l.get(1));
+                    }
 
-            @Override public void remove() {
-                throw new UnsupportedOperationException();
+                    @Override public void remove() {
+                        throw new UnsupportedOperationException();
+                    }
+                };
             }
         };
 
-        return new QueryCursorImpl<Cache.Entry<K,V>>(iter) {
+        // No metadata for SQL queries.
+        return new QueryCursorImpl<Cache.Entry<K,V>>(converted) {
             @Override public void close() {
                 res.close();
             }
@@ -844,7 +853,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
 
         GridCacheTwoStepQuery twoStepQry;
-        Collection<GridQueryFieldMetadata> meta;
+        List<GridQueryFieldMetadata> meta;
 
         try {
             twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), qry.isCollocated());
@@ -863,7 +872,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         twoStepQry.pageSize(qry.getPageSize());
 
-        QueryCursorImpl<List<?>> cursor = (QueryCursorImpl<List<?>>)queryTwoStep(cctx, twoStepQry);
+        QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(queryTwoStep(cctx, twoStepQry));
 
         cursor.fieldsMeta(meta);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 50c30a5..cfacfcf 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -269,7 +269,7 @@ public class GridReduceQueryExecutor {
      * @param qry Query.
      * @return Cursor.
      */
-    public QueryCursor<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry) {
+    public Iterator<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry) {
         long qryReqId = reqIdGen.incrementAndGet();
 
         QueryRun r = new QueryRun();
@@ -356,7 +356,7 @@ public class GridReduceQueryExecutor {
 //                dropTable(r.conn, tbl.getName()); TODO
             }
 
-            return new QueryCursorImpl<>(new GridQueryCacheObjectsIterator(new Iter(res), cctx, cctx.keepPortable()));
+            return new GridQueryCacheObjectsIterator(new Iter(res), cctx, cctx.keepPortable());
         }
         catch (IgniteCheckedException | InterruptedException | RuntimeException e) {
             U.closeQuiet(r.conn);
@@ -381,7 +381,7 @@ public class GridReduceQueryExecutor {
      * @return Cursor for plans.
      * @throws IgniteCheckedException if failed.
      */
-    private QueryCursor<List<?>> explainPlan(JdbcConnection c, String space, GridCacheTwoStepQuery qry)
+    private Iterator<List<?>> explainPlan(JdbcConnection c, String space, GridCacheTwoStepQuery qry)
         throws IgniteCheckedException {
         List<List<?>> lists = new ArrayList<>();
 
@@ -403,7 +403,7 @@ public class GridReduceQueryExecutor {
 
         lists.add(F.asList(getPlan(rs)));
 
-        return new QueryCursorImpl<>(lists.iterator());
+        return lists.iterator();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
index 4e9bf31..dd7c879 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
@@ -127,9 +127,17 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
 
         q.addMapQuery("_cnts_", "select count(*) x from \"partitioned\".FactPurchase where ? = ?", 2, 2);
 
-        Object cnt = qryProc.queryTwoStep(cache, q).getAll().iterator().next().get(0);
+        Iterator<List<?>> it = qryProc.queryTwoStep(cache, q).iterator();
 
-        assertEquals(10L, cnt);
+        try {
+            Object cnt = it.next().get(0);
+
+            assertEquals(10L, cnt);
+        }
+        finally {
+            if (it instanceof AutoCloseable)
+                ((AutoCloseable)it).close();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index fa62361..0d45711 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 import org.apache.ignite.internal.processors.cache.query.*;
+import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -987,6 +988,28 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
     /**
      * @throws Exception If failed.
      */
+    public void testFieldsQueryMetadata() throws Exception {
+        IgniteCache<UUID, Person> cache = ignite.cache(null);
+
+        for (int i = 0; i < 100; i++)
+            cache.put(UUID.randomUUID(), new Person("name-" + i, (i + 1) * 100));
+
+        QueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("select name, salary from Person where name like ?")
+            .setArgs("name-"));
+
+        assertTrue(cur instanceof QueryCursorEx);
+
+        QueryCursorEx<List<?>> curEx = (QueryCursorEx<List<?>>)cur;
+
+        List<GridQueryFieldMetadata> meta = curEx.fieldsMeta();
+
+        assertNotNull(meta);
+        assertEquals(2, meta.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     private void checkSqlQueryEvents() throws Exception {
         final CountDownLatch execLatch = new CountDownLatch(cacheMode() == REPLICATED ? 1 : gridCount());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 84055d6..a4a25f5 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.0.7-SNAPSHOT</version>
+    <version>1.1.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>
@@ -58,16 +58,12 @@
             <groupId>org.apache.spark</groupId>
             <artifactId>spark-core_2.10</artifactId>
             <version>1.3.1</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>com.twitter</groupId>
-                    <artifactId>chill_2.11</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.twitter</groupId>
-                    <artifactId>chill-java</artifactId>
-                </exclusion>
-            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_2.10</artifactId>
+            <version>1.3.1</version>
         </dependency>
 
         <!-- Test dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
index 6259665..5cdbad0 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.IgnitionEx
 import org.apache.ignite.{Ignition, Ignite}
 import org.apache.ignite.configuration.{CacheConfiguration, IgniteConfiguration}
 import org.apache.spark.SparkContext
+import org.apache.spark.sql.SQLContext
 
 /**
  * Ignite context.
@@ -42,6 +43,8 @@ class IgniteContext[K, V](
         this(sc, () ⇒ IgnitionEx.loadConfiguration(springUrl).get1())
     }
 
+    val sqlContext = new SQLContext(sparkContext)
+
     def fromCache(cacheName: String): IgniteRDD[K, V] = {
         new IgniteRDD[K, V](this, cacheName, null)
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index f286b58..0b8e845 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -18,14 +18,18 @@ package org.apache.ignite.spark
 
 import javax.cache.Cache
 
-import org.apache.ignite.cache.query.{SqlFieldsQuery, SqlQuery, ScanQuery}
+import org.apache.ignite.cache.query._
 import org.apache.ignite.cluster.ClusterNode
 import org.apache.ignite.configuration.CacheConfiguration
+import org.apache.ignite.internal.processors.cache.query.QueryCursorEx
+import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata
 import org.apache.ignite.lang.IgniteUuid
-import org.apache.ignite.spark.impl.{IgniteAbstractRDD, IgniteSqlRDD, IgnitePartition, IgniteQueryIterator}
+import org.apache.ignite.spark.impl._
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode
 import org.apache.spark.rdd.RDD
-import org.apache.spark.{TaskContext, Partition}
+import org.apache.spark.sql.types._
+import org.apache.spark.sql._
+import org.apache.spark._
 
 import scala.collection.JavaConversions._
 
@@ -98,12 +102,16 @@ class IgniteRDD[K, V] (
         new IgniteSqlRDD[(K, V), Cache.Entry[K, V], K, V](ic, cacheName, cacheCfg, qry, entry ⇒ (entry.getKey, entry.getValue))
     }
 
-    def sql(sql: String, args: Any*): RDD[Seq[Any]] = {
+    def sql(sql: String, args: Any*): DataFrame = {
         val qry = new SqlFieldsQuery(sql)
 
         qry.setArgs(args.map(_.asInstanceOf[Object]):_*)
 
-        new IgniteSqlRDD[Seq[Any], java.util.List[_], K, V](ic, cacheName, cacheCfg, qry, list ⇒ list)
+        val schema = buildSchema(ensureCache().query(qry).asInstanceOf[QueryCursorEx[java.util.List[_]]].fieldsMeta())
+
+        val rowRdd = new IgniteSqlRDD[Row, java.util.List[_], K, V](ic, cacheName, cacheCfg, qry, list ⇒ Row.fromSeq(list))
+
+        ic.sqlContext.createDataFrame(rowRdd, schema)
     }
 
     def saveValues(rdd: RDD[V]) = {
@@ -138,10 +146,6 @@ class IgniteRDD[K, V] (
             // Make sure to deploy the cache
             ensureCache()
 
-            val locNode = ig.cluster().localNode()
-
-            val node: Option[ClusterNode] = ig.cluster().forHost(locNode).nodes().find(!_.eq(locNode))
-
             val streamer = ig.dataStreamer[K, V](cacheName)
 
             try {
@@ -159,7 +163,49 @@ class IgniteRDD[K, V] (
         ensureCache().removeAll()
     }
 
-    private def affinityKeyFunc(value: V, node: ClusterNode): Object = {
-        IgniteUuid.randomUuid()
+    /**
+     * Builds spark schema from query metadata.
+     *
+     * @param fieldsMeta Fields metadata.
+     * @return Spark schema.
+     */
+    private def buildSchema(fieldsMeta: java.util.List[GridQueryFieldMetadata]): StructType = {
+        new StructType(fieldsMeta.map(i ⇒ new StructField(i.fieldName(), dataType(i.fieldTypeName()), nullable = true))
+            .toArray)
+    }
+
+    /**
+     * Gets Spark data type based on type name.
+     *
+     * @param typeName Type name.
+     * @return Spark data type.
+     */
+    private def dataType(typeName: String): DataType = typeName match {
+        case "java.lang.Boolean" ⇒ BooleanType
+        case "java.lang.Byte" ⇒ ByteType
+        case "java.lang.Short" ⇒ ShortType
+        case "java.lang.Integer" ⇒ IntegerType
+        case "java.lang.Long" ⇒ LongType
+        case "java.lang.Float" ⇒ FloatType
+        case "java.lang.Double" ⇒ DoubleType
+        case "java.lang.String" ⇒ StringType
+        case "java.util.Date" ⇒ DateType
+        case "java.sql.Timestamp" ⇒ TimestampType
+        case "[B" ⇒ BinaryType
+
+        case _ ⇒ StructType(new Array[StructField](0)) // TODO Do we need to fill user types?
+    }
+
+    /**
+     * Generates affinity key for given cluster node.
+     *
+     * @param value Value to generate key for.
+     * @param node Node to generate key for.
+     * @return Affinity key.
+     */
+    private def affinityKeyFunc(value: V, node: ClusterNode): IgniteUuid = {
+        val aff = ic.ignite().affinity[IgniteUuid](cacheName)
+
+        Stream.continually(IgniteUuid.randomUuid()).find(node == null || aff.mapKeyToNode(_).eq(node)).get
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
index db8b5a3..ab91c62 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/examples/IgniteProcessExample.scala
@@ -47,6 +47,6 @@ object IgniteProcessExample {
         ignite.fromCache("indexed").objectSql("Person", "age > ? and organizationId = ?", 20, 12).collect()
 
         // SQL fields query
-        val sqlRes: RDD[Seq[Any]] = ignite.fromCache("indexed").sql("select name, age from Person where age > ?", 20)
+        val df = ignite.fromCache("indexed").sql("select name, age from Person where age > ?", 20)
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2aa1ace0/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
index 68273da..26ce693 100644
--- a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
+++ b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
@@ -117,39 +117,29 @@ class IgniteRddSpec extends FunSpec with Matchers with BeforeAndAfterAll with Be
 
                 val cache: IgniteRDD[String, Entity] = ic.fromCache(PARTITIONED_CACHE_NAME)
 
+                import ic.sqlContext.implicits._
+
                 cache.savePairs(sc.parallelize(0 to 1000, 2).map(i ⇒ (String.valueOf(i), new Entity(i, "name" + i, i * 100))))
 
-                val res = cache.sql("select id, name, salary from Entity where name = ? and salary = ?", "name50", 5000).collect()
+                val df = cache.sql("select id, name, salary from Entity where name = ? and salary = ?", "name50", 5000)
+
+                df.printSchema()
+
+                val res = df.collect()
 
                 assert(res.length == 1, "Invalid result length")
-                assert(50 == res(0).head, "Invalid result")
+                assert(50 == res(0)(0), "Invalid result")
                 assert("name50" == res(0)(1), "Invalid result")
                 assert(5000 == res(0)(2), "Invalid result")
 
-                assert(500 == cache.sql("select id from Entity where id > 500").count(), "Invalid count")
-            }
-            finally {
-                sc.stop()
-            }
-        }
-
-        it("should successfully store values RDD") {
-            val sc = new SparkContext("local[*]", "test")
-
-            try {
-                val ic = new IgniteContext[String, Entity](sc,
-                    () ⇒ configuration("client", client = true))
+                val df0 = cache.sql("select id, name, salary from Entity").where('NAME === "name50" and 'SALARY === 5000)
 
-                val cache: IgniteRDD[String, Entity] = ic.fromCache(PARTITIONED_CACHE_NAME)
+                val res0 = df0.collect()
 
-                cache.savePairs(sc.parallelize(0 to 1000, 2).map(i ⇒ (String.valueOf(i), new Entity(i, "name" + i, i * 100))))
-
-                val res = cache.sql("select id, name, salary from Entity where name = ? and salary = ?", "name50", 5000).collect()
-
-                assert(res.length == 1, "Invalid result length")
-                assert(50 == res(0).head, "Invalid result")
-                assert("name50" == res(0)(1), "Invalid result")
-                assert(5000 == res(0)(2), "Invalid result")
+                assert(res0.length == 1, "Invalid result length")
+                assert(50 == res0(0)(0), "Invalid result")
+                assert("name50" == res0(0)(1), "Invalid result")
+                assert(5000 == res0(0)(2), "Invalid result")
 
                 assert(500 == cache.sql("select id from Entity where id > 500").count(), "Invalid count")
             }


[26/50] incubator-ignite git commit: IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389

Posted by sb...@apache.org.
IGNITE-389 - Merge branch ignite-sprint-5 into ignite-389


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

Branch: refs/heads/ignite-745
Commit: 1552a4b2474bc92ed3aa654c2a477cf444d88e0a
Parents: fa97def 0fa2853
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 8 15:27:35 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 8 15:27:35 2015 -0700

----------------------------------------------------------------------
 DEVNOTES.txt                                    |  42 +-
 .../java/org/apache/ignite/IgniteCache.java     |  16 +
 .../apache/ignite/internal/IgniteKernal.java    |  28 +-
 .../ignite/internal/IgniteNodeAttributes.java   |   5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   8 +-
 .../internal/MarshallerContextAdapter.java      |  36 +-
 .../internal/managers/GridManagerAdapter.java   |   9 +
 .../checkpoint/GridCheckpointManager.java       |  52 +-
 .../discovery/GridDiscoveryManager.java         |  28 +-
 .../affinity/GridAffinityProcessor.java         |  23 +-
 .../cache/DynamicCacheDescriptor.java           |  17 +
 .../processors/cache/GridCacheAdapter.java      |  21 +-
 .../processors/cache/GridCacheContext.java      |  13 +
 .../GridCachePartitionExchangeManager.java      |  26 +-
 .../processors/cache/GridCacheProcessor.java    |  37 +-
 .../processors/cache/GridCacheProxyImpl.java    |  14 +-
 .../processors/cache/GridCacheTtlManager.java   |   9 +-
 .../processors/cache/IgniteCacheProxy.java      |  23 +
 .../processors/cache/IgniteInternalCache.java   |  11 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  22 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |  40 +-
 .../GridDhtPartitionsExchangeFuture.java        |  50 +-
 .../transactions/IgniteTxLocalAdapter.java      |  28 +
 .../cache/transactions/IgniteTxManager.java     |   3 -
 .../datastreamer/DataStreamerImpl.java          |  92 ++-
 .../datastructures/DataStructuresProcessor.java | 107 +++-
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../service/GridServiceProcessor.java           |   4 +-
 .../timeout/GridSpiTimeoutObject.java           |  73 +++
 .../timeout/GridTimeoutProcessor.java           | 105 +++-
 .../IgniteTxRollbackCheckedException.java       |   9 +
 .../util/nio/GridCommunicationClient.java       |  30 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |  13 +-
 .../util/nio/GridTcpCommunicationClient.java    | 554 -------------------
 .../util/nio/GridTcpNioCommunicationClient.java |   8 -
 .../ignite/internal/visor/cache/VisorCache.java |   2 +-
 .../VisorCacheConfigurationCollectorJob.java    |   6 +-
 .../internal/visor/cache/VisorCacheMetrics.java |  19 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |  10 +-
 .../cache/VisorCacheStoreConfiguration.java     |   5 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  27 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  10 +
 .../ignite/spi/IgniteSpiTimeoutObject.java      |  44 ++
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |   3 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 443 ++++-----------
 .../tcp/TcpCommunicationSpiMBean.java           |   2 -
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   3 -
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  10 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 156 +-----
 ...acheReadOnlyTransactionalClientSelfTest.java | 327 +++++++++++
 .../cache/GridCacheAbstractFullApiSelfTest.java |  83 +++
 .../GridCacheExAbstractFullApiSelfTest.java     | 103 ----
 .../IgniteCountDownLatchAbstractSelfTest.java   | 102 ++++
 .../GridCacheExColocatedFullApiSelfTest.java    |  33 --
 .../near/GridCacheExNearFullApiSelfTest.java    |  39 --
 .../GridCacheExReplicatedFullApiSelfTest.java   |  33 --
 .../IgniteCacheClientNearCacheExpiryTest.java   | 103 ++++
 .../IgniteCacheExpiryPolicyTestSuite.java       |   2 +
 .../local/GridCacheExLocalFullApiSelfTest.java  |  30 -
 .../DataStreamerMultiThreadedSelfTest.java      |  59 +-
 .../DataStreamerMultinodeCreateCacheTest.java   |  97 ++++
 .../internal/util/nio/GridNioSelfTest.java      |   2 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   1 +
 .../IgniteMessagingWithClientTest.java          |   2 +
 .../GridTcpCommunicationSpiAbstractTest.java    |   4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   2 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   2 -
 ...cpCommunicationSpiMultithreadedSelfTest.java |   2 +-
 .../discovery/AbstractDiscoverySelfTest.java    |  13 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  25 +
 .../testframework/GridSpiTestContext.java       |  10 +
 .../IgniteCacheFullApiSelfTestSuite.java        |   6 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 74 files changed, 1825 insertions(+), 1556 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/DEVNOTES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 3768db5,359de1c..a661965
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@@ -691,23 -634,8 +678,14 @@@ public class TcpCommunicationSpi extend
      /** Socket write timeout. */
      private long sockWriteTimeout = DFLT_SOCK_WRITE_TIMEOUT;
  
+     /** Recovery and idle clients handler. */
+     private CommunicationWorker commWorker;
++    
 +    /** Shared memory accept worker. */
 +    private ShmemAcceptWorker shmemAcceptWorker;
 +
-     /** Idle client worker. */
-     private IdleClientWorker idleClientWorker;
- 
-     /** Flush client worker. */
-     private ClientFlushWorker clientFlushWorker;
- 
-     /** Socket timeout worker. */
-     private SocketTimeoutWorker sockTimeoutWorker;
- 
-     /** Recovery worker. */
-     private RecoveryWorker recoveryWorker;
- 
 +    /** Shared memory workers. */
 +    private final Collection<ShmemWorker> shmemWorkers = new ConcurrentLinkedDeque8<>();
  
      /** Clients. */
      private final ConcurrentMap<UUID, GridCommunicationClient> clients = GridConcurrentFactory.newMap();
@@@ -1354,31 -1239,11 +1321,17 @@@
  
          registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
  
 +        if (shmemSrv != null) {
 +            shmemAcceptWorker = new ShmemAcceptWorker(shmemSrv);
 +
 +            new IgniteThread(shmemAcceptWorker).start();
 +        }
 +
          nioSrvr.start();
  
-         idleClientWorker = new IdleClientWorker();
+         commWorker = new CommunicationWorker();
  
-         idleClientWorker.start();
- 
-         recoveryWorker = new RecoveryWorker();
- 
-         recoveryWorker.start();
- 
-         if (connBufSize > 0) {
-             clientFlushWorker = new ClientFlushWorker();
- 
-             clientFlushWorker.start();
-         }
- 
-         sockTimeoutWorker = new SocketTimeoutWorker();
- 
-         sockTimeoutWorker.start();
+         commWorker.start();
  
          // Ack start.
          if (log.isDebugEnabled())
@@@ -1586,24 -1398,10 +1539,17 @@@
          if (nioSrvr != null)
              nioSrvr.stop();
  
+         U.interrupt(commWorker);
 -
+         U.join(commWorker, log);
+ 
 +        U.cancel(shmemAcceptWorker);
 +        U.join(shmemAcceptWorker, log);
 +
-         U.interrupt(idleClientWorker);
-         U.interrupt(clientFlushWorker);
-         U.interrupt(sockTimeoutWorker);
-         U.interrupt(recoveryWorker);
- 
-         U.join(idleClientWorker, log);
-         U.join(clientFlushWorker, log);
-         U.join(sockTimeoutWorker, log);
-         U.join(recoveryWorker, log);
- 
 +        U.cancel(shmemWorkers);
 +        U.join(shmemWorkers, log);
 +
 +        shmemWorkers.clear();
 +
          // Force closing on stop (safety).
          for (GridCommunicationClient client : clients.values())
              client.forceClose();
@@@ -2400,147 -2095,12 +2340,150 @@@
      }
  
      /**
 +     * This worker takes responsibility to shut the server down when stopping,
 +     * No other thread shall stop passed server.
 +     */
 +    private class ShmemAcceptWorker extends GridWorker {
 +        /** */
 +        private final IpcSharedMemoryServerEndpoint srv;
 +
 +        /**
 +         * @param srv Server.
 +         */
 +        ShmemAcceptWorker(IpcSharedMemoryServerEndpoint srv) {
 +            super(gridName, "shmem-communication-acceptor", TcpCommunicationSpi.this.log);
 +
 +            this.srv = srv;
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override protected void body() throws InterruptedException {
 +            try {
 +                while (!Thread.interrupted()) {
 +                    ShmemWorker e = new ShmemWorker(srv.accept());
 +
 +                    shmemWorkers.add(e);
 +
 +                    new IgniteThread(e).start();
 +                }
 +            }
 +            catch (IgniteCheckedException e) {
 +                if (!isCancelled())
 +                    U.error(log, "Shmem server failed.", e);
 +            }
 +            finally {
 +                srv.close();
 +            }
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public void cancel() {
 +            super.cancel();
 +
 +            srv.close();
 +        }
 +    }
 +
 +    /**
 +     *
 +     */
 +    private class ShmemWorker extends GridWorker {
 +        /** */
 +        private final IpcEndpoint endpoint;
 +
 +        /**
 +         * @param endpoint Endpoint.
 +         */
 +        private ShmemWorker(IpcEndpoint endpoint) {
 +            super(gridName, "shmem-worker", TcpCommunicationSpi.this.log);
 +
 +            this.endpoint = endpoint;
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override protected void body() throws InterruptedException {
 +            try {
 +                MessageFactory msgFactory = new MessageFactory() {
 +                    private MessageFactory impl;
 +
 +                    @Nullable @Override public Message create(byte type) {
 +                        if (impl == null)
 +                            impl = getSpiContext().messageFactory();
 +
 +                        assert impl != null;
 +
 +                        return impl.create(type);
 +                    }
 +                };
 +
 +                MessageFormatter msgFormatter = new MessageFormatter() {
 +                    private MessageFormatter impl;
 +
 +                    @Override public MessageWriter writer() {
 +                        if (impl == null)
 +                            impl = getSpiContext().messageFormatter();
 +
 +                        assert impl != null;
 +
 +                        return impl.writer();
 +                    }
 +
 +                    @Override public MessageReader reader(MessageFactory factory) {
 +                        if (impl == null)
 +                            impl = getSpiContext().messageFormatter();
 +
 +                        assert impl != null;
 +
 +                        return impl.reader(factory);
 +                    }
 +                };
 +
 +                IpcToNioAdapter<Message> adapter = new IpcToNioAdapter<>(
 +                    metricsLsnr,
 +                    log,
 +                    endpoint,
 +                    srvLsnr,
 +                    msgFormatter,
 +                    new GridNioCodecFilter(new GridDirectParser(msgFactory, msgFormatter), log, true),
 +                    new GridConnectionBytesVerifyFilter(log)
 +                );
 +
 +                adapter.serve();
 +            }
 +            finally {
 +                shmemWorkers.remove(this);
 +
 +                endpoint.close();
 +            }
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public void cancel() {
 +            super.cancel();
 +
 +            endpoint.close();
 +        }
 +
 +        /** @{@inheritDoc} */
 +        @Override protected void cleanup() {
 +            super.cleanup();
 +
 +            endpoint.close();
 +        }
 +
 +        /** @{@inheritDoc} */
 +        @Override public String toString() {
 +            return S.toString(ShmemWorker.class, this);
 +        }
 +    }
 +
 +    /**
       *
       */
-     private class IdleClientWorker extends IgniteSpiThread {
+     private class CommunicationWorker extends IgniteSpiThread {
+         /** */
+         private final BlockingQueue<GridNioRecoveryDescriptor> q = new LinkedBlockingQueue<>();
+ 
          /**
           *
           */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1552a4b2/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------


[44/50] incubator-ignite git commit: #IGNITE-389 - Minor

Posted by sb...@apache.org.
#IGNITE-389 - Minor


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

Branch: refs/heads/ignite-745
Commit: 89a4f7c569e3f50d1623d461756e8a0bc2c1dd13
Parents: f149c82
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Wed Jun 10 15:59:15 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Wed Jun 10 15:59:15 2015 -0700

----------------------------------------------------------------------
 .../internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/89a4f7c5/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
index 5b3274d..d4ae147 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
@@ -163,7 +163,7 @@ public class IpcSharedMemoryNativeLoader {
                 }
             }
             catch (IgniteCheckedException ignore) {
-
+                // No-op.
             }
 
             // Failed to find the library.