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

[01/10] ignite git commit: ignite-2146 Call service node filter outside of system cache transaction.

Repository: ignite
Updated Branches:
  refs/heads/ignite-843-rc2 e49779567 -> 46709b002


ignite-2146 Call service node filter outside of system cache transaction.


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

Branch: refs/heads/ignite-843-rc2
Commit: ab8ba9746f7f8f3be88eba73cc9abfb84cd86ecc
Parents: df08d3d
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 16 08:39:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 16 08:39:20 2015 +0300

----------------------------------------------------------------------
 .../service/GridServiceProcessor.java           |  30 +++-
 .../ServicePredicateAccessCacheTest.java        | 155 +++++++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 3 files changed, 180 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ab8ba974/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 ed54f00..6b05edd 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
@@ -704,13 +704,33 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         Object affKey = cfg.getAffinityKey();
 
         while (true) {
+            GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer);
+
+             Collection<ClusterNode> nodes;
+
+             // Call node filter outside of transaction.
+            if (affKey == null) {
+                nodes = ctx.discovery().nodes(topVer);
+
+                if (assigns.nodeFilter() != null) {
+                    Collection<ClusterNode> nodes0 = new ArrayList<>();
+
+                    for (ClusterNode node : nodes) {
+                        if (assigns.nodeFilter().apply(node))
+                            nodes0.add(node);
+                    }
+
+                    nodes = nodes0;
+                }
+            }
+            else
+                nodes = null;
+
             try (IgniteInternalTx tx = cache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                 GridServiceAssignmentsKey key = new GridServiceAssignmentsKey(cfg.getName());
 
                 GridServiceAssignments oldAssigns = (GridServiceAssignments)cache.get(key);
 
-                GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer);
-
                 Map<UUID, Integer> cnts = new HashMap<>();
 
                 if (affKey != null) {
@@ -723,10 +743,6 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                     }
                 }
                 else {
-                    Collection<ClusterNode> nodes = assigns.nodeFilter() == null ?
-                        ctx.discovery().nodes(topVer) :
-                        F.view(ctx.discovery().nodes(topVer), assigns.nodeFilter());
-
                     if (!nodes.isEmpty()) {
                         int size = nodes.size();
 
@@ -805,7 +821,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
                 assigns.assigns(cnts);
 
-                cache.getAndPut(key, assigns);
+                cache.put(key, assigns);
 
                 tx.commit();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab8ba974/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServicePredicateAccessCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServicePredicateAccessCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServicePredicateAccessCacheTest.java
new file mode 100644
index 0000000..c91d9f1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServicePredicateAccessCacheTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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.service;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class ServicePredicateAccessCacheTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static CountDownLatch latch;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 60_000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPredicateAccessCache() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+
+        ignite0.getOrCreateCache(new CacheConfiguration<String, Object>()
+            .setName("testCache")
+            .setAtomicityMode(ATOMIC)
+            .setCacheMode(REPLICATED)
+            .setWriteSynchronizationMode(FULL_SYNC)
+            .setAtomicWriteOrderMode(PRIMARY));
+
+        latch = new CountDownLatch(1);
+
+        final ClusterGroup grp = ignite0.cluster().forPredicate(new IgnitePredicate<ClusterNode>() {
+            @Override public boolean apply(ClusterNode node) {
+                System.out.println("Predicated started [thread=" + Thread.currentThread().getName() + ']');
+
+                latch.countDown();
+
+                try {
+                    Thread.sleep(3000);
+                }
+                catch (InterruptedException ignore) {
+                    // No-op.
+                }
+
+                System.out.println("Call contains key [thread=" + Thread.currentThread().getName() + ']');
+
+                boolean ret = ignite0.cache("testCache").containsKey(node.id().toString());
+
+                System.out.println("After contains key [ret=" + ret +
+                    ", thread=" + Thread.currentThread().getName() + ']');
+
+                return ret;
+            }
+        });
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                info("Start deploy service.");
+
+                ignite0.services(grp).deployNodeSingleton("testService", new TestService());
+
+                info("Service deployed.");
+
+                return null;
+            }
+        }, "deploy-thread");
+
+        latch.await();
+
+        startGrid(1);
+
+        fut.get();
+    }
+
+    /**
+     *
+     */
+    public static class TestService implements Service {
+        /** {@inheritDoc} */
+        public void execute(ServiceContext ctx) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        public void init(ServiceContext ctx) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        public void cancel(ServiceContext ctx) {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab8ba974/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index a41859e..deb49b7 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@ -52,6 +52,7 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessorProxySe
 import org.apache.ignite.internal.processors.service.GridServiceProcessorSingleNodeSelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceProcessorStopSelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceReassignmentSelfTest;
+import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest;
 import org.apache.ignite.internal.util.GridStartupWithSpecifiedWorkDirectorySelfTest;
 import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest;
 import org.apache.ignite.spi.communication.GridCacheMessageSelfTest;
@@ -117,6 +118,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridServiceReassignmentSelfTest.class);
         suite.addTestSuite(GridServiceClientNodeTest.class);
         suite.addTestSuite(GridServiceProcessorStopSelfTest.class);
+        suite.addTestSuite(ServicePredicateAccessCacheTest.class);
 
         return suite;
     }


[09/10] ignite git commit: Merge branches 'ignite-1.5' and 'ignite-843-rc2' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-843-rc2

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


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

Branch: refs/heads/ignite-843-rc2
Commit: 0eb69c614d6b5003f2145f8235fec215e7c612c8
Parents: e497795 490f552
Author: Andrey <an...@gridgain.com>
Authored: Wed Dec 16 17:18:44 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Wed Dec 16 17:18:44 2015 +0700

----------------------------------------------------------------------
 .../transactions/IgniteTxLocalAdapter.java      |  23 ++-
 .../service/GridServiceProcessor.java           |  30 ++-
 .../CacheSerializableTransactionsTest.java      |   8 +
 .../GridServiceProcessorProxySelfTest.java      |   3 +-
 .../ServicePredicateAccessCacheTest.java        | 155 ++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../common/include/ignite/common/concurrent.h   |   2 +-
 .../cpp/common/include/ignite/common/utils.h    |   1 -
 .../cpp/common/os/win/src/concurrent_os.cpp     |  26 ++-
 modules/platforms/cpp/common/src/java.cpp       |  20 +-
 .../platforms/cpp/core-test/src/cache_test.cpp  |   4 +-
 modules/platforms/cpp/core/impl/doxygen.h       |  53 +++++
 .../core/include/ignite/binary/binary_type.h    |   4 +-
 .../cpp/core/include/ignite/cache/cache.h       |  12 +-
 .../include/ignite/cache/query/query_scan.h     |  28 +--
 .../include/ignite/cache/query/query_text.h     |  38 ++--
 .../cpp/core/include/ignite/ignition.h          |  10 +-
 .../core/include/ignite/impl/cache/cache_impl.h |   6 +-
 .../cpp/core/include/ignite/impl/operations.h   |   8 +-
 modules/platforms/cpp/cpp.dxg                   |   4 +-
 .../Examples/ExamplesTest.cs                    |   4 +-
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../dotnet/Apache.Ignite.Core/Index.cs          |  31 +++
 modules/platforms/dotnet/Apache.Ignite.sln      |   1 +
 .../Apache.Ignite.Examples.csproj               |   1 -
 .../Datagrid/CrossPlatformExample.cs            | 205 -------------------
 26 files changed, 388 insertions(+), 292 deletions(-)
