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

[1/5] ignite git commit: IGNITE-3408 - Provided basic implementation of AddressResolver

Repository: ignite
Updated Branches:
  refs/heads/ignite-3443 8033f119e -> d8b92b4b4


IGNITE-3408 - Provided basic implementation of AddressResolver


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

Branch: refs/heads/ignite-3443
Commit: a90d05b30f39da001156fdd4053790f8cd352406
Parents: ae23dab
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Aug 3 22:00:15 2016 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Aug 3 22:00:15 2016 -0700

----------------------------------------------------------------------
 .../configuration/BasicAddressResolver.java     | 146 ++++++++++++
 .../spi/GridTcpSpiForwardingSelfTest.java       | 237 +++++++++++++++++--
 2 files changed, 364 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a90d05b3/modules/core/src/main/java/org/apache/ignite/configuration/BasicAddressResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/BasicAddressResolver.java b/modules/core/src/main/java/org/apache/ignite/configuration/BasicAddressResolver.java
new file mode 100644
index 0000000..d29689f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/BasicAddressResolver.java
@@ -0,0 +1,146 @@
+/*
+ * 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.configuration;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Basic implementation of {@link AddressResolver}.
+ * <p>
+ * Allows to provide simple mapping between different address, which is useful
+ * different parts of the cluster are located in different subnetworks and port
+ * forwarding on the router is used for communication between them. Another
+ * common case is Docker environment which can create a pair of public and private
+ * address per container.
+ * <p>
+ * There are two different types of mapping supported by this implementation of
+ * {@link AddressResolver}.
+ * <p>
+ * First type maps specific socket addresses (host and port pairs) are mapped to
+ * each other. This is useful for port forwarding, where multiple nodes will
+ * typically map to the same external address (router), but with different port
+ * numbers. Here is the example:
+ * <pre name="code" class="xml">
+ * &lt;property name=&quot;addressResolver&quot;&gt;
+ *     &lt;bean class=&quot;org.apache.ignite.configuration.BasicAddressResolver&quot;&gt;
+ *         &lt;constructor-arg&gt;
+ *             &lt;map&gt;
+ *                 &lt;entry key=&quot;10.0.0.1:47100&quot; value=&quot;123.123.123.123:1111&quot;/&gt;
+ *                 &lt;entry key=&quot;10.0.0.2:47100&quot; value=&quot;123.123.123.123:2222&quot;/&gt;
+ *                 &lt;entry key=&quot;10.0.0.3:47100&quot; value=&quot;123.123.123.123:3333&quot;/&gt;
+ *             &lt;/map&gt;
+ *         &lt;/constructor-arg&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * <p>
+ * Second type maps one host to another. In this case any internal address a node
+ * is bound to will be mapped to the corresponding external host with the same
+ * host number. Here is the example:
+ * <pre name="code" class="xml">
+ * &lt;property name=&quot;addressResolver&quot;&gt;
+ *     &lt;bean class=&quot;org.apache.ignite.configuration.BasicAddressResolver&quot;&gt;
+ *         &lt;constructor-arg&gt;
+ *             &lt;map&gt;
+ *                 &lt;entry key=&quot;10.0.0.1&quot; value=&quot;123.123.123.123&quot;/&gt;
+ *             &lt;/map&gt;
+ *         &lt;/constructor-arg&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * Here any port on {@code 10.0.0.1} will be mapped to the same port number on {@code 123.123.123.123}.
+ * E.g., {@code 10.0.0.1:47100} will be mapped to {@code 123.123.123.123:47100}, and {@code 10.0.0.1:47500}
+ * will be mapped to {@code 123.123.123.123:47500}.
+ * <p>
+ * Two types of mappings described above can be mixed within one address resolver.
+ */
+public class BasicAddressResolver implements AddressResolver {
+    /** Address map. */
+    private final Map<InetAddress, InetAddress> inetAddrMap;
+
+    /** Socket address map. */
+    private final Map<InetSocketAddress, InetSocketAddress> inetSockAddrMap;
+
+    /**
+     * Created the address resolver.
+     *
+     * @param addrMap Address mappings.
+     * @throws UnknownHostException If any of the hosts can't be resolved.
+     */
+    public BasicAddressResolver(Map<String, String> addrMap) throws UnknownHostException {
+        if (addrMap == null || addrMap.isEmpty())
+            throw new IllegalArgumentException("At least one address mapping is required.");
+
+        inetAddrMap = U.newHashMap(addrMap.size());
+        inetSockAddrMap = U.newHashMap(addrMap.size());
+
+        for (Map.Entry<String, String> e : addrMap.entrySet()) {
+            String from = e.getKey();
+            String to = e.getValue();
+
+            if (F.isEmpty(from) || F.isEmpty(to))
+                throw new IllegalArgumentException("Invalid address mapping: " + e);
+
+            String[] fromArr = from.split(":");
+            String[] toArr = to.split(":");
+
+            assert fromArr.length > 0;
+            assert toArr.length > 0;
+
+            if (fromArr.length == 1) {
+                if (toArr.length != 1)
+                    throw new IllegalArgumentException("Invalid address mapping: " + e);
+
+                inetAddrMap.put(InetAddress.getByName(fromArr[0]), InetAddress.getByName(toArr[0]));
+            }
+            else if (fromArr.length == 2) {
+                if (toArr.length != 2)
+                    throw new IllegalArgumentException("Invalid address mapping: " + e);
+
+                inetSockAddrMap.put(new InetSocketAddress(fromArr[0], Integer.parseInt(fromArr[1])),
+                    new InetSocketAddress(toArr[0], Integer.parseInt(toArr[1])));
+            }
+            else
+                throw new IllegalArgumentException("Invalid address mapping: " + e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<InetSocketAddress> getExternalAddresses(InetSocketAddress addr)
+        throws IgniteCheckedException {
+        InetSocketAddress inetSockAddr = inetSockAddrMap.get(addr);
+
+        if (inetSockAddr != null)
+            return Collections.singletonList(inetSockAddr);
+
+        InetAddress inetAddr = inetAddrMap.get(addr.getAddress());
+
+        if (inetAddr != null)
+            return Collections.singletonList(new InetSocketAddress(inetAddr, addr.getPort()));
+
+        return Collections.emptyList();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a90d05b3/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 0c3ae12..652e47f 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
@@ -23,10 +23,12 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.AddressResolver;
+import org.apache.ignite.configuration.BasicAddressResolver;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.nio.GridCommunicationClient;
 import org.apache.ignite.internal.util.typedef.F;
@@ -35,10 +37,11 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
- * Test for {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi} and {@link org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi}.
+ * Test for {@link TcpDiscoverySpi} and {@link TcpCommunicationSpi}.
  */
 public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
     /** */
@@ -65,30 +68,34 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
     /** */
     private static final int commExtPort2 = 20100;
 
+    /** */
+    private AddressResolver resolver;
+
+    /** */
+    private boolean ipFinderUseLocPorts;
+
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "deprecation"})
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder();
-        ipFinder.setAddresses(Arrays.asList("127.0.0.1:" + extPort1, "127.0.0.1:" + extPort2));
+
+        if (ipFinderUseLocPorts)
+            ipFinder.setAddresses(Arrays.asList("127.0.0.1:" + locPort1, "127.0.0.1:" + locPort2));
+        else
+            ipFinder.setAddresses(Arrays.asList("127.0.0.1:" + extPort1, "127.0.0.1:" + extPort2));
 
         TcpDiscoverySpi spi = new TcpDiscoverySpi();
 
         final int locPort;
-        final int extPort;
-        final int commExtPort;
         final int commLocPort;
 
         if (getTestGridName(0).equals(gridName)) {
             locPort = locPort1;
-            extPort = extPort1;
             commLocPort = commLocPort1;
-            commExtPort = commExtPort1;
         }
         else if (getTestGridName(1).equals(gridName)) {
             locPort = locPort2;
-            extPort = extPort2;
             commLocPort = commLocPort2;
-            commExtPort = commExtPort2;
         }
         else
             throw new IllegalArgumentException("Unknown grid name");
@@ -106,6 +113,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         TcpCommunicationSpi commSpi = new TcpCommunicationSpi() {
             @Override protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException {
                 Map<String, Object> attrs = new HashMap<>(node.attributes());
+
                 attrs.remove(createSpiAttributeName(ATTR_PORT));
 
                 ((TcpDiscoveryNode)node).setAttributes(attrs);
@@ -121,16 +129,9 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
 
         cfg.setCommunicationSpi(commSpi);
 
-        final Map<InetSocketAddress, ? extends Collection<InetSocketAddress>> mp = F.asMap(
-            new InetSocketAddress("127.0.0.1", locPort), F.asList(new InetSocketAddress("127.0.0.1", extPort)),
-            new InetSocketAddress("127.0.0.1", commLocPort), F.asList(new InetSocketAddress("127.0.0.1", commExtPort))
-        );
+        assert resolver != null;
 
-        cfg.setAddressResolver(new AddressResolver() {
-            @Override public Collection<InetSocketAddress> getExternalAddresses(InetSocketAddress addr) {
-                return mp.get(addr);
-            }
-        });
+        cfg.setAddressResolver(resolver);
 
         return cfg;
     }
@@ -138,8 +139,206 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If any error occurs.
      */
+    public void testCustomResolver() throws Exception {
+        final Map<InetSocketAddress, Collection<InetSocketAddress>> map = new HashMap<>();
+
+        map.put(new InetSocketAddress("127.0.0.1", locPort1), F.asList(new InetSocketAddress("127.0.0.1", extPort1)));
+        map.put(new InetSocketAddress("127.0.0.1", commLocPort1), F.asList(new InetSocketAddress("127.0.0.1", commExtPort1)));
+        map.put(new InetSocketAddress("127.0.0.1", locPort2), F.asList(new InetSocketAddress("127.0.0.1", extPort2)));
+        map.put(new InetSocketAddress("127.0.0.1", commLocPort2), F.asList(new InetSocketAddress("127.0.0.1", commExtPort2)));
+
+        resolver = new AddressResolver() {
+            @Override public Collection<InetSocketAddress> getExternalAddresses(InetSocketAddress addr) {
+                return map.get(addr);
+            }
+        };
+
+        doTestForward();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBasicResolverMapPorts() throws Exception {
+        Map<String, String> map = new HashMap<>();
+
+        map.put("127.0.0.1:" + locPort1, "127.0.0.1:" + extPort1);
+        map.put("127.0.0.1:" + commLocPort1, "127.0.0.1:" + commExtPort1);
+        map.put("127.0.0.1:" + locPort2, "127.0.0.1:" + extPort2);
+        map.put("127.0.0.1:" + commLocPort2, "127.0.0.1:" + commExtPort2);
+
+        resolver = new BasicAddressResolver(map);
+
+        doTestForward();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBasicResolverMapAddress() throws Exception {
+        Map<String, String> map = new HashMap<>();
+
+        map.put("127.0.0.1", "127.0.0.1");
+
+        resolver = new BasicAddressResolver(map);
+
+        ipFinderUseLocPorts = true;
+
+        doTestForward();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBasicResolverErrors() throws Exception {
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    return new BasicAddressResolver(null);
+                }
+            },
+            IllegalArgumentException.class,
+            "At least one address mapping is required."
+        );
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    return new BasicAddressResolver(new HashMap<String, String>());
+                }
+            },
+            IllegalArgumentException.class,
+            "At least one address mapping is required."
+        );
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Map<String, String> map = new HashMap<>();
+
+                    map.put("from", null);
+
+                    return new BasicAddressResolver(map);
+                }
+            },
+            IllegalArgumentException.class,
+            "Invalid address mapping: from=null"
+        );
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Map<String, String> map = new HashMap<>();
+
+                    map.put("from", "");
+
+                    return new BasicAddressResolver(map);
+                }
+            },
+            IllegalArgumentException.class,
+            "Invalid address mapping: from="
+        );
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Map<String, String> map = new HashMap<>();
+
+                    map.put(null, "to");
+
+                    return new BasicAddressResolver(map);
+                }
+            },
+            IllegalArgumentException.class,
+            "Invalid address mapping: null=to"
+        );
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Map<String, String> map = new HashMap<>();
+
+                    map.put("", "to");
+
+                    return new BasicAddressResolver(map);
+                }
+            },
+            IllegalArgumentException.class,
+            "Invalid address mapping: =to"
+        );
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Map<String, String> map = new HashMap<>();
+
+                    map.put("from", "to:1111");
+
+                    return new BasicAddressResolver(map);
+                }
+            },
+            IllegalArgumentException.class,
+            "Invalid address mapping: from=to:1111"
+        );
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Map<String, String> map = new HashMap<>();
+
+                    map.put("from:1111", "to");
+
+                    return new BasicAddressResolver(map);
+                }
+            },
+            IllegalArgumentException.class,
+            "Invalid address mapping: from:1111=to"
+        );
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Map<String, String> map = new HashMap<>();
+
+                    map.put("from:1111:2222", "to:1111");
+
+                    return new BasicAddressResolver(map);
+                }
+            },
+            IllegalArgumentException.class,
+            "Invalid address mapping: from:1111:2222=to:1111"
+        );
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Map<String, String> map = new HashMap<>();
+
+                    map.put("from:1111", "to:1111:2222");
+
+                    return new BasicAddressResolver(map);
+                }
+            },
+            IllegalArgumentException.class,
+            "Invalid address mapping: from:1111=to:1111:2222"
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     @SuppressWarnings("UnusedDeclaration")
-    public void testForward() throws Exception {
+    private void doTestForward() throws Exception {
         InetAddress locHost = InetAddress.getByName("127.0.0.1");
 
         try (
@@ -163,4 +362,4 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
             assertEquals(F.asList(13, 13), t);
         }
     }
-}
\ No newline at end of file
+}


