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

[38/50] incubator-ignite git commit: ignite-989 Fix multicast for client discovery

ignite-989 Fix multicast for client discovery


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

Branch: refs/heads/ignite-gg-9621
Commit: 9ad3617d39f02eb98ef01f332e2aac20f97870c2
Parents: 97d0bc1
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 3 12:06:51 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 3 12:11:26 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   3 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  31 -----
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  56 +++++++-
 .../tcp/ipfinder/TcpDiscoveryIpFinder.java      |  10 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  47 +++++--
 .../tcp/TcpClientDiscoverySpiMulticastTest.java | 129 +++++++++++++++++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |   1 +
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |  43 ++++---
 8 files changed, 244 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ad3617d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index e672d64..6a5b644 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -166,6 +166,9 @@ class ClientImpl extends TcpDiscoveryImpl {
         msgWorker = new MessageWorker();
         msgWorker.start();
 
+        if (spi.ipFinder.isShared())
+            registerLocalNodeAddress();
+
         try {
             joinLatch.await();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ad3617d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 57c13d6..698835f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -272,37 +272,6 @@ class ServerImpl extends TcpDiscoveryImpl {
         spi.printStartInfo();
     }
 
-    /**
-     * @throws IgniteSpiException If failed.
-     */
-    @SuppressWarnings("BusyWait")
-    private void registerLocalNodeAddress() throws IgniteSpiException {
-        // Make sure address registration succeeded.
-        while (true) {
-            try {
-                spi.ipFinder.initializeLocalAddresses(locNode.socketAddresses());
-
-                // Success.
-                break;
-            }
-            catch (IllegalStateException e) {
-                throw new IgniteSpiException("Failed to register local node address with IP finder: " +
-                    locNode.socketAddresses(), e);
-            }
-            catch (IgniteSpiException e) {
-                LT.error(log, e, "Failed to register local node address in IP finder on start " +
-                    "(retrying every 2000 ms).");
-            }
-
-            try {
-                U.sleep(2000);
-            }
-            catch (IgniteInterruptedCheckedException e) {
-                throw new IgniteSpiException("Thread has been interrupted.", e);
-            }
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
         spiCtx.registerPort(tcpSrvr.port, TCP);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ad3617d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index f285279..b7e9e53 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -19,6 +19,8 @@ package org.apache.ignite.spi.discovery.tcp;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.discovery.tcp.internal.*;
@@ -58,7 +60,7 @@ abstract class TcpDiscoveryImpl {
     }
 
     /**
-     *
+     * @return Local node ID.
      */
     public UUID getLocalNodeId() {
         return spi.getLocalNodeId();
@@ -78,42 +80,47 @@ abstract class TcpDiscoveryImpl {
     public abstract void dumpDebugInfo(IgniteLogger log);
 
     /**
-     *
+     * @return SPI state string.
      */
     public abstract String getSpiState();
 
     /**
-     *
+     * @return Message worker queue current size.
      */
     public abstract int getMessageWorkerQueueSize();
 
     /**
-     *
+     * @return Coordinator ID.
      */
     public abstract UUID getCoordinator();
 
     /**
-     *
+     * @return Collection of remote nodes.
      */
     public abstract Collection<ClusterNode> getRemoteNodes();
 
     /**
      * @param nodeId Node id.
+     * @return Node with given ID or {@code null} if node is not found.
      */
     @Nullable public abstract ClusterNode getNode(UUID nodeId);
 
     /**
      * @param nodeId Node id.
+     * @return {@code true} if node alive, {@code false} otherwise.
      */
     public abstract boolean pingNode(UUID nodeId);
 
     /**
+     * Tells discovery SPI to disconnect from topology.
      *
+     * @throws IgniteSpiException If failed.
      */
     public abstract void disconnect() throws IgniteSpiException;
 
     /**
      * @param msg Message.
+     * @throws IgniteException If failed.
      */
     public abstract void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException;
 
@@ -124,16 +131,18 @@ abstract class TcpDiscoveryImpl {
 
     /**
      * @param gridName Grid name.
+     * @throws IgniteSpiException If failed.
      */
     public abstract void spiStart(@Nullable String gridName) throws IgniteSpiException;
 
     /**
-     *
+     * @throws IgniteSpiException If failed.
      */
     public abstract void spiStop() throws IgniteSpiException;
 
     /**
      * @param spiCtx Spi context.
+     * @throws IgniteSpiException If failed.
      */
     public abstract void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException;
 
@@ -164,7 +173,40 @@ abstract class TcpDiscoveryImpl {
     public abstract void brakeConnection();
 
     /**
-     * FOR TEST PURPOSE ONLY!
+     * <strong>FOR TEST ONLY!!!</strong>
+     *
+     * @return Worker thread.
      */
     protected abstract IgniteSpiThread workerThread();
+
+    /**
+     * @throws IgniteSpiException If failed.
+     */
+    @SuppressWarnings("BusyWait")
+    protected final void registerLocalNodeAddress() throws IgniteSpiException {
+        // Make sure address registration succeeded.
+        while (true) {
+            try {
+                spi.ipFinder.initializeLocalAddresses(locNode.socketAddresses());
+
+                // Success.
+                break;
+            }
+            catch (IllegalStateException e) {
+                throw new IgniteSpiException("Failed to register local node address with IP finder: " +
+                    locNode.socketAddresses(), e);
+            }
+            catch (IgniteSpiException e) {
+                LT.error(log, e, "Failed to register local node address in IP finder on start " +
+                    "(retrying every 2000 ms).");
+            }
+
+            try {
+                U.sleep(2000);
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                throw new IgniteSpiException("Thread has been interrupted.", e);
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ad3617d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java
index 51ad7b4..95758e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java
@@ -31,7 +31,7 @@ public interface TcpDiscoveryIpFinder {
      * method is completed, SPI context can be stored for future access.
      *
      * @param spiCtx Spi context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public void onSpiContextInitialized(IgniteSpiContext spiCtx) throws IgniteSpiException;
 
@@ -46,7 +46,7 @@ public interface TcpDiscoveryIpFinder {
      * Initializes addresses discovery SPI binds to.
      *
      * @param addrs Addresses discovery SPI binds to.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public void initializeLocalAddresses(Collection<InetSocketAddress> addrs) throws IgniteSpiException;
 
@@ -54,7 +54,7 @@ public interface TcpDiscoveryIpFinder {
      * Gets all addresses registered in this finder.
      *
      * @return All known addresses, potentially empty, but never {@code null}.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public Collection<InetSocketAddress> getRegisteredAddresses() throws IgniteSpiException;
 
@@ -76,7 +76,7 @@ public interface TcpDiscoveryIpFinder {
      * is already registered.
      *
      * @param addrs Addresses to register. Not {@code null} and not empty.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public void registerAddresses(Collection<InetSocketAddress> addrs) throws IgniteSpiException;
 
@@ -87,7 +87,7 @@ public interface TcpDiscoveryIpFinder {
      * registered quietly (just no-op).
      *
      * @param addrs Addresses to unregister. Not {@code null} and not empty.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public void unregisterAddresses(Collection<InetSocketAddress> addrs) throws IgniteSpiException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ad3617d/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
index 45d0816..a992620 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
@@ -26,6 +26,8 @@ import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.jetbrains.annotations.*;
 
@@ -254,6 +256,20 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
                 "(it is recommended in production to specify at least one address in " +
                 "TcpDiscoveryMulticastIpFinder.getAddresses() configuration property)");
 
+        boolean clientMode;
+
+        if (ignite != null) { // Can be null if used in tests without starting Ignite.
+            DiscoverySpi discoSpi = ignite.configuration().getDiscoverySpi();
+
+            if (!(discoSpi instanceof TcpDiscoverySpi))
+                throw new IgniteSpiException("TcpDiscoveryMulticastIpFinder should be used with " +
+                    "TcpDiscoverySpi: " + discoSpi);
+
+            clientMode = ((TcpDiscoverySpi)discoSpi).isClientMode();
+        }
+        else
+            clientMode = false;
+
         InetAddress mcastAddr;
 
         try {
@@ -296,7 +312,8 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
 
             if (!addr.isLoopbackAddress()) {
                 try {
-                    addrSnds.add(new AddressSender(mcastAddr, addr, addrs));
+                    if (!clientMode)
+                        addrSnds.add(new AddressSender(mcastAddr, addr, addrs));
 
                     reqItfs.add(addr);
                 }
@@ -309,20 +326,24 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
             }
         }
 
-        if (addrSnds.isEmpty()) {
-            try {
-                // Create non-bound socket if local host is loopback or failed to create sockets explicitly
-                // bound to interfaces.
-                addrSnds.add(new AddressSender(mcastAddr, null, addrs));
-            }
-            catch (IOException e) {
-                throw new IgniteSpiException("Failed to create multicast socket [mcastAddr=" + mcastAddr +
-                    ", mcastGrp=" + mcastGrp + ", mcastPort=" + mcastPort + ']', e);
+        if (!clientMode) {
+            if (addrSnds.isEmpty()) {
+                try {
+                    // Create non-bound socket if local host is loopback or failed to create sockets explicitly
+                    // bound to interfaces.
+                    addrSnds.add(new AddressSender(mcastAddr, null, addrs));
+                }
+                catch (IOException e) {
+                    throw new IgniteSpiException("Failed to create multicast socket [mcastAddr=" + mcastAddr +
+                        ", mcastGrp=" + mcastGrp + ", mcastPort=" + mcastPort + ']', e);
+                }
             }
-        }
 
-        for (AddressSender addrSnd :addrSnds)
-            addrSnd.start();
+            for (AddressSender addrSnd : addrSnds)
+                addrSnd.start();
+        }
+        else
+            assert addrSnds.isEmpty() : addrSnds;
 
         Collection<InetSocketAddress> ret;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ad3617d/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java
new file mode 100644
index 0000000..d1b6232
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.discovery.tcp;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class TcpClientDiscoverySpiMulticastTest extends GridCommonAbstractTest {
+    /** */
+    private boolean forceSrv;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setLocalHost("127.0.0.1");
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(new TcpDiscoveryMulticastIpFinder());
+
+        if (getTestGridName(1).equals(gridName)) {
+            cfg.setClientMode(true);
+
+            spi.setForceServerMode(forceSrv);
+        }
+
+        cfg.setDiscoverySpi(spi);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWithMulticast() throws Exception {
+        joinWithMulticast();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWithMulticastForceServer() throws Exception {
+        forceSrv = true;
+
+        joinWithMulticast();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void joinWithMulticast() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        assertSpi(ignite0, false);
+
+        Ignite ignite1 = startGrid(1);
+
+        assertSpi(ignite1, !forceSrv);
+
+        assertTrue(ignite1.configuration().isClientMode());
+
+        assertEquals(2, ignite0.cluster().nodes().size());
+        assertEquals(2, ignite1.cluster().nodes().size());
+
+        Ignite ignite2 = startGrid(2);
+
+        assertSpi(ignite2, false);
+
+        assertEquals(3, ignite0.cluster().nodes().size());
+        assertEquals(3, ignite1.cluster().nodes().size());
+        assertEquals(3, ignite2.cluster().nodes().size());
+    }
+
+    /**
+     * @param ignite Ignite.
+     * @param client Expected client mode flag.
+     */
+    private void assertSpi(Ignite ignite, boolean client) {
+        DiscoverySpi spi = ignite.configuration().getDiscoverySpi();
+
+        assertSame(TcpDiscoverySpi.class, spi.getClass());
+
+        TcpDiscoverySpi spi0 = (TcpDiscoverySpi)spi;
+
+        assertSame(TcpDiscoveryMulticastIpFinder.class, spi0.getIpFinder().getClass());
+
+        assertEquals(client, spi0.isClientMode());
+
+        Collection<Object> addrSnds = GridTestUtils.getFieldValue(spi0.getIpFinder(), "addrSnds");
+
+        assertNotNull(addrSnds);
+
+        if (client)
+            assertTrue(addrSnds.isEmpty()); // Check client does not send its address.
+        else
+            assertFalse(addrSnds.isEmpty());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ad3617d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
index dc35b24..ea5a7ac 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
@@ -53,6 +53,7 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(TcpClientDiscoverySpiSelfTest.class));
         suite.addTest(new TestSuite(TcpClientDiscoveryMarshallerCheckSelfTest.class));
+        suite.addTest(new TestSuite(TcpClientDiscoverySpiMulticastTest.class));
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9ad3617d/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java b/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java
index b496f60..48991e8 100644
--- a/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java
+++ b/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java
@@ -68,34 +68,37 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
  * Note that this finder is shared by default (see {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder#isShared()}.
  */
 public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapter {
-    /* Default object's content. */
+    /** Default object's content. */
     private final static ByteArrayInputStream OBJECT_CONTENT =  new ByteArrayInputStream(new byte[0]);
 
     /** Grid logger. */
     @LoggerResource
     private IgniteLogger log;
 
-    /* Google Cloud Platform's project name.*/
+    /** Google Cloud Platform's project name.*/
     private String projectName;
 
-    /* Google Storage bucket name. */
+    /** Google Storage bucket name. */
     private String bucketName;
 
-    /* Service account p12 private key file name. */
-    private String serviceAccountP12FilePath;
+    /** Service account p12 private key file name. */
+    private String srvcAccountP12FilePath;
 
-    /* Service account id. */
-    private String serviceAccountId;
+    /** Service account id. */
+    private String srvcAccountId;
 
-    /* Google storage. */
+    /** Google storage. */
     private Storage storage;
 
-    /* Init routine guard. */
+    /** Init routine guard. */
     private final AtomicBoolean initGuard = new AtomicBoolean();
 
-    /* Init routine latch. */
+    /** Init routine latch. */
     private final CountDownLatch initLatch = new CountDownLatch(1);
 
+    /**
+     *
+     */
     public TcpDiscoveryGoogleStorageIpFinder() {
         setShared(true);
     }
@@ -221,7 +224,7 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
      */
     @IgniteSpiConfiguration(optional = false)
     public void setServiceAccountP12FilePath(String p12FileName) {
-        this.serviceAccountP12FilePath = p12FileName;
+        this.srvcAccountP12FilePath = p12FileName;
     }
 
     /**
@@ -235,7 +238,7 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
      */
     @IgniteSpiConfiguration(optional = false)
     public void setServiceAccountId(String id) {
-        this.serviceAccountId = id;
+        this.srvcAccountId = id;
     }
 
     /**
@@ -245,13 +248,13 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
      */
     private void init() throws IgniteSpiException {
         if (initGuard.compareAndSet(false, true)) {
-            if (serviceAccountId == null ||
-                serviceAccountP12FilePath == null ||
+            if (srvcAccountId == null ||
+                srvcAccountP12FilePath == null ||
                 projectName == null ||
                 bucketName == null) {
                 throw new IgniteSpiException(
                     "One or more of the required parameters is not set [serviceAccountId=" +
-                        serviceAccountId + ", serviceAccountP12FilePath=" + serviceAccountP12FilePath + ", projectName=" +
+                        srvcAccountId + ", serviceAccountP12FilePath=" + srvcAccountP12FilePath + ", projectName=" +
                         projectName + ", bucketName=" + bucketName + "]");
             }
 
@@ -265,12 +268,12 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
                     throw new IgniteSpiException(e);
                 }
 
-                GoogleCredential credential;
+                GoogleCredential cred;
 
                 try {
-                    credential = new GoogleCredential.Builder().setTransport(httpTransport)
-                        .setJsonFactory(JacksonFactory.getDefaultInstance()).setServiceAccountId(serviceAccountId)
-                        .setServiceAccountPrivateKeyFromP12File(new File(serviceAccountP12FilePath))
+                    cred = new GoogleCredential.Builder().setTransport(httpTransport)
+                        .setJsonFactory(JacksonFactory.getDefaultInstance()).setServiceAccountId(srvcAccountId)
+                        .setServiceAccountPrivateKeyFromP12File(new File(srvcAccountP12FilePath))
                         .setServiceAccountScopes(Collections.singleton(StorageScopes.DEVSTORAGE_FULL_CONTROL)).build();
 
                 }
@@ -279,7 +282,7 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
                 }
 
                 try {
-                    storage = new Storage.Builder(httpTransport, JacksonFactory.getDefaultInstance(), credential)
+                    storage = new Storage.Builder(httpTransport, JacksonFactory.getDefaultInstance(), cred)
                         .setApplicationName(projectName).build();
                 }
                 catch (Exception e) {