----------------------------------------------------------------------



[06/10] ignite git commit: IGNITE-2159: CPP: Fixed compilation for interlocked operations on Windows.

Posted by an...@apache.org.
IGNITE-2159: CPP: Fixed compilation for interlocked operations on Windows.


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

Branch: refs/heads/ignite-843-rc2
Commit: 65b176665e97e753f263231700b6907e51e94b00
Parents: 1acbce1
Author: isapego <is...@gridgain.com>
Authored: Wed Dec 16 11:32:11 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 16 11:32:11 2015 +0300

----------------------------------------------------------------------
 .../cpp/common/os/win/src/concurrent_os.cpp     | 26 +++++++++++++++++++-
 .../platforms/cpp/core-test/src/cache_test.cpp  |  4 +--
 2 files changed, 27 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/65b17666/modules/platforms/cpp/common/os/win/src/concurrent_os.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/os/win/src/concurrent_os.cpp b/modules/platforms/cpp/common/os/win/src/concurrent_os.cpp
index a21f7ec..676d8b6 100644
--- a/modules/platforms/cpp/common/os/win/src/concurrent_os.cpp
+++ b/modules/platforms/cpp/common/os/win/src/concurrent_os.cpp
@@ -17,6 +17,8 @@
 
 #include "ignite/common/concurrent_os.h"
 
+#pragma intrinsic(_InterlockedCompareExchange64)
+
 namespace ignite
 {
     namespace common
@@ -103,17 +105,39 @@ namespace ignite
 
             int64_t Atomics::CompareAndSet64Val(int64_t* ptr, int64_t expVal, int64_t newVal)
             {
-                return InterlockedCompareExchange64(reinterpret_cast<LONG64*>(ptr), newVal, expVal);
+                return _InterlockedCompareExchange64(reinterpret_cast<LONG64*>(ptr), newVal, expVal);
             }
 
             int64_t Atomics::IncrementAndGet64(int64_t* ptr)
             {
+#ifdef _WIN64
                 return InterlockedIncrement64(reinterpret_cast<LONG64*>(ptr));
+#else 
+                while (true)
+                {
+                    int64_t expVal = *ptr;
+                    int64_t newVal = expVal + 1;
+
+                    if (CompareAndSet64(ptr, expVal, newVal))
+                        return newVal;
+                }
+#endif
             }
 
             int64_t Atomics::DecrementAndGet64(int64_t* ptr)
             {
+#ifdef _WIN64
                 return InterlockedDecrement64(reinterpret_cast<LONG64*>(ptr));
+#else 
+                while (true)
+                {
+                    int64_t expVal = *ptr;
+                    int64_t newVal = expVal - 1;
+
+                    if (CompareAndSet64(ptr, expVal, newVal))
+                        return newVal;
+                }
+#endif
             }
             
             bool ThreadLocal::OnProcessAttach()

http://git-wip-us.apache.org/repos/asf/ignite/blob/65b17666/modules/platforms/cpp/core-test/src/cache_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_test.cpp b/modules/platforms/cpp/core-test/src/cache_test.cpp
index c56054f..32c5bd6 100644
--- a/modules/platforms/cpp/core-test/src/cache_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_test.cpp
@@ -234,12 +234,12 @@ BOOST_AUTO_TEST_CASE(TestGetAll)
     
     std::set<int> keySet (keys, keys + 5);
 
-    for (int i = 0; i < keySet.size(); i++)
+    for (int i = 0; i < static_cast<int>(keySet.size()); i++)
         cache.Put(i + 1, i + 1);
 
     std::map<int, int> map = cache.GetAll(keySet);
 
-    for (int i = 0; i < keySet.size(); i++)
+    for (int i = 0; i < static_cast<int>(keySet.size()); i++)
         BOOST_REQUIRE(i + 1 == map[i + 1]);
 }
 


[07/10] ignite git commit: IGNITE-2164: CPP: Added namespace description. Docs fixes.

Posted by an...@apache.org.
IGNITE-2164: CPP: Added namespace description. Docs fixes.


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

Branch: refs/heads/ignite-843-rc2
Commit: 77c6d08a41d349aea850b6c6ca4190b8ec77e22a
Parents: 65b1766
Author: isapego <is...@gridgain.com>
Authored: Wed Dec 16 11:36:40 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 16 11:36:40 2015 +0300

----------------------------------------------------------------------
 .../common/include/ignite/common/concurrent.h   |  2 +-
 .../cpp/common/include/ignite/common/utils.h    |  1 -
 modules/platforms/cpp/common/src/java.cpp       | 20 ++++----
 modules/platforms/cpp/core/impl/doxygen.h       | 53 ++++++++++++++++++++
 .../core/include/ignite/binary/binary_type.h    |  4 +-
 .../cpp/core/include/ignite/cache/cache.h       | 12 ++---
 .../include/ignite/cache/query/query_scan.h     | 28 +++++------
 .../include/ignite/cache/query/query_text.h     | 38 +++++++-------
 .../cpp/core/include/ignite/ignition.h          | 10 +---
 .../core/include/ignite/impl/cache/cache_impl.h |  6 +--
 .../cpp/core/include/ignite/impl/operations.h   |  8 +--
 modules/platforms/cpp/cpp.dxg                   |  4 +-
 12 files changed, 115 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/common/include/ignite/common/concurrent.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/concurrent.h b/modules/platforms/cpp/common/include/ignite/common/concurrent.h
index fbba808..a4cc3f7 100644
--- a/modules/platforms/cpp/common/include/ignite/common/concurrent.h
+++ b/modules/platforms/cpp/common/include/ignite/common/concurrent.h
@@ -214,7 +214,7 @@ namespace ignite
                     return impl ? static_cast<T*>(impl->Pointer()) : NULL;
                 }
 
-                /*
+                /**
                  * Check whether underlying raw pointer is valid.
                  *
                  * @return True if valid.

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/common/include/ignite/common/utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/utils.h b/modules/platforms/cpp/common/include/ignite/common/utils.h
index 4b590d9..44db6e8 100644
--- a/modules/platforms/cpp/common/include/ignite/common/utils.h
+++ b/modules/platforms/cpp/common/include/ignite/common/utils.h
@@ -1,4 +1,3 @@
-#pragma once
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/common/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/java.cpp b/modules/platforms/cpp/common/src/java.cpp
index 7bc18f1..64f5d9c 100644
--- a/modules/platforms/cpp/common/src/java.cpp
+++ b/modules/platforms/cpp/common/src/java.cpp
@@ -78,7 +78,7 @@ namespace ignite
                 }
             };
 
-            /*
+            /**
              * Heloper function to copy characters.
              *
              * @param src Source.
@@ -148,7 +148,7 @@ namespace ignite
                     delete[] errMsg;
             }
 
-            /*
+            /**
              * Guard to ensure global reference cleanup.
              */
             class JniGlobalRefGuard