[3/5] ignite git commit: Merge branch 'ignite-1.6.4' into ignite-1.7.1

Posted by ak...@apache.org.
Merge branch 'ignite-1.6.4' into ignite-1.7.1


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

Branch: refs/heads/ignite-3443
Commit: 8c4a1c7e82ba211e69f2f6769569bff25526d7f1
Parents: e6d6286 a90d05b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 4 12:45:57 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 4 12:45:57 2016 +0300

----------------------------------------------------------------------
 .../configuration/BasicAddressResolver.java     | 146 ++++++++++++
 .../spi/GridTcpSpiForwardingSelfTest.java       | 237 +++++++++++++++++--
 2 files changed, 364 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



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

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


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

Branch: refs/heads/ignite-3443
Commit: d8b92b4b45f8332b839908c2d524a55d8f52425a
Parents: 8033f11 59fb5ef
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 4 16:52:03 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 4 16:52:03 2016 +0700

----------------------------------------------------------------------
 .../configuration/BasicAddressResolver.java     | 146 ++++++++++++
 .../spi/GridTcpSpiForwardingSelfTest.java       | 237 +++++++++++++++++--
 .../IgniteConfigurationSerializerTest.cs        | 145 ++++++++++--
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  65 +++++
 4 files changed, 559 insertions(+), 34 deletions(-)
----------------------------------------------------------------------



[4/5] ignite git commit: Merge branch 'ignite-1.7.1'

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


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

Branch: refs/heads/ignite-3443
Commit: 59fb5ef082821058ff9b858a5888cf466a5ec5e3
Parents: e087152 8c4a1c7
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 4 12:47:15 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 4 12:47:15 2016 +0300

----------------------------------------------------------------------
 .../configuration/BasicAddressResolver.java     | 146 ++++++++++++
 .../spi/GridTcpSpiForwardingSelfTest.java       | 237 +++++++++++++++++--
 2 files changed, 364 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[2/5] ignite git commit: IGNITE-3359 .NET: Added IgniteConfiguration.ToXml() method. This closes #918.

Posted by ak...@apache.org.
IGNITE-3359 .NET: Added IgniteConfiguration.ToXml() method. This closes #918.


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

Branch: refs/heads/ignite-3443
Commit: e087152cb86375c3f75d8dbf5f77f974bb3adb45
Parents: c421614
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Aug 4 09:12:14 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 4 09:12:14 2016 +0300

----------------------------------------------------------------------
 .../IgniteConfigurationSerializerTest.cs        | 145 +++++++++++++++++--
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  65 +++++++++
 2 files changed, 195 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e087152c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 73e0047..b9e0236 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Tests
     using System;
     using System.Collections;
     using System.Collections.Generic;