@@ -397,7 +397,7 @@ namespace ignite
 
             /* HELPER METHODS. */
 
-            /*
+            /**
              * Throw exception to Java in case of missing callback pointer. It means that callback is not implemented in
              * native platform and Java -> platform operation cannot proceede further. As JniContext is not available at
              * this point, we have to obtain exception details from scratch. This is not critical from performance
@@ -728,7 +728,7 @@ namespace ignite
                 return members;
             }
 
-            /*
+            /**
              * Create JVM.
              */
             void CreateJvm(char** opts, int optsLen, JavaVM** jvm, JNIEnv** env) {
@@ -2084,9 +2084,9 @@ namespace ignite
                 jvm->GetJvm()->DestroyJavaVM();
             }
 
-            /*
-            * Attach thread to JVM.
-            */
+            /**
+             * Attach thread to JVM.
+             */
             JNIEnv* JniContext::Attach() {
                 JNIEnv* env;
 
@@ -2164,9 +2164,9 @@ namespace ignite
                 }
             }
 
-            /*
-            * Convert local reference to global.
-            */
+            /**
+             * Convert local reference to global.
+             */
             jobject JniContext::LocalToGlobal(JNIEnv* env, jobject localRef) {
                 if (localRef) {
                     jobject globalRef = env->NewGlobalRef(localRef);

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/core/impl/doxygen.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/impl/doxygen.h b/modules/platforms/cpp/core/impl/doxygen.h
new file mode 100644
index 0000000..ed237f6
--- /dev/null
+++ b/modules/platforms/cpp/core/impl/doxygen.h
@@ -0,0 +1,53 @@
+/*
+ * 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.
+ */
+
+/**
+ * \mainpage Apache Ignite C++
+ *
+ * Apache Ignite In-Memory Data Fabric is a high-performance, integrated and distributed in-memory platform for
+ * computing and transacting on large-scale data sets in real-time, orders of magnitude faster than possible with
+ * traditional disk-based or flash-based technologies.
+ */
+ 
+ /**
+  * Apache %Ignite API.
+  */
+ namespace ignite
+ {
+	 /**
+	  * %Ignite Binary Objects API.
+	  */
+	 namespace binary
+	 {
+		 // Empty.
+	 }
+	 
+	 /**
+	  * %Ignite %Cache API.
+	  */
+	 namespace cache
+	 {
+		 /**
+		  * Contains APIs for creating and executing cache queries.
+		  */
+		 namespace query
+		 {
+			 // Empty.
+		 }
+	 }
+ }
+ 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/core/include/ignite/binary/binary_type.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/binary/binary_type.h b/modules/platforms/cpp/core/include/ignite/binary/binary_type.h
index 576a4d6..353b3bb 100644
--- a/modules/platforms/cpp/core/include/ignite/binary/binary_type.h
+++ b/modules/platforms/cpp/core/include/ignite/binary/binary_type.h
@@ -226,8 +226,8 @@ namespace ignite
             }
         };
 
-        /*
-         * Templated binary type for pointers.
+        /**
+         * Templated binary type specification for pointers.
          */
         template <typename T>
         struct IGNITE_IMPORT_EXPORT BinaryType<T*>

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/core/include/ignite/cache/cache.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/cache.h b/modules/platforms/cpp/core/include/ignite/cache/cache.h
index 7581d86..ad23d62 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/cache.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/cache.h
@@ -1083,7 +1083,7 @@ namespace ignite
                 return query::QueryCursor<K, V>(cursorImpl);
             }
 
-            /*
+            /**
              * Perform text query.
              *
              * @param qry Query.
@@ -1100,7 +1100,7 @@ namespace ignite
                 return res;
             }
 
-            /*
+            /**
              * Perform text query.
              *
              * @param qry Query.
@@ -1114,7 +1114,7 @@ namespace ignite
                 return query::QueryCursor<K, V>(cursorImpl);
             }
 
-            /*
+            /**
              * Perform scan query.
              *
              * @param qry Query.
@@ -1131,7 +1131,7 @@ namespace ignite
                 return res;
             }
 
-            /*
+            /**
              * Perform scan query.
              *
              * @param qry Query.
@@ -1145,7 +1145,7 @@ namespace ignite
                 return query::QueryCursor<K, V>(cursorImpl);
             }
 
-            /*
+            /**
              * Perform sql fields query.
              *
              * @param qry Query.
@@ -1162,7 +1162,7 @@ namespace ignite
                 return res;
             }
 
-            /*
+            /**
              * Perform sql fields query.
              *
              * @param qry Query.

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/core/include/ignite/cache/query/query_scan.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_scan.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_scan.h
index c8389b2..fa47269 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_scan.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_scan.h
@@ -29,13 +29,13 @@ namespace ignite
     {
         namespace query
         {         
-            /*
-             * Scab query.
+            /**
+             * Scan query.
              */
             class ScanQuery
             {
             public:
-                /* 
+                /**
                  * Constructor.
                  */
                 ScanQuery() : part(-1), pageSize(1024), loc(false)
@@ -43,7 +43,7 @@ namespace ignite
                     // No-op.
                 }
                 
-                /*
+                /**
                  * Constructor.
                  *
                  * @param part Partition.
@@ -53,7 +53,7 @@ namespace ignite
                     // No-op.
                 }
                 
-                /*
+                /**
                  * Get partition to scan.
                  *
                  * @return Partition to scan.
@@ -63,7 +63,7 @@ namespace ignite
                     return part;
                 }
 
-                /*
+                /**
                  * Set partition to scan.
                  *
                  * @param part Partition to scan.
@@ -73,7 +73,7 @@ namespace ignite
                     this->part = part;
                 }
 
-                /*
+                /**
                  * Get page size.
                  *
                  * @return Page size.
@@ -83,7 +83,7 @@ namespace ignite
                     return pageSize;
                 }
 
-                /*
+                /**
                  * Set page size.
                  *
                  * @param pageSize Page size.
@@ -93,7 +93,7 @@ namespace ignite
                     this->pageSize = pageSize;
                 }
 
-                /*
+                /**
                  * Get local flag.
                  *
                  * @return Local flag.
@@ -103,7 +103,7 @@ namespace ignite
                     return loc;
                 }
 
-                /*
+                /**
                  * Set local flag.
                  *
                  * @param loc Local flag.
@@ -113,7 +113,7 @@ namespace ignite
                     this->loc = loc;
                 }
                 
-                /*
+                /**
                  * Write query info to the stream.
                  *
                  * @param writer Writer.
@@ -135,13 +135,13 @@ namespace ignite
                 }
 
             private:
-                /* Partition. */
+                /** Partition. */
                 int32_t part;
 
-                /* Page size. */
+                /** Page size. */
                 int32_t pageSize;
 