+    using System.Configuration;
     using System.Globalization;
     using System.IO;
     using System.Linq;
@@ -111,9 +112,8 @@ namespace Apache.Ignite.Core.Tests
                             <atomicConfiguration backups='2' cacheMode='Local' atomicSequenceReserveSize='250' />
                             <transactionConfiguration defaultTransactionConcurrency='Optimistic' defaultTransactionIsolation='RepeatableRead' defaultTimeout='0:1:2' pessimisticTransactionLogSize='15' pessimisticTransactionLogLinger='0:0:33' />
                         </igniteConfig>";
-            var reader = XmlReader.Create(new StringReader(xml));
 
-            var cfg = IgniteConfigurationXmlSerializer.Deserialize(reader);
+            var cfg = IgniteConfiguration.FromXml(xml);
 
             Assert.AreEqual("c:", cfg.WorkDirectory);
             Assert.AreEqual("127.1.1.1", cfg.Localhost);
@@ -279,6 +279,132 @@ namespace Apache.Ignite.Core.Tests
         }
 
         /// <summary>
+        /// Tests the XML conversion.
+        /// </summary>
+        [Test]
+        public void TestToXml()
+        {
+            // Empty config
+            Assert.AreEqual("<?xml version=\"1.0\" encoding=\"utf-16\"?>\r\n<igniteConfiguration " +
+                            "xmlns=\"http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection\" />",
+                new IgniteConfiguration().ToXml());
+
+            // Some properties
+            var cfg = new IgniteConfiguration
+            {
+                GridName = "myGrid",
+                ClientMode = true,
+                CacheConfiguration = new[]
+                {
+                    new CacheConfiguration("myCache")
+                    {
+                        CacheMode = CacheMode.Replicated,
+                        QueryEntities = new[]
+                        {
+                            new QueryEntity(typeof(int)),
+                            new QueryEntity(typeof(int), typeof(string))
+                        }
+                    }
+                },
+                IncludedEventTypes = new[]
+                {
+                    EventType.CacheEntryCreated,
+                    EventType.CacheNodesLeft
+                }
+            };
+
+            Assert.AreEqual(FixLineEndings(@"<?xml version=""1.0"" encoding=""utf-16""?>
+<igniteConfiguration gridName=""myGrid"" clientMode=""true"" xmlns=""http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"">
+  <cacheConfiguration>
+    <cacheConfiguration name=""myCache"" cacheMode=""Replicated"">
+      <queryEntities>
+        <queryEntity valueTypeName=""java.lang.Integer"" valueType=""System.Int32"" />
+        <queryEntity keyTypeName=""java.lang.Integer"" keyType=""System.Int32"" valueTypeName=""java.lang.String"" valueType=""System.String"" />
+      </queryEntities>
+    </cacheConfiguration>
+  </cacheConfiguration>
+  <includedEventTypes>
+    <int>CacheEntryCreated</int>
+    <int>CacheNodesLeft</int>
+  </includedEventTypes>
+</igniteConfiguration>"), cfg.ToXml());
+
+            // Custom section name and indent
+            var sb = new StringBuilder();
+
+            var settings = new XmlWriterSettings
+            {
+                Indent = true,
+                IndentChars = " "
+            };
+
+            using (var xmlWriter = XmlWriter.Create(sb, settings))
+            {
+                cfg.ToXml(xmlWriter, "igCfg");
+            }
+
+            Assert.AreEqual(FixLineEndings(@"<?xml version=""1.0"" encoding=""utf-16""?>
+<igCfg gridName=""myGrid"" clientMode=""true"" xmlns=""http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"">
+ <cacheConfiguration>
+  <cacheConfiguration name=""myCache"" cacheMode=""Replicated"">
+   <queryEntities>
+    <queryEntity valueTypeName=""java.lang.Integer"" valueType=""System.Int32"" />
+    <queryEntity keyTypeName=""java.lang.Integer"" keyType=""System.Int32"" valueTypeName=""java.lang.String"" valueType=""System.String"" />
+   </queryEntities>
+  </cacheConfiguration>
+ </cacheConfiguration>
+ <includedEventTypes>
+  <int>CacheEntryCreated</int>
+  <int>CacheNodesLeft</int>
+ </includedEventTypes>
+</igCfg>"), sb.ToString());
+        }
+
+        /// <summary>
+        /// Tests the deserialization.
+        /// </summary>
+        [Test]
+        public void TestFromXml()
+        {
+            // Empty section.
+            var cfg = IgniteConfiguration.FromXml("<x />");
+            AssertReflectionEqual(new IgniteConfiguration(), cfg);
+
+            // Empty section with XML header.
+            cfg = IgniteConfiguration.FromXml("<?xml version=\"1.0\" encoding=\"utf-16\"?><x />");
+            AssertReflectionEqual(new IgniteConfiguration(), cfg);
+
+            // Simple test.
+            cfg = IgniteConfiguration.FromXml(@"<igCfg gridName=""myGrid"" clientMode=""true"" />");
+            AssertReflectionEqual(new IgniteConfiguration {GridName = "myGrid", ClientMode = true}, cfg);
+
+            // Invalid xml.
+            var ex = Assert.Throws<ConfigurationErrorsException>(() =>
+                IgniteConfiguration.FromXml(@"<igCfg foo=""bar"" />"));
+
+            Assert.AreEqual("Invalid IgniteConfiguration attribute 'foo=bar', there is no such property " +
+                            "on 'Apache.Ignite.Core.IgniteConfiguration'", ex.Message);
+
+            // Xml reader.
+            using (var xmlReader = XmlReader.Create(
+                new StringReader(@"<igCfg gridName=""myGrid"" clientMode=""true"" />")))
+            {
+                cfg = IgniteConfiguration.FromXml(xmlReader);
+            }
+            AssertReflectionEqual(new IgniteConfiguration { GridName = "myGrid", ClientMode = true }, cfg);
+        }
+
+        /// <summary>
+        /// Ensures windows-style \r\n line endings in a string literal.
+        /// Git settings may cause string literals in both styles.
+        /// </summary>
+        private static string FixLineEndings(string s)
+        {
+            return s.Split('\n').Select(x => x.TrimEnd('\r'))
+                .Aggregate((acc, x) => string.Format("{0}\r\n{1}", acc, x));
+        }
+
+        /// <summary>
         /// Checks the schema validation.
         /// </summary>
         private static void CheckSchemaValidation()
@@ -325,20 +451,9 @@ namespace Apache.Ignite.Core.Tests
         /// </summary>
         private static IgniteConfiguration SerializeDeserialize(IgniteConfiguration cfg)
         {
-            var sb = new StringBuilder();
+            var xml = cfg.ToXml();
 
-            using (var xmlWriter = XmlWriter.Create(sb))
-            {
-                IgniteConfigurationXmlSerializer.Serialize(cfg, xmlWriter, "igniteConfig");
-            }
-
-            var xml = sb.ToString();
-
-            using (var xmlReader = XmlReader.Create(new StringReader(xml)))
-            {
-                xmlReader.MoveToContent();
-                return IgniteConfigurationXmlSerializer.Deserialize(xmlReader);
-            }
+            return IgniteConfiguration.FromXml(xml);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e087152c/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 6bdf1ab..e033ce0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -24,6 +24,8 @@
     using System.Diagnostics.CodeAnalysis;
     using System.IO;
     using System.Linq;
+    using System.Text;
+    using System.Xml;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Configuration;
@@ -678,5 +680,68 @@
             get { return _isLateAffinityAssignment ?? DefaultIsLateAffinityAssignment; }
             set { _isLateAffinityAssignment = value; }
         }
+
+        /// <summary>
+        /// Serializes this instance to the specified XML writer.
+        /// </summary>
+        /// <param name="writer">The writer.</param>
+        /// <param name="rootElementName">Name of the root element.</param>
+        public void ToXml(XmlWriter writer, string rootElementName)
+        {
+            IgniteArgumentCheck.NotNull(writer, "writer");
+            IgniteArgumentCheck.NotNullOrEmpty(rootElementName, "rootElementName");
+
+            IgniteConfigurationXmlSerializer.Serialize(this, writer, rootElementName);
+        }
+
+        /// <summary>
+        /// Serializes this instance to an XML string.
+        /// </summary>
+        public string ToXml()
+        {
+            var sb = new StringBuilder();
+
+            var settings = new XmlWriterSettings
+            {
+                Indent = true
+            };
+
+            using (var xmlWriter = XmlWriter.Create(sb, settings))
+            {
+                ToXml(xmlWriter, "igniteConfiguration");
+            }
+
+            return sb.ToString();
+        }
+
+        /// <summary>
+        /// Deserializes IgniteConfiguration from the XML reader.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        /// <returns>Deserialized instance.</returns>
+        public static IgniteConfiguration FromXml(XmlReader reader)
+        {
+            IgniteArgumentCheck.NotNull(reader, "reader");
+
+            return IgniteConfigurationXmlSerializer.Deserialize(reader);
+        }
+
+        /// <summary>
+        /// Deserializes IgniteConfiguration from the XML string.
+        /// </summary>
+        /// <param name="xml">Xml string.</param>
+        /// <returns>Deserialized instance.</returns>
+        public static IgniteConfiguration FromXml(string xml)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(xml, "xml");
+
+            using (var xmlReader = XmlReader.Create(new StringReader(xml)))
+            {
+                // Skip XML header.
+                xmlReader.MoveToContent();
+
+                return FromXml(xmlReader);
+            }
+        }
     }
 }