-                /* Local flag. */
+                /** Local flag. */
                 bool loc;
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/core/include/ignite/cache/query/query_text.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_text.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_text.h
index f4e7f99..4494883 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_text.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_text.h
@@ -29,13 +29,13 @@ namespace ignite
     {
         namespace query
         {         
-            /*
+            /**
              * Text query.
              */
             class TextQuery
             {
             public:
-                /*
+                /**
                  * Constructor.
                  *
                  * @param type Type name.
@@ -47,7 +47,7 @@ namespace ignite
                     // No-op.
                 }
                 
-                /*
+                /**
                  * Get type name.
                  *
                  * @return Type name.
@@ -57,7 +57,7 @@ namespace ignite
                     return type;
                 }
 
-                /*
+                /**
                  * Set type name.
                  *
                  * @param sql Type name.
@@ -67,7 +67,7 @@ namespace ignite
                     this->type = type;
                 }
 
-                /*
+                /**
                  * Get text string.
                  *
                  * @return text string.
@@ -77,7 +77,7 @@ namespace ignite
                     return text;
                 }
 
-                /*
+                /**
                  * Set text string.
                  *
                  * @param text Text string.
@@ -87,7 +87,7 @@ namespace ignite
                     this->text = text;
                 }
 
-                /*
+                /**
                  * Get page size.
                  *
                  * @return Page size.
@@ -97,7 +97,7 @@ namespace ignite
                     return pageSize;
                 }
 
-                /*
+                /**
                  * Set page size.
                  *
                  * @param pageSize Page size.
@@ -107,7 +107,7 @@ namespace ignite
                     this->pageSize = pageSize;
                 }
 
-                /*
+                /**
                  * Get local flag.
                  *
                  * @return Local flag.
@@ -117,17 +117,17 @@ namespace ignite
                     return loc;
                 }
 
-                /*
-                    * Set local flag.
-                    *
-                    * @param loc Local flag.
-                    */
+                /**
+                 * Set local flag.
+                 *
+                 * @param loc Local flag.
+                 */
                 void SetLocal(bool loc)
                 {
                     this->loc = loc;
                 }
                 
-                /*
+                /**
                  * Write query info to the stream.
                  *
                  * @param writer Writer.
@@ -141,16 +141,16 @@ namespace ignite
                 }
 
             private:
-                /* Type name. */
+                /** Type name. */
                 std::string type;
 
-                /* Text string. */
+                /** Text string. */
                 std::string text;
 
-                /* Page size. */
+                /** Page size. */
                 int32_t pageSize;
 
-                /* Local flag. */
+                /** Local flag. */
                 bool loc;
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/core/include/ignite/ignition.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/ignition.h b/modules/platforms/cpp/core/include/ignite/ignition.h
index 93ce166..f0046b0 100644
--- a/modules/platforms/cpp/core/include/ignite/ignition.h
+++ b/modules/platforms/cpp/core/include/ignite/ignition.h
@@ -15,14 +15,6 @@
  * limitations under the License.
  */
 
-/**
- * \mainpage Apache Ignite C++
- *
- * Apache Ignite In-Memory Data Fabric is a high-performance, integrated and distributed in-memory platform for
- * computing and transacting on large-scale data sets in real-time, orders of magnitude faster than possible with
- * traditional disk-based or flash-based technologies.
- */
-
 #ifndef _IGNITE_IGNITION
 #define _IGNITE_IGNITION
 
@@ -46,7 +38,7 @@ namespace ignite
          */
         static Ignite Start(const IgniteConfiguration& cfg);
 
-        /*
+        /**
          * Start Ignite instance.
          *
          * @param cfg Configuration.

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
index 31ebca1..546a97b 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
@@ -300,7 +300,7 @@ namespace ignite
                  */
                 query::QueryCursorImpl* QuerySql(const ignite::cache::query::SqlQuery& qry, IgniteError* err);
 
-                /*
+                /**
                  * Invoke text query.
                  *
                  * @param qry Query.
@@ -309,7 +309,7 @@ namespace ignite
                  */
                 query::QueryCursorImpl* QueryText(const ignite::cache::query::TextQuery& qry, IgniteError* err);
 
-                /*
+                /**
                  * Invoke scan query.
                  *
                  * @param qry Query.
@@ -318,7 +318,7 @@ namespace ignite
                  */
                 query::QueryCursorImpl* QueryScan(const ignite::cache::query::ScanQuery& qry, IgniteError* err);
 
-                /*
+                /**
                  * Invoke sql fields query.
                  *
                  * @param qry Query.

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/core/include/ignite/impl/operations.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/operations.h b/modules/platforms/cpp/core/include/ignite/impl/operations.h
index 5423a56..9f1f333 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/operations.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/operations.h
@@ -154,7 +154,7 @@ namespace ignite
             IGNITE_NO_COPY_ASSIGNMENT(In3Operation)
         };
 
-        /*
+        /**
          * Input set operation.
          */
         template<typename T>
@@ -192,7 +192,7 @@ namespace ignite
         class InMapOperation : public InputOperation
         {
         public:
-            /*
+            /**
              * Constructor.
              *
              * @param val Value.
@@ -360,7 +360,7 @@ namespace ignite
             IGNITE_NO_COPY_ASSIGNMENT(Out2Operation)
         };
         
-        /*
+        /**
          * Output map operation.
          */
         template<typename T1, typename T2>
@@ -412,7 +412,7 @@ namespace ignite
             IGNITE_NO_COPY_ASSIGNMENT(OutMapOperation)
         };
 
-        /*
+        /**
          * Output query GET ALL operation.
          */
         template<typename K, typename V>

http://git-wip-us.apache.org/repos/asf/ignite/blob/77c6d08a/modules/platforms/cpp/cpp.dxg
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/cpp.dxg b/modules/platforms/cpp/cpp.dxg
index 281c32b..42b0a36 100644
--- a/modules/platforms/cpp/cpp.dxg
+++ b/modules/platforms/cpp/cpp.dxg
@@ -150,7 +150,7 @@ SHORT_NAMES            = NO
 # comments will behave just like regular Qt-style comments
 # (thus requiring an explicit @brief command for a brief description.)
 
-JAVADOC_AUTOBRIEF      = NO
+JAVADOC_AUTOBRIEF      = YES
 
 # If the QT_AUTOBRIEF tag is set to YES then Doxygen will
 # interpret the first line (until the first dot) of a Qt-style
@@ -1715,7 +1715,7 @@ GENERATE_LEGEND        = YES
 
 DOT_CLEANUP            = YES
 
-;INPUT=core/include/ignite core/src
+;INPUT=core
 ;EXCLUDE=core/include/ignite/impl core/os/linux/include/ignite/impl core/os/linux/src/impl core/os/win/include/ignite/impl core/os/win/src/impl core/src/impl
 ;STRIP_FROM_PATH=core/include/ignite core/src
 ;OUTPUT_DIRECTORY=../../clients/target/cppdoc


[08/10] ignite git commit: ignite-1.5 Fixed tx entry add to always check for enlistWrite errors.

Posted by an...@apache.org.
ignite-1.5 Fixed tx entry add to always check for enlistWrite errors.


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

Branch: refs/heads/ignite-843-rc2
Commit: 490f552b7a7a205a5dc2d34051c365ccc8e9f205
Parents: 77c6d08
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 16 11:50:03 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 16 11:50:03 2015 +0300

----------------------------------------------------------------------
 .../transactions/IgniteTxLocalAdapter.java      | 23 +++++++++++++++++++-
 1 file changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/490f552b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index a3aed34..720832e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -2952,6 +2952,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
             if (pessimistic()) {
                 assert loadFut == null || loadFut.isDone() : loadFut;
 
+                if (loadFut != null)
+                    loadFut.get();
+
                 final Collection<KeyCacheObject> enlisted = Collections.singleton(cacheKey);
 
                 if (log.isDebugEnabled())
@@ -3124,6 +3127,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
             if (pessimistic()) {
                 assert loadFut == null || loadFut.isDone() : loadFut;
 
+                if (loadFut != null) {
+                    try {
+                        loadFut.get();
+                    }
+                    catch (IgniteCheckedException e) {
+                        return new GridFinishedFuture(e);
+                    }
+                }
+
                 if (log.isDebugEnabled())
                     log.debug("Before acquiring transaction lock for put on keys: " + enlisted);
 
@@ -3382,7 +3394,16 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         // Otherwise, during rollback we will not know whether locks need
         // to be rolled back.
         if (pessimistic()) {
-            assert loadFut.isDone() : loadFut;
+            assert loadFut == null || loadFut.isDone() : loadFut;
+
+            if (loadFut != null) {
+                try {
+                    loadFut.get();
+                }
+                catch (IgniteCheckedException e) {
+                    return new GridFinishedFuture<>(e);
+                }
+            }
 
             if (log.isDebugEnabled())
                 log.debug("Before acquiring transaction lock for remove on keys: " + enlisted);


[05/10] ignite git commit: IGNITE-1994: Removed .Net: CrossPlatformExample.

Posted by an...@apache.org.
IGNITE-1994: Removed .Net: CrossPlatformExample.


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

Branch: refs/heads/ignite-843-rc2
Commit: 1acbce1e7a59ae9e7436e083ab919df618e57c1a
Parents: ec3fd19
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Wed Dec 16 11:16:55 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 16 11:16:55 2015 +0300

----------------------------------------------------------------------
 .../Examples/ExamplesTest.cs                    |   4 +-
 .../Apache.Ignite.Examples.csproj               |   1 -
 .../Datagrid/CrossPlatformExample.cs            | 205 -------------------
 3 files changed, 2 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1acbce1e/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
index bcf6baa..4e8a036 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
@@ -68,10 +68,10 @@ namespace Apache.Ignite.Core.Tests.Examples
         /// <param name="clientMode">Client mode flag.</param>
         private static void TestRemoteNodes(Example example, bool clientMode)
         {
-            // Exclude CrossPlatformExample and LifecycleExample
+            // Exclude LifecycleExample
             if (string.IsNullOrEmpty(example.SpringConfigUrl))
             {
-                Assert.IsTrue(new[] {"CrossPlatformExample", "LifecycleExample"}.Contains(example.Name));
+                Assert.AreEqual("LifecycleExample", example.Name);
 
                 return;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1acbce1e/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
index 2f4ce18..abe4b14 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
@@ -57,7 +57,6 @@
     <Compile Include="Compute\ClosureExample.cs" />
     <Compile Include="Compute\TaskExample.cs" />
     <Compile Include="Datagrid\ContinuousQueryExample.cs" />
-    <Compile Include="Datagrid\CrossPlatformExample.cs" />
     <Compile Include="Datagrid\DataStreamerExample.cs" />
     <Compile Include="Datagrid\PutGetExample.cs" />
     <Compile Include="Datagrid\QueryExample.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/1acbce1e/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/CrossPlatformExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/CrossPlatformExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/CrossPlatformExample.cs
deleted file mode 100644
index 5c07204..0000000
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/CrossPlatformExample.cs
+++ /dev/null
@@ -1,205 +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.
- */
-
-using System;
-using System.Collections.Generic;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Binary;
-
-namespace Apache.Ignite.Examples.Datagrid
-{
-    using Apache.Ignite.ExamplesDll.Binary;
-
-    /// <summary>
-    /// This example demonstrates use of binary objects between different platforms.
-    /// <para/>
-    /// This example must be run with standalone Java node. To achieve this start a node from %IGNITE_HOME%
-    /// using "ignite.bat" with proper configuration:
-    /// <example>'bin\ignite.bat examples\config\example-server.xml'</example>.
-    /// <para />
-    /// Once remote node is started, launch this example as follows:
-    /// 1) Build the project Apache.Ignite.ExamplesDll (select it -> right-click -> Build);
-    /// 2) Set this class as startup object (Apache.Ignite.Examples project -> right-click -> Properties -> 
-    ///     Application -> Startup object); 
-    /// 3) Start application (F5 or Ctrl+F5).
-    /// <para />
-    /// To see how objects can be transferred between platforms, start cross-platform Java example 
-    /// without restarting remote node.
-    /// </summary>
-    public class CrossPlatformExample
-    {
-        /// <summary>Key for Java object.</summary>
-        private const int KeyJava = 100;
-
-        /// <summary>Key for .Net object.</summary>
-        private const int KeyDotnet = 200;
-
-        /// <summary>Key for C++ object.</summary>
-        private const int KeyCpp = 300;
-
-        /// <summary>Cache Name.</summary>
-        private const string CacheName = "cacheCrossPlatform";
-
-        /// <summary>
-        /// Runs the example.
-        /// </summary>
-        [STAThread]
-        public static void Main()
-        {
-            var cfg = new IgniteConfiguration
-            {
-                SpringConfigUrl = @"platforms\dotnet\examples\config\example-cache.xml",
-                JvmOptions = new List<string> { "-Xms512m", "-Xmx1024m" }
-            };
-
-            using (var ignite = Ignition.Start(cfg))
-            {
-                Console.WriteLine();
-                Console.WriteLine(">>> Cross-platform example started.");
-
-                if (ignite.GetCluster().ForRemotes().GetNodes().Count == 0)
-                {
-                    Console.WriteLine();
-                    Console.WriteLine(">>> This example requires remote nodes to be started.");
-                    Console.WriteLine(">>> Please start at least 1 remote node.");
-                    Console.WriteLine(">>> Refer to example's documentation for details on configuration.");
-                    Console.WriteLine();
-                }
-                else
-                {
-                    var cache = ignite.GetOrCreateCache<int, Organization>(CacheName);
-
-                    // Create new Organization object to store in cache.
-                    Organization org = new Organization(
-                        "Apache",
-                        new Address("1065 East Hillsdale Blvd, Foster City, CA", 94404),
-                        OrganizationType.Private,
-                        DateTime.Now
-                    );
-
-                    // Put created data entry to cache.
-                    cache.Put(KeyDotnet, org);
-
-                    // Retrieve value stored by Java client.
-                    GetFromJava(ignite);
-
-                    // Retrieve value stored by C++ client.
-                    GetFromCpp(ignite);
-
-                    // Gets binary value from cache in binary format, without de-serializing it.
-                    GetDotNetBinaryInstance(ignite);
-
-                    // Gets binary value form cache as a strongly-typed fully de-serialized instance.
-                    GetDotNetTypedInstance(ignite);
-
-                    Console.WriteLine();
-                }
-            }
-
-            Console.WriteLine();
-            Console.WriteLine(">>> Example finished, press any key to exit ...");
-            Console.ReadKey();
-        }
-
-        /// <summary>
-        /// Gets entry put by Java client. In order for entry to be in cache, Java client example
-        /// must be run before this example.
-        /// </summary>
-        /// <param name="ignite">Ignite instance.</param>
-        private static void GetFromJava(IIgnite ignite)
-        {
-            var cache = ignite.GetOrCreateCache<int, IBinaryObject>(CacheName)
-                .WithKeepBinary<int, IBinaryObject>();
-
-            var orgBinary = cache.GetAsync(KeyJava).Result;
-
-            if (orgBinary == null)
-            {
-                Console.WriteLine(">>> Java client hasn't put entry to cache. Run Java example before this example " +
-                    "to see the output.");
-            }
-            else
-            {
-                Console.WriteLine(">>> Entry from Java client:");
-                Console.WriteLine(">>>     Binary:     " + orgBinary);
-                Console.WriteLine(">>>     Deserialized: " + orgBinary.Deserialize<Organization>());
-            }
-        }
-
-        /// <summary>
-        /// Gets entry put by C++ client. In order for entry to be in cache, C++ client example
-        /// must be run before this example.
-        /// </summary>
-        /// <param name="ignite">Ignite instance.</param>
-        private static void GetFromCpp(IIgnite ignite)
-        {
-            var cache = ignite.GetOrCreateCache<int, IBinaryObject>(CacheName)
-                .WithKeepBinary<int, IBinaryObject>();
-
-            var orgBinary = cache.GetAsync(KeyCpp).Result;
-
-            Console.WriteLine();
-
-            if (orgBinary == null)
-            {
-                Console.WriteLine(">>> CPP client hasn't put entry to cache. Run CPP example before this example " +
-                    "to see the output.");
-            }
-            else
-            {
-                Console.WriteLine(">>> Entry from CPP client:");
-                Console.WriteLine(">>>     Binary:     " + orgBinary);
-                Console.WriteLine(">>>     Deserialized: " + orgBinary.Deserialize<Organization>());
-            }
-        }
-
-        /// <summary>
-        /// Gets binary value from cache in binary format, without de-serializing it.
-        /// </summary>
-        /// <param name="ignite">Ignite instance.</param>
-        private static void GetDotNetBinaryInstance(IIgnite ignite)
-        {
-            // Apply "KeepBinary" flag on data projection.
-            var cache = ignite.GetOrCreateCache<int, IBinaryObject>(CacheName)
-                .WithKeepBinary<int, IBinaryObject>();
-
-            var org = cache.Get(KeyDotnet);
-
-            string name = org.GetField<string>("name");
-
-            Console.WriteLine();
-            Console.WriteLine(">>> Retrieved organization name from binary field: " + name);
-        }
-
-        /// <summary>
-        /// Gets binary value form cache as a strongly-typed fully de-serialized instance.
-        /// </summary>
-        /// <param name="ignite">Ignite instance.</param>
-        private static void GetDotNetTypedInstance(IIgnite ignite)
-        {
-            var cache = ignite.GetOrCreateCache<int, Organization>(CacheName);
-
-            // Get recently created employee as a strongly-typed fully de-serialized instance.
-            Organization emp = cache.Get(KeyDotnet);
-
-            string name = emp.Name;
-
-            Console.WriteLine();
-            Console.WriteLine(">>> Retrieved organization name from deserialized Organization instance: " + name);
-        }
-    }
-}


[10/10] ignite git commit: IGNITE-2132 Enable test drive for sql from web.

Posted by an...@apache.org.
IGNITE-2132 Enable test drive for sql from web.


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

Branch: refs/heads/ignite-843-rc2
Commit: 46709b002efdcbe4fb2bd08f8f41250da0195353
Parents: 0eb69c6
Author: Andrey <an...@gridgain.com>
Authored: Wed Dec 16 17:28:19 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Wed Dec 16 17:28:19 2015 +0700

----------------------------------------------------------------------
 .../ignite/agent/handlers/RestExecutor.java     | 11 +++++++++-
 .../agent/testdrive/AgentSqlTestDrive.java      |  8 +++++--
 .../src/main/js/agents/agent-manager.js         |  8 +++++++
 .../src/main/js/controllers/common-module.js    | 22 +++++++++++++++++++-
 .../src/main/js/public/stylesheets/style.scss   |  4 ++++
 .../src/main/js/routes/agent.js                 | 16 +++++++++++++-
 .../js/views/configuration/metadata-load.jade   |  2 +-
 .../main/js/views/templates/agent-download.jade |  4 ++--
 8 files changed, 67 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/46709b00/modules/control-center-agent/src/main/java/org/apache/ignite/agent/handlers/RestExecutor.java
----------------------------------------------------------------------
diff --git a/modules/control-center-agent/src/main/java/org/apache/ignite/agent/handlers/RestExecutor.java b/modules/control-center-agent/src/main/java/org/apache/ignite/agent/handlers/RestExecutor.java
index 2d4d154..f91c14c 100644
--- a/modules/control-center-agent/src/main/java/org/apache/ignite/agent/handlers/RestExecutor.java
+++ b/modules/control-center-agent/src/main/java/org/apache/ignite/agent/handlers/RestExecutor.java
@@ -40,6 +40,7 @@ import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClientBuilder;
 import org.apache.ignite.agent.AgentConfiguration;
 import org.apache.ignite.agent.remote.Remote;
+import org.apache.ignite.agent.testdrive.AgentSqlTestDrive;
 
 import static org.apache.ignite.agent.AgentConfiguration.DFLT_NODE_PORT;
 
@@ -98,7 +99,7 @@ public class RestExecutor {
 
         if (path != null) {
             if (!path.startsWith("/") && !cfg.nodeUri().endsWith("/"))
-                path = '/' +  path;
+                path = '/' + path;
 
             builder.setPath(path);
         }
@@ -166,6 +167,14 @@ public class RestExecutor {
     }
 
     /**
+     * Enable test-drive SQL.
+     */
+    @Remote
+    public boolean enableTestDriveSQL() {
+        return AgentSqlTestDrive.testDrive(cfg);
+    }
+
+    /**
      * Request result.
      */
     public static class RestResult {

http://git-wip-us.apache.org/repos/asf/ignite/blob/46709b00/modules/control-center-agent/src/main/java/org/apache/ignite/agent/testdrive/AgentSqlTestDrive.java
----------------------------------------------------------------------
diff --git a/modules/control-center-agent/src/main/java/org/apache/ignite/agent/testdrive/AgentSqlTestDrive.java b/modules/control-center-agent/src/main/java/org/apache/ignite/agent/testdrive/AgentSqlTestDrive.java
index d9fe0d2..221b586 100644
--- a/modules/control-center-agent/src/main/java/org/apache/ignite/agent/testdrive/AgentSqlTestDrive.java
+++ b/modules/control-center-agent/src/main/java/org/apache/ignite/agent/testdrive/AgentSqlTestDrive.java
@@ -458,7 +458,7 @@ public class AgentSqlTestDrive {
     /**
      * Start ignite node with cacheEmployee and populate it with data.
      */
-    public static void testDrive(AgentConfiguration acfg) {
+    public static boolean testDrive(AgentConfiguration acfg) {
         if (initLatch.compareAndSet(false, true)) {
             log.log(Level.INFO, "TEST-DRIVE-SQL: Starting embedded node for sql test-drive...");
 
@@ -496,7 +496,7 @@ public class AgentSqlTestDrive {
                 if (F.isEmpty(host) || port == null) {
                     log.log(Level.SEVERE, "TEST-DRIVE-SQL: Failed to start embedded node with rest!");
 
-                    return;
+                    return false;
                 }
 
                 acfg.nodeUri(String.format("http://%s:%d", "0.0.0.0".equals(host) ? "127.0.0.1" : host, port));
@@ -507,7 +507,11 @@ public class AgentSqlTestDrive {
             }
             catch (Exception e) {
                 log.log(Level.SEVERE, "TEST-DRIVE-SQL: Failed to start embedded node for sql test-drive!", e);
+
+                return false;
             }
         }
+
+        return true;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/46709b00/modules/control-center-web/src/main/js/agents/agent-manager.js
----------------------------------------------------------------------
diff --git a/modules/control-center-web/src/main/js/agents/agent-manager.js b/modules/control-center-web/src/main/js/agents/agent-manager.js
index cd6b084..c75f4ca 100644
--- a/modules/control-center-web/src/main/js/agents/agent-manager.js
+++ b/modules/control-center-web/src/main/js/agents/agent-manager.js
@@ -194,6 +194,14 @@ Client.prototype.availableDrivers = function(cb) {
     this._invokeRmtMethod('availableDrivers', arguments)
 };
 
+/**
+ * @param {Function} cb Callback. Take two arguments: {Object} exception, {Object} result.
+ * @return {Boolean} If test-drive SQL was enabled.
+ */
+Client.prototype.enableTestDriveSQL = function(cb) {
+    this._invokeRmtMethod('enableTestDriveSQL', arguments)
+};
+
 Client.prototype._invokeRmtMethod = function(methodName, args) {
     var cb = null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/46709b00/modules/control-center-web/src/main/js/controllers/common-module.js
----------------------------------------------------------------------
diff --git a/modules/control-center-web/src/main/js/controllers/common-module.js b/modules/control-center-web/src/main/js/controllers/common-module.js
index 7545d6d..2357a02 100644
--- a/modules/control-center-web/src/main/js/controllers/common-module.js
+++ b/modules/control-center-web/src/main/js/controllers/common-module.js
@@ -1936,7 +1936,10 @@ consoleModule.controller('auth', [
 
 // Download agent controller.
 consoleModule.controller('agent-download', [
-    '$http', '$common', '$scope', '$interval', '$modal', '$state', function ($http, $common, $scope, $interval, $modal, $state) {
+    '$http', '$common', '$scope', '$interval', '$modal', '$loading', '$state',
+        function ($http, $common, $scope, $interval, $modal, $loading, $state) {
+        $scope.loadingAgentOptions = { text: 'Enabling test-drive SQL...' };
+
         // Pre-fetch modal dialogs.
         var _agentDownloadModal = $modal({scope: $scope, templateUrl: '/templates/agent-download.html', show: false, backdrop: 'static'});
 
@@ -1977,6 +1980,21 @@ consoleModule.controller('agent-download', [
             document.body.removeChild(lnk);
         };
 
+        $scope.enableTestDriveSQL = function () {
+            $loading.start('loadingAgent');
+
+            $http.post('/api/v1/agent/testdrive/sql')
+                .success(function (result) {
+                    if (!result)
+                        $common.showError('Failed to start test-drive sql', 'top-right', 'body', true);
+                })
+                .error(function (errMsg, status) {
+                    $loading.finish('loadingAgent');
+
+                    _handleException(errMsg, status);
+                });
+        };
+
         /**
          * Base handler of exceptions on agent interaction
          *
@@ -2034,6 +2052,8 @@ consoleModule.controller('agent-download', [
                     if (_agentDownloadModal.awaitFirstSuccess)
                         _stopInterval();
 
+                    $loading.finish('loadingAgent');
+
                     _agentDownloadModal.checkFn(result, _agentDownloadModal.hide, _handleException);
                 })
                 .error(function (errMsg, status) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/46709b00/modules/control-center-web/src/main/js/public/stylesheets/style.scss
----------------------------------------------------------------------
diff --git a/modules/control-center-web/src/main/js/public/stylesheets/style.scss b/modules/control-center-web/src/main/js/public/stylesheets/style.scss
index 1f217f9..c3c5fa3 100644
--- a/modules/control-center-web/src/main/js/public/stylesheets/style.scss
+++ b/modules/control-center-web/src/main/js/public/stylesheets/style.scss
@@ -1628,6 +1628,10 @@ a {
     .dw-loading.dw-loading-overlay {
         z-index: 9999;
     }
+
+    .dw-loading-body {
+        left: 10%;
+    }
 }
 
 .panel-tip-container {

http://git-wip-us.apache.org/repos/asf/ignite/blob/46709b00/modules/control-center-web/src/main/js/routes/agent.js
----------------------------------------------------------------------
diff --git a/modules/control-center-web/src/main/js/routes/agent.js b/modules/control-center-web/src/main/js/routes/agent.js
index 1d40dba..f915df2 100644
--- a/modules/control-center-web/src/main/js/routes/agent.js
+++ b/modules/control-center-web/src/main/js/routes/agent.js
@@ -45,7 +45,7 @@ router.get('/download/zip', function (req, res) {
     var JSZip = require('jszip');
     var config = require('../helpers/configuration-loader.js');
 
-    var agentFld = 'ignite-web-agent-1.5.0-SNAPSHOT';
+    var agentFld = 'ignite-web-agent-1.5.0-b2-SNAPSHOT';
     var agentZip = agentFld + '.zip';
 
     // Read a zip file.
@@ -280,6 +280,20 @@ router.post('/ping', function (req, res) {
         res.sendStatus(200);
 });
 
+/* Enable test-drive for sql. */
+router.post('/testdrive/sql', function (req, res) {
+    var client = _client(req, res);
+
+    if (client) {
+        client.enableTestDriveSQL(function (err, drivers) {
+            if (err)
+                return res.status(500).send(err);
+
+            res.sendStatus(200);
+        });
+    }
+});
+
 /* Get JDBC drivers list. */
 router.post('/drivers', function (req, res) {
     var client = _client(req, res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/46709b00/modules/control-center-web/src/main/js/views/configuration/metadata-load.jade
----------------------------------------------------------------------
diff --git a/modules/control-center-web/src/main/js/views/configuration/metadata-load.jade b/modules/control-center-web/src/main/js/views/configuration/metadata-load.jade
index b8c85ac..a6800d0 100644
--- a/modules/control-center-web/src/main/js/views/configuration/metadata-load.jade
+++ b/modules/control-center-web/src/main/js/views/configuration/metadata-load.jade
@@ -20,7 +20,7 @@ mixin chk(mdl, change, tip)
     input(type='checkbox' ng-model=mdl ng-change=change bs-tooltip='' data-title=tip data-placement='bottom')
 
 .modal.center(role='dialog')
-    .modal-dialog(ng-cloak)
+    .modal-dialog
         .modal-content(dw-loading='loadingMetadataFromDb' dw-loading-options='loadMeta.loadingOptions')
             #errors-container.modal-header.header
                 button.close(ng-click='$hide()' aria-hidden='true') &times;

http://git-wip-us.apache.org/repos/asf/ignite/blob/46709b00/modules/control-center-web/src/main/js/views/templates/agent-download.jade
----------------------------------------------------------------------
diff --git a/modules/control-center-web/src/main/js/views/templates/agent-download.jade b/modules/control-center-web/src/main/js/views/templates/agent-download.jade
index 05a73e6..aef3ae7 100644
--- a/modules/control-center-web/src/main/js/views/templates/agent-download.jade
+++ b/modules/control-center-web/src/main/js/views/templates/agent-download.jade
@@ -14,7 +14,7 @@
 
 .modal.center(tabindex='-1' role='dialog')
     .modal-dialog
-        .modal-content
+        .modal-content(dw-loading='loadingAgent' dw-loading-options='loadingAgentOptions')
             #errors-container.modal-header.header
                 h4.modal-title(ng-if='!nodeFailedConnection') Connection to Ignite Web Agent is not established
                 h4.modal-title(ng-if='nodeFailedConnection') Connection to Ignite Node is not established
@@ -53,5 +53,5 @@
                         | &nbsp; in agent folder for more information
             .modal-footer
                 button.btn.btn-default(ng-click='goBack()') Back to {{::agentDownloadBackTo}}
-                button.btn.btn-primary(ng-if='nodeFailedConnection' ng-click='downloadAgent()') Start test-drive
+                button.btn.btn-primary(ng-if='nodeFailedConnection' ng-click='enableTestDriveSQL()') Start test-drive sql
                 button.btn.btn-primary(ng-if='!nodeFailedConnection' ng-click='downloadAgent()') Download zip


[03/10] ignite git commit: ignite-1.5 Fixed test to avoid timeout.

Posted by an...@apache.org.
ignite-1.5 Fixed test to avoid timeout.


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

Branch: refs/heads/ignite-843-rc2
Commit: 85ec80870e640725d4e532f9a3c1b3de304fc6de
Parents: a586b49
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 16 09:49:33 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 16 09:49:33 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheSerializableTransactionsTest.java  | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/85ec8087/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
index f4533f2..9906ad3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
@@ -2832,6 +2832,8 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
     public void testRandomOperations() throws Exception {
         Ignite ignite0 = ignite(0);
 
+        long stopTime = U.currentTimeMillis() + getTestTimeout() - 30_000;
+
         for (CacheConfiguration<Integer, Integer> ccfg : cacheConfigurations()) {
             logCacheInfo(ccfg);
 
@@ -2869,6 +2871,9 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
                             tx.commit();
                         }
+
+                        if (i % 100 == 0 && U.currentTimeMillis() > stopTime)
+                            break;
                     }
 
                     for (int key = 0; key < KEYS; key++) {
@@ -2877,6 +2882,9 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
                         for (int node = 1; node < SRVS + CLIENTS; node++)
                             assertEquals(val, ignite(node).cache(cache.getName()).get(key));
                     }
+
+                    if (U.currentTimeMillis() > stopTime)
+                        break;
                 }
             }
             finally {


[02/10] ignite git commit: ignite-1.5 More info in test assert.

Posted by an...@apache.org.
ignite-1.5 More info in test assert.


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

Branch: refs/heads/ignite-843-rc2
Commit: a586b49aec8671b4485886ed0ccbe0e011fe4a3e
Parents: ab8ba97
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 16 09:04:40 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 16 09:04:40 2015 +0300

----------------------------------------------------------------------
 .../processors/service/GridServiceProcessorProxySelfTest.java     | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a586b49a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
index f69176c..6fc7e02 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
@@ -29,7 +29,6 @@ import org.apache.ignite.services.ServiceContext;
  * Service proxy test.
  */
 public class GridServiceProcessorProxySelfTest extends GridServiceProcessorAbstractSelfTest {
-
     /** {@inheritDoc} */
     @Override protected int nodeCount() {
         return 4;
@@ -216,7 +215,7 @@ public class GridServiceProcessorProxySelfTest extends GridServiceProcessorAbstr
             MapService<Integer, String> svc =  grid(i).services().serviceProxy(name, MapService.class, false);
 
             // Make sure service is a local instance.
-            assertTrue(svc instanceof Service);
+            assertTrue("Invalid service instance [srv=" + svc + ", node=" + i + ']', svc instanceof Service);
 
             svc.put(i, Integer.toString(i));
         }


[04/10] ignite git commit: IGNITE-2152: Main page description for .NET docs.

Posted by an...@apache.org.
IGNITE-2152: Main page description for .NET docs.


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

Branch: refs/heads/ignite-843-rc2
Commit: ec3fd1968ef477605738eafc0a2795a808290da2
Parents: 85ec808
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Wed Dec 16 11:14:53 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 16 11:14:53 2015 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.csproj                   |  1 +
 .../dotnet/Apache.Ignite.Core/Index.cs          | 31 ++++++++++++++++++++
 modules/platforms/dotnet/Apache.Ignite.sln      |  1 +
 3 files changed, 33 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ec3fd196/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index b3bd712..740852d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -309,6 +309,7 @@
     <Compile Include="Impl\Unmanaged\UnmanagedNonReleaseableTarget.cs" />
     <Compile Include="Impl\Unmanaged\UnmanagedTarget.cs" />
     <Compile Include="Impl\Unmanaged\UnmanagedUtils.cs" />
+    <Compile Include="Index.cs" />
     <Compile Include="Lifecycle\ILifecycleBean.cs" />
     <Compile Include="Lifecycle\LifecycleEventType.cs" />
     <Compile Include="Messaging\IMessageListener.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec3fd196/modules/platforms/dotnet/Apache.Ignite.Core/Index.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Index.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Index.cs
new file mode 100644
index 0000000..c6dc5d8
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Index.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core
+{
+    // Doxygen main page
+
+    /**
+
+    \mainpage Apache Ignite.NET In-Memory Data Fabric
+
+    Apache Ignite.NET In-Memory Data Fabric is a high-performance, integrated and distributed in-memory platform for 
+    computing and transacting on large-scale data sets in real-time, orders of magnitude faster than possible with 
+    traditional disk-based or flash-based technologies.
+
+    */
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec3fd196/modules/platforms/dotnet/Apache.Ignite.sln
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.sln b/modules/platforms/dotnet/Apache.Ignite.sln
index 19916f2..5c6ce05 100644
--- a/modules/platforms/dotnet/Apache.Ignite.sln
+++ b/modules/platforms/dotnet/Apache.Ignite.sln
@@ -22,6 +22,7 @@ Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Benchmarks",
 EndProject
 Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Solution Items", "Solution Items", "{E0784F76-949C-456E-A529-A55D0E389165}"
 	ProjectSection(SolutionItems) = preProject
+		Apache.Ignite.dxg = Apache.Ignite.dxg
 		Apache.Ignite.FxCop = Apache.Ignite.FxCop
 		Apache.Ignite.sln.DotSettings = Apache.Ignite.sln.DotSettings
 		build.bat = build.bat