You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/01/13 19:00:57 UTC

ignite git commit: ignite-2350 Pass update notifier flag in discovery data (all cluster nodes will have the same notifier status as first cluster node) (cherry picked from commit 7175a42)

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.5.4 02dbcfd8e -> 86c2ba2a6


ignite-2350 Pass update notifier flag in discovery data (all cluster nodes will have the same notifier status as first cluster node)
(cherry picked from commit 7175a42)


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

Branch: refs/heads/ignite-1.5.4
Commit: 86c2ba2a601e82b824cf17422683e5398a4d8c7d
Parents: 02dbcfd
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jan 13 18:40:08 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jan 13 20:58:12 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/GridComponent.java   |   5 +-
 .../ignite/internal/GridUpdateNotifier.java     | 454 ------------------
 .../apache/ignite/internal/IgniteKernal.java    | 126 +----
 .../discovery/GridDiscoveryManager.java         |   6 +-
 .../processors/cluster/ClusterProcessor.java    | 174 +++++++
 .../processors/cluster/GridUpdateNotifier.java  | 457 +++++++++++++++++++
 .../internal/GridUpdateNotifierSelfTest.java    | 137 ------
 ...UpdateNotifierPerClusterSettingSelfTest.java | 130 ++++++
 .../cluster/GridUpdateNotifierSelfTest.java     | 140 ++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   4 +-
 10 files changed, 920 insertions(+), 713 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
index 0e234cd..5c77aee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
@@ -40,7 +40,10 @@ public interface GridComponent {
         CACHE_PROC,
 
         /** */
-        PLUGIN
+        PLUGIN,
+
+        /** */
+        CLUSTER_PROC
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java b/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
deleted file mode 100644
index 5d2cf35..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
+++ /dev/null
@@ -1,454 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal;
-
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.PrintWriter;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.io.UnsupportedEncodingException;
-import java.net.URL;
-import java.net.URLConnection;
-import java.util.Collection;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.atomic.AtomicReference;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.internal.util.worker.GridWorker;
-import org.apache.ignite.plugin.PluginProvider;
-import org.jetbrains.annotations.Nullable;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.EntityResolver;
-import org.xml.sax.InputSource;
-
-import static java.net.URLEncoder.encode;
-
-/**
- * This class is responsible for notification about new version availability.
- * <p>
- * Note also that this connectivity is not necessary to successfully start the system as it will
- * gracefully ignore any errors occurred during notification and verification process.
- */
-class GridUpdateNotifier {
-    /** Default encoding. */
-    private static final String CHARSET = "UTF-8";
-
-    /** Access URL to be used to access latest version data. */
-    private static final String UPD_STATUS_PARAMS = IgniteProperties.get("ignite.update.status.params");
-
-    /** Throttling for logging out. */
-    private static final long THROTTLE_PERIOD = 24 * 60 * 60 * 1000; // 1 day.
-
-    /** Sleep milliseconds time for worker thread. */
-    public static final int WORKER_THREAD_SLEEP_TIME = 5000;
-
-    /** Grid version. */
-    private final String ver;
-
-    /** Site. */
-    private final String url;
-
-    /** Latest version. */
-    private volatile String latestVer;
-
-    /** Download url for latest version. */
-    private volatile String downloadUrl;
-
-    /** HTML parsing helper. */
-    private final DocumentBuilder documentBuilder;
-
-    /** Grid name. */
-    private final String gridName;
-
-    /** Whether or not to report only new version. */
-    private volatile boolean reportOnlyNew;
-
-    /** */
-    private volatile int topSize;
-
-    /** System properties */
-    private final String vmProps;
-
-    /** Plugins information for request */
-    private final String pluginsVers;
-
-    /** Kernal gateway */
-    private final GridKernalGateway gw;
-
-    /** */
-    private long lastLog = -1;
-
-    /** Command for worker thread. */
-    private final AtomicReference<Runnable> cmd = new AtomicReference<>();
-
-    /** Worker thread to process http request. */
-    private final Thread workerThread;
-
-    /**
-     * Creates new notifier with default values.
-     *
-     * @param gridName gridName
-     * @param ver Compound Ignite version.
-     * @param gw Kernal gateway.
-     * @param pluginProviders Kernal gateway.
-     * @param reportOnlyNew Whether or not to report only new version.
-     * @throws IgniteCheckedException If failed.
-     */
-    GridUpdateNotifier(String gridName, String ver, GridKernalGateway gw, Collection<PluginProvider> pluginProviders,
-        boolean reportOnlyNew) throws IgniteCheckedException {
-        try {
-            DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
-
-            documentBuilder = factory.newDocumentBuilder();
-
-            documentBuilder.setEntityResolver(new EntityResolver() {
-                @Override public InputSource resolveEntity(String publicId, String sysId) {
-                    if (sysId.endsWith(".dtd"))
-                        return new InputSource(new StringReader(""));
-
-                    return null;
-                }
-            });
-
-            this.ver = ver;
-
-            url = "http://ignite.run/update_status_ignite.php";
-
-            this.gridName = gridName == null ? "null" : gridName;
-            this.gw = gw;
-
-            SB pluginsBuilder = new SB();
-
-            for (PluginProvider provider : pluginProviders)
-                pluginsBuilder.a("&").a(provider.name() + "-plugin-version").a("=").
-                    a(encode(provider.version(), CHARSET));
-
-            pluginsVers = pluginsBuilder.toString();
-
-            this.reportOnlyNew = reportOnlyNew;
-
-            vmProps = getSystemProperties();
-
-            workerThread = new Thread(new Runnable() {
-                @Override public void run() {
-                    try {
-                        while(!Thread.currentThread().isInterrupted()) {
-                            Runnable cmd0 = cmd.getAndSet(null);
-
-                            if (cmd0 != null)
-                                cmd0.run();
-                            else
-                                Thread.sleep(WORKER_THREAD_SLEEP_TIME);
-                        }
-                    }
-                    catch (InterruptedException ignore) {
-                        // No-op.
-                    }
-                }
-            }, "upd-ver-checker");
-
-            workerThread.setDaemon(true);
-
-            workerThread.start();
-        }
-        catch (ParserConfigurationException e) {
-            throw new IgniteCheckedException("Failed to create xml parser.", e);
-        }
-        catch (UnsupportedEncodingException e) {
-            throw new IgniteCheckedException("Failed to encode.", e);
-        }
-    }
-
-    /**
-     * Gets system properties.
-     *
-     * @return System properties.
-     */
-    private static String getSystemProperties() {
-        try {
-            StringWriter sw = new StringWriter();
-
-            try {
-                System.getProperties().store(new PrintWriter(sw), "");
-            }
-            catch (IOException ignore) {
-                return null;
-            }
-
-            return sw.toString();
-        }
-        catch (SecurityException ignore) {
-            return null;
-        }
-    }
-
-    /**
-     * @param reportOnlyNew Whether or not to report only new version.
-     */
-    void reportOnlyNew(boolean reportOnlyNew) {
-        this.reportOnlyNew = reportOnlyNew;
-    }
-
-    /**
-     * @param topSize Size of topology for license verification purpose.
-     */
-    void topologySize(int topSize) {
-        this.topSize = topSize;
-    }
-
-    /**
-     * @return Latest version.
-     */
-    String latestVersion() {
-        return latestVer;
-    }
-
-    /**
-     * Starts asynchronous process for retrieving latest version data.
-     *
-     * @param log Logger.
-     */
-    void checkForNewVersion(IgniteLogger log) {
-        assert log != null;
-
-        log = log.getLogger(getClass());
-
-        try {
-            cmd.set(new UpdateChecker(log));
-        }
-        catch (RejectedExecutionException e) {
-            U.error(log, "Failed to schedule a thread due to execution rejection (safely ignoring): " +
-                e.getMessage());
-        }
-    }
-
-    /**
-     * Logs out latest version notification if such was received and available.
-     *
-     * @param log Logger.
-     */
-    void reportStatus(IgniteLogger log) {
-        assert log != null;
-
-        log = log.getLogger(getClass());
-
-        String latestVer = this.latestVer;
-        String downloadUrl = this.downloadUrl;
-
-        downloadUrl = downloadUrl != null ? downloadUrl : IgniteKernal.SITE;
-
-        if (latestVer != null)
-            if (latestVer.equals(ver)) {
-                if (!reportOnlyNew)
-                    throttle(log, false, "Your version is up to date.");
-            }
-            else
-                throttle(log, true, "New version is available at " + downloadUrl + ": " + latestVer);
-        else
-            if (!reportOnlyNew)
-                throttle(log, false, "Update status is not available.");
-    }
-
-    /**
-     *
-     * @param log Logger to use.
-     * @param warn Whether or not this is a warning.
-     * @param msg Message to log.
-     */
-    private void throttle(IgniteLogger log, boolean warn, String msg) {
-        assert(log != null);
-        assert(msg != null);
-
-        long now = U.currentTimeMillis();
-
-        if (now - lastLog > THROTTLE_PERIOD) {
-            if (!warn)
-                U.log(log, msg);
-            else {
-                U.quiet(true, msg);
-
-                if (log.isInfoEnabled())
-                    log.warning(msg);
-            }
-
-            lastLog = now;
-        }
-    }
-
-    /**
-     * Stops update notifier.
-     */
-    public void stop() {
-        workerThread.interrupt();
-    }
-
-    /**
-     * Asynchronous checker of the latest version available.
-     */
-    private class UpdateChecker extends GridWorker {
-        /** Logger. */
-        private final IgniteLogger log;
-
-        /**
-         * Creates checked with given logger.
-         *
-         * @param log Logger.
-         */
-        UpdateChecker(IgniteLogger log) {
-            super(gridName, "grid-version-checker", log);
-
-            this.log = log.getLogger(getClass());
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            try {
-                String stackTrace = gw != null ? gw.userStackTrace() : null;
-
-                String postParams =
-                    "gridName=" + encode(gridName, CHARSET) +
-                    (!F.isEmpty(UPD_STATUS_PARAMS) ? "&" + UPD_STATUS_PARAMS : "") +
-                    (topSize > 0 ? "&topSize=" + topSize : "") +
-                    (!F.isEmpty(stackTrace) ? "&stackTrace=" + encode(stackTrace, CHARSET) : "") +
-                    (!F.isEmpty(vmProps) ? "&vmProps=" + encode(vmProps, CHARSET) : "") +
-                        pluginsVers;
-
-                URLConnection conn = new URL(url).openConnection();
-
-                if (!isCancelled()) {
-                    conn.setDoOutput(true);
-                    conn.setRequestProperty("Accept-Charset", CHARSET);
-                    conn.setRequestProperty("Content-Type", "application/x-www-form-urlencoded;charset=" + CHARSET);
-
-                    conn.setConnectTimeout(3000);
-                    conn.setReadTimeout(3000);
-
-                    Document dom = null;
-
-                    try {
-                        try (OutputStream os = conn.getOutputStream()) {
-                            os.write(postParams.getBytes(CHARSET));
-                        }
-
-                        try (InputStream in = conn.getInputStream()) {
-                            if (in == null)
-                                return;
-
-                            BufferedReader reader = new BufferedReader(new InputStreamReader(in, CHARSET));
-
-                            StringBuilder xml = new StringBuilder();
-
-                            String line;
-
-                            while ((line = reader.readLine()) != null) {
-                                if (line.contains("<meta") && !line.contains("/>"))
-                                    line = line.replace(">", "/>");
-
-                                xml.append(line).append('\n');
-                            }
-
-                            dom = documentBuilder.parse(new ByteArrayInputStream(xml.toString().getBytes(CHARSET)));
-                        }
-                    }
-                    catch (IOException e) {
-                        if (log.isDebugEnabled())
-                            log.debug("Failed to connect to Ignite update server. " + e.getMessage());
-                    }
-
-                    if (dom != null) {
-                        latestVer = obtainVersionFrom(dom);
-
-                        downloadUrl = obtainDownloadUrlFrom(dom);
-                    }
-                }
-            }
-            catch (Exception e) {
-                if (log.isDebugEnabled())
-                    log.debug("Unexpected exception in update checker. " + e.getMessage());
-            }
-        }
-
-        /**
-         * Gets the version from the current {@code node}, if one exists.
-         *
-         * @param node W3C DOM node.
-         * @return Version or {@code null} if one's not found.
-         */
-        @Nullable private String obtainMeta(String metaName, Node node) {
-            assert node != null;
-
-            if (node instanceof Element && "meta".equals(node.getNodeName().toLowerCase())) {
-                Element meta = (Element)node;
-
-                String name = meta.getAttribute("name");
-
-                if (metaName.equals(name)) {
-                    String content = meta.getAttribute("content");
-
-                    if (content != null && !content.isEmpty())
-                        return content;
-                }
-            }
-
-            NodeList childNodes = node.getChildNodes();
-
-            for (int i = 0; i < childNodes.getLength(); i++) {
-                String ver = obtainMeta(metaName, childNodes.item(i));
-
-                if (ver != null)
-                    return ver;
-            }
-
-            return null;
-        }
-
-        /**
-         * Gets the version from the current {@code node}, if one exists.
-         *
-         * @param node W3C DOM node.
-         * @return Version or {@code null} if one's not found.
-         */
-        @Nullable private String obtainVersionFrom(Node node) {
-            return obtainMeta("version", node);
-        }
-
-        /**
-         * Gets the download url from the current {@code node}, if one exists.
-         *
-         * @param node W3C DOM node.
-         * @return download url or {@code null} if one's not found.
-         */
-        @Nullable private String obtainDownloadUrlFrom(Node node) {
-            return obtainMeta("downloadUrl", node);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 14b5816..d011d1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -28,7 +28,6 @@ import java.io.ObjectStreamException;
 import java.io.Serializable;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
-import java.lang.ref.WeakReference;
 import java.lang.reflect.Constructor;
 import java.text.DateFormat;
 import java.text.DecimalFormat;
@@ -41,7 +40,6 @@ import java.util.List;
 import java.util.ListIterator;
 import java.util.Map;
 import java.util.Properties;
-import java.util.Timer;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -83,6 +81,8 @@ import org.apache.ignite.configuration.CollectionConfiguration;
 import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.binary.BinaryEnumCache;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
 import org.apache.ignite.internal.managers.GridManager;
@@ -96,8 +96,6 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager;
 import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.managers.swapspace.GridSwapSpaceManager;
-import org.apache.ignite.internal.binary.BinaryEnumCache;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.processors.GridProcessor;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
@@ -135,7 +133,6 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessor;
 import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
-import org.apache.ignite.internal.util.GridTimerTask;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -178,7 +175,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALL
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_STARVATION_CHECK_INTERVAL;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SUCCESS_FILE;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 import static org.apache.ignite.IgniteSystemProperties.snapshot;
 import static org.apache.ignite.internal.GridKernalState.DISCONNECTED;
@@ -234,17 +230,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     private static final long serialVersionUID = 0L;
 
     /** Ignite site that is shown in log messages. */
-    static final String SITE = "ignite.apache.org";
+    public static final String SITE = "ignite.apache.org";
 
     /** System line separator. */
     private static final String NL = U.nl();
 
-    /** Periodic version check delay. */
-    private static final long PERIODIC_VER_CHECK_DELAY = 1000 * 60 * 60; // Every hour.
-
-    /** Periodic version check delay. */
-    private static final long PERIODIC_VER_CHECK_CONN_TIMEOUT = 10 * 1000; // 10 seconds.
-
     /** Periodic starvation check interval. */
     private static final long PERIODIC_STARVATION_CHECK_FREQ = 1000 * 30;
 
@@ -299,10 +289,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** */
     @GridToStringExclude
-    private Timer updateNtfTimer;
-
-    /** */
-    @GridToStringExclude
     private GridTimeoutProcessor.CancelableTask starveTask;
 
     /** */
@@ -325,10 +311,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     @GridToStringExclude
     private final AtomicBoolean stopGuard = new AtomicBoolean();
 
-    /** Version checker. */
-    @GridToStringExclude
-    private GridUpdateNotifier verChecker;
-
     /**
      * No-arg constructor is required by externalization.
      */
@@ -745,9 +727,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         // Run background network diagnostics.
         GridDiagnostic.runBackgroundCheck(gridName, execSvc, log);
 
-        boolean notifyEnabled = IgniteSystemProperties.getBoolean(IGNITE_UPDATE_NOTIFIER,
-            Boolean.parseBoolean(IgniteProperties.get("ignite.update.notifier.enabled.by.default")));
-
         // Ack 3-rd party licenses location.
         if (log.isInfoEnabled() && cfg.getIgniteHome() != null)
             log.info("3-rd party licenses can be found at: " + cfg.getIgniteHome() + File.separatorChar + "libs" +
@@ -786,9 +765,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             cfg.getMarshaller().setContext(ctx.marshallerContext());
 
-            startProcessor(new ClusterProcessor(ctx));
+            ClusterProcessor clusterProc = new ClusterProcessor(ctx);
 
-            fillNodeAttributes(notifyEnabled);
+            startProcessor(clusterProc);
+
+            fillNodeAttributes(clusterProc.updateNotifierEnabled());
 
             U.onGridStart();
 
@@ -820,24 +801,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             startProcessor(new IgnitePluginProcessor(ctx, cfg, plugins));
 
-            verChecker = null;
-
-            if (notifyEnabled) {
-                try {
-                    verChecker = new GridUpdateNotifier(gridName, VER_STR, gw, ctx.plugins().allProviders(), false);
-
-                    updateNtfTimer = new Timer("ignite-update-notifier-timer", true);
-
-                    // Setup periodic version check.
-                    updateNtfTimer.scheduleAtFixedRate(new UpdateNotifierTimerTask(this, execSvc, verChecker),
-                        0, PERIODIC_VER_CHECK_DELAY);
-                }
-                catch (IgniteCheckedException e) {
-                    if (log.isDebugEnabled())
-                        log.debug("Failed to create GridUpdateNotifier: " + e);
-                }
-            }
-
             // Off-heap processor has no dependencies.
             startProcessor(new GridOffHeapProcessor(ctx));
 
@@ -1857,13 +1820,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 }
             }
 
-            // Cancel update notification timer.
-            if (updateNtfTimer != null)
-                updateNtfTimer.cancel();
-
-            if (verChecker != null)
-                verChecker.stop();
-
             if (starveTask != null)
                 starveTask.close();
 
@@ -2842,7 +2798,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ctx.gateway().readLock();
 
         try {
-            return verChecker != null ? verChecker.latestVersion() : null;
+            return ctx.cluster().latestVersion();
         }
         finally {
             ctx.gateway().readUnlock();
@@ -3281,70 +3237,4 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     @Override public String toString() {
         return S.toString(IgniteKernal.class, this);
     }
-
-    /**
-     * Update notifier timer task.
-     */
-    private static class UpdateNotifierTimerTask extends GridTimerTask {
-        /** Reference to kernal. */
-        private final WeakReference<IgniteKernal> kernalRef;
-
-        /** Logger. */
-        private final IgniteLogger log;
-
-        /** Executor service. */
-        private final ExecutorService execSvc;
-
-        /** Version checker. */
-        private final GridUpdateNotifier verChecker;
-
-        /** Whether this is the first run. */
-        private boolean first = true;
-
-        /**
-         * Constructor.
-         *
-         * @param kernal Kernal.
-         * @param execSvc Executor service.
-         * @param verChecker Version checker.
-         */
-        private UpdateNotifierTimerTask(IgniteKernal kernal, ExecutorService execSvc, GridUpdateNotifier verChecker) {
-            kernalRef = new WeakReference<>(kernal);
-
-            log = kernal.log.getLogger(UpdateNotifierTimerTask.class);
-
-            this.execSvc = execSvc;
-            this.verChecker = verChecker;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void safeRun() throws InterruptedException {
-            if (!first) {
-                IgniteKernal kernal = kernalRef.get();
-
-                if (kernal != null)
-                    verChecker.topologySize(kernal.cluster().nodes().size());
-            }
-
-            verChecker.checkForNewVersion(log);
-
-            // Just wait for 10 secs.
-            Thread.sleep(PERIODIC_VER_CHECK_CONN_TIMEOUT);
-
-            // Just wait another 60 secs in order to get
-            // version info even on slow connection.
-            for (int i = 0; i < 60 && verChecker.latestVersion() == null; i++)
-                Thread.sleep(1000);
-
-            // Report status if one is available.
-            // No-op if status is NOT available.
-            verChecker.reportStatus(log);
-
-            if (first) {
-                first = false;
-
-                verChecker.reportOnlyNew(true);
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 72a2bef..23a85e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -651,8 +651,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                     if (comp != null)
                         comp.onDiscoveryDataReceived(joiningNodeId, nodeId, e.getValue());
-                    else
-                        U.warn(log, "Received discovery data for unknown component: " + e.getKey());
+                    else {
+                        if (log.isDebugEnabled())
+                            log.debug("Received discovery data for unknown component: " + e.getKey());
+                    }
                 }
             }
         });

http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
index a72615c..5e8e98d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
@@ -17,25 +17,65 @@
 
 package org.apache.ignite.internal.processors.cluster;
 
+import java.io.Serializable;
+import java.lang.ref.WeakReference;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Timer;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.IgniteProperties;
 import org.apache.ignite.internal.cluster.IgniteClusterImpl;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.GridTimerTask;
 import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.lang.IgniteFuture;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER;
+import static org.apache.ignite.internal.IgniteVersionUtils.VER_STR;
 
 /**
  *
  */
 public class ClusterProcessor extends GridProcessorAdapter {
     /** */
+    private static final String ATTR_UPDATE_NOTIFIER_STATUS = "UPDATE_NOTIFIER_STATUS";
+
+    /** Periodic version check delay. */
+    private static final long PERIODIC_VER_CHECK_DELAY = 1000 * 60 * 60; // Every hour.
+
+    /** Periodic version check delay. */
+    private static final long PERIODIC_VER_CHECK_CONN_TIMEOUT = 10 * 1000; // 10 seconds.
+
+    /** */
     private IgniteClusterImpl cluster;
 
+    /** */
+    private boolean notifyEnabled;
+
+    /** */
+    @GridToStringExclude
+    private Timer updateNtfTimer;
+
+    /** Version checker. */
+    @GridToStringExclude
+    private GridUpdateNotifier verChecker;
+
     /**
      * @param ctx Kernal context.
      */
     public ClusterProcessor(GridKernalContext ctx) {
         super(ctx);
 
+        notifyEnabled = IgniteSystemProperties.getBoolean(IGNITE_UPDATE_NOTIFIER,
+            Boolean.parseBoolean(IgniteProperties.get("ignite.update.notifier.enabled.by.default")));
+
         cluster = new IgniteClusterImpl(ctx);
     }
 
@@ -54,4 +94,138 @@ public class ClusterProcessor extends GridProcessorAdapter {
 
         return fut != null ? fut : new IgniteFinishedFutureImpl<>();
     }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryDataExchangeType discoveryDataType() {
+        return DiscoveryDataExchangeType.CLUSTER_PROC;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Serializable collectDiscoveryData(UUID nodeId) {
+        HashMap<String, Object> map = new HashMap<>();
+
+        map.put(ATTR_UPDATE_NOTIFIER_STATUS, notifyEnabled);
+
+        return map;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable data) {
+        if (joiningNodeId.equals(ctx.localNodeId())) {
+            Map<String, Object> map = (Map<String, Object>)data;
+
+            if (map != null && map.containsKey(ATTR_UPDATE_NOTIFIER_STATUS))
+                notifyEnabled = (Boolean)map.get(ATTR_UPDATE_NOTIFIER_STATUS);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStart() throws IgniteCheckedException {
+        if (notifyEnabled) {
+            try {
+                verChecker = new GridUpdateNotifier(ctx.gridName(),
+                    VER_STR,
+                    ctx.gateway(),
+                    ctx.plugins().allProviders(),
+                    false);
+
+                updateNtfTimer = new Timer("ignite-update-notifier-timer", true);
+
+                // Setup periodic version check.
+                updateNtfTimer.scheduleAtFixedRate(new UpdateNotifierTimerTask((IgniteKernal)ctx.grid(), verChecker),
+                    0, PERIODIC_VER_CHECK_DELAY);
+            }
+            catch (IgniteCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to create GridUpdateNotifier: " + e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
+        // Cancel update notification timer.
+        if (updateNtfTimer != null)
+            updateNtfTimer.cancel();
+
+        if (verChecker != null)
+            verChecker.stop();
+
+    }
+
+    /**
+     * @return Update notifier status.
+     */
+    public boolean updateNotifierEnabled() {
+        return notifyEnabled;
+    }
+
+    /**
+     * @return Latest version string.
+     */
+    public String latestVersion() {
+        return verChecker != null ? verChecker.latestVersion() : null;
+    }
+
+    /**
+     * Update notifier timer task.
+     */
+    private static class UpdateNotifierTimerTask extends GridTimerTask {
+        /** Reference to kernal. */
+        private final WeakReference<IgniteKernal> kernalRef;
+
+        /** Logger. */
+        private final IgniteLogger log;
+
+        /** Version checker. */
+        private final GridUpdateNotifier verChecker;
+
+        /** Whether this is the first run. */
+        private boolean first = true;
+
+        /**
+         * Constructor.
+         *
+         * @param kernal Kernal.
+         * @param verChecker Version checker.
+         */
+        private UpdateNotifierTimerTask(IgniteKernal kernal, GridUpdateNotifier verChecker) {
+            kernalRef = new WeakReference<>(kernal);
+
+            log = kernal.context().log(UpdateNotifierTimerTask.class);
+
+            this.verChecker = verChecker;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void safeRun() throws InterruptedException {
+            if (!first) {
+                IgniteKernal kernal = kernalRef.get();
+
+                if (kernal != null)
+                    verChecker.topologySize(kernal.cluster().nodes().size());
+            }
+
+            verChecker.checkForNewVersion(log);
+
+            // Just wait for 10 secs.
+            Thread.sleep(PERIODIC_VER_CHECK_CONN_TIMEOUT);
+
+            // Just wait another 60 secs in order to get
+            // version info even on slow connection.
+            for (int i = 0; i < 60 && verChecker.latestVersion() == null; i++)
+                Thread.sleep(1000);
+
+            // Report status if one is available.
+            // No-op if status is NOT available.
+            verChecker.reportStatus(log);
+
+            if (first) {
+                first = false;
+
+                verChecker.reportOnlyNew(true);
+            }
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java
new file mode 100644
index 0000000..2e2f9e4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifier.java
@@ -0,0 +1,457 @@
+/*
+ * 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.cluster;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.io.UnsupportedEncodingException;
+import java.net.URL;
+import java.net.URLConnection;
+import java.util.Collection;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalGateway;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.IgniteProperties;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.plugin.PluginProvider;
+import org.jetbrains.annotations.Nullable;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.EntityResolver;
+import org.xml.sax.InputSource;
+
+import static java.net.URLEncoder.encode;
+
+/**
+ * This class is responsible for notification about new version availability.
+ * <p>
+ * Note also that this connectivity is not necessary to successfully start the system as it will
+ * gracefully ignore any errors occurred during notification and verification process.
+ */
+class GridUpdateNotifier {
+    /** Default encoding. */
+    private static final String CHARSET = "UTF-8";
+
+    /** Access URL to be used to access latest version data. */
+    private final String UPD_STATUS_PARAMS = IgniteProperties.get("ignite.update.status.params");
+
+    /** Throttling for logging out. */
+    private static final long THROTTLE_PERIOD = 24 * 60 * 60 * 1000; // 1 day.
+
+    /** Sleep milliseconds time for worker thread. */
+    public static final int WORKER_THREAD_SLEEP_TIME = 5000;
+
+    /** Grid version. */
+    private final String ver;
+
+    /** Site. */
+    private final String url;
+
+    /** Latest version. */
+    private volatile String latestVer;
+
+    /** Download url for latest version. */
+    private volatile String downloadUrl;
+
+    /** HTML parsing helper. */
+    private final DocumentBuilder documentBuilder;
+
+    /** Grid name. */
+    private final String gridName;
+
+    /** Whether or not to report only new version. */
+    private volatile boolean reportOnlyNew;
+
+    /** */
+    private volatile int topSize;
+
+    /** System properties */
+    private final String vmProps;
+
+    /** Plugins information for request */
+    private final String pluginsVers;
+
+    /** Kernal gateway */
+    private final GridKernalGateway gw;
+
+    /** */
+    private long lastLog = -1;
+
+    /** Command for worker thread. */
+    private final AtomicReference<Runnable> cmd = new AtomicReference<>();
+
+    /** Worker thread to process http request. */
+    private final Thread workerThread;
+
+    /**
+     * Creates new notifier with default values.
+     *
+     * @param gridName gridName
+     * @param ver Compound Ignite version.
+     * @param gw Kernal gateway.
+     * @param pluginProviders Kernal gateway.
+     * @param reportOnlyNew Whether or not to report only new version.
+     * @throws IgniteCheckedException If failed.
+     */
+    GridUpdateNotifier(String gridName, String ver, GridKernalGateway gw, Collection<PluginProvider> pluginProviders,
+        boolean reportOnlyNew) throws IgniteCheckedException {
+        try {
+            DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+
+            documentBuilder = factory.newDocumentBuilder();
+
+            documentBuilder.setEntityResolver(new EntityResolver() {
+                @Override public InputSource resolveEntity(String publicId, String sysId) {
+                    if (sysId.endsWith(".dtd"))
+                        return new InputSource(new StringReader(""));
+
+                    return null;
+                }
+            });
+
+            this.ver = ver;
+
+            url = "http://ignite.run/update_status_ignite.php";
+
+            this.gridName = gridName == null ? "null" : gridName;
+            this.gw = gw;
+
+            SB pluginsBuilder = new SB();
+
+            for (PluginProvider provider : pluginProviders)
+                pluginsBuilder.a("&").a(provider.name() + "-plugin-version").a("=").
+                    a(encode(provider.version(), CHARSET));
+
+            pluginsVers = pluginsBuilder.toString();
+
+            this.reportOnlyNew = reportOnlyNew;
+
+            vmProps = getSystemProperties();
+
+            workerThread = new Thread(new Runnable() {
+                @Override public void run() {
+                    try {
+                        while(!Thread.currentThread().isInterrupted()) {
+                            Runnable cmd0 = cmd.getAndSet(null);
+
+                            if (cmd0 != null)
+                                cmd0.run();
+                            else
+                                Thread.sleep(WORKER_THREAD_SLEEP_TIME);
+                        }
+                    }
+                    catch (InterruptedException ignore) {
+                        // No-op.
+                    }
+                }
+            }, "upd-ver-checker");
+
+            workerThread.setDaemon(true);
+
+            workerThread.start();
+        }
+        catch (ParserConfigurationException e) {
+            throw new IgniteCheckedException("Failed to create xml parser.", e);
+        }
+        catch (UnsupportedEncodingException e) {
+            throw new IgniteCheckedException("Failed to encode.", e);
+        }
+    }
+
+    /**
+     * Gets system properties.
+     *
+     * @return System properties.
+     */
+    private static String getSystemProperties() {
+        try {
+            StringWriter sw = new StringWriter();
+
+            try {
+                System.getProperties().store(new PrintWriter(sw), "");
+            }
+            catch (IOException ignore) {
+                return null;
+            }
+
+            return sw.toString();
+        }
+        catch (SecurityException ignore) {
+            return null;
+        }
+    }
+
+    /**
+     * @param reportOnlyNew Whether or not to report only new version.
+     */
+    void reportOnlyNew(boolean reportOnlyNew) {
+        this.reportOnlyNew = reportOnlyNew;
+    }
+
+    /**
+     * @param topSize Size of topology for license verification purpose.
+     */
+    void topologySize(int topSize) {
+        this.topSize = topSize;
+    }
+
+    /**
+     * @return Latest version.
+     */
+    String latestVersion() {
+        return latestVer;
+    }
+
+    /**
+     * Starts asynchronous process for retrieving latest version data.
+     *
+     * @param log Logger.
+     */
+    void checkForNewVersion(IgniteLogger log) {
+        assert log != null;
+
+        log = log.getLogger(getClass());
+
+        try {
+            cmd.set(new UpdateChecker(log));
+        }
+        catch (RejectedExecutionException e) {
+            U.error(log, "Failed to schedule a thread due to execution rejection (safely ignoring): " +
+                e.getMessage());
+        }
+    }
+
+    /**
+     * Logs out latest version notification if such was received and available.
+     *
+     * @param log Logger.
+     */
+    void reportStatus(IgniteLogger log) {
+        assert log != null;
+
+        log = log.getLogger(getClass());
+
+        String latestVer = this.latestVer;
+        String downloadUrl = this.downloadUrl;
+
+        downloadUrl = downloadUrl != null ? downloadUrl : IgniteKernal.SITE;
+
+        if (latestVer != null)
+            if (latestVer.equals(ver)) {
+                if (!reportOnlyNew)
+                    throttle(log, false, "Your version is up to date.");
+            }
+            else
+                throttle(log, true, "New version is available at " + downloadUrl + ": " + latestVer);
+        else
+            if (!reportOnlyNew)
+                throttle(log, false, "Update status is not available.");
+    }
+
+    /**
+     *
+     * @param log Logger to use.
+     * @param warn Whether or not this is a warning.
+     * @param msg Message to log.
+     */
+    private void throttle(IgniteLogger log, boolean warn, String msg) {
+        assert(log != null);
+        assert(msg != null);
+
+        long now = U.currentTimeMillis();
+
+        if (now - lastLog > THROTTLE_PERIOD) {
+            if (!warn)
+                U.log(log, msg);
+            else {
+                U.quiet(true, msg);
+
+                if (log.isInfoEnabled())
+                    log.warning(msg);
+            }
+
+            lastLog = now;
+        }
+    }
+
+    /**
+     * Stops update notifier.
+     */
+    public void stop() {
+        workerThread.interrupt();
+    }
+
+    /**
+     * Asynchronous checker of the latest version available.
+     */
+    private class UpdateChecker extends GridWorker {
+        /** Logger. */
+        private final IgniteLogger log;
+
+        /**
+         * Creates checked with given logger.
+         *
+         * @param log Logger.
+         */
+        UpdateChecker(IgniteLogger log) {
+            super(gridName, "grid-version-checker", log);
+
+            this.log = log.getLogger(getClass());
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            try {
+                String stackTrace = gw != null ? gw.userStackTrace() : null;
+
+                String postParams =
+                    "gridName=" + encode(gridName, CHARSET) +
+                    (!F.isEmpty(UPD_STATUS_PARAMS) ? "&" + UPD_STATUS_PARAMS : "") +
+                    (topSize > 0 ? "&topSize=" + topSize : "") +
+                    (!F.isEmpty(stackTrace) ? "&stackTrace=" + encode(stackTrace, CHARSET) : "") +
+                    (!F.isEmpty(vmProps) ? "&vmProps=" + encode(vmProps, CHARSET) : "") +
+                        pluginsVers;
+
+                URLConnection conn = new URL(url).openConnection();
+
+                if (!isCancelled()) {
+                    conn.setDoOutput(true);
+                    conn.setRequestProperty("Accept-Charset", CHARSET);
+                    conn.setRequestProperty("Content-Type", "application/x-www-form-urlencoded;charset=" + CHARSET);
+
+                    conn.setConnectTimeout(3000);
+                    conn.setReadTimeout(3000);
+
+                    Document dom = null;
+
+                    try {
+                        try (OutputStream os = conn.getOutputStream()) {
+                            os.write(postParams.getBytes(CHARSET));
+                        }
+
+                        try (InputStream in = conn.getInputStream()) {
+                            if (in == null)
+                                return;
+
+                            BufferedReader reader = new BufferedReader(new InputStreamReader(in, CHARSET));
+
+                            StringBuilder xml = new StringBuilder();
+
+                            String line;
+
+                            while ((line = reader.readLine()) != null) {
+                                if (line.contains("<meta") && !line.contains("/>"))
+                                    line = line.replace(">", "/>");
+
+                                xml.append(line).append('\n');
+                            }
+
+                            dom = documentBuilder.parse(new ByteArrayInputStream(xml.toString().getBytes(CHARSET)));
+                        }
+                    }
+                    catch (IOException e) {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to connect to Ignite update server. " + e.getMessage());
+                    }
+
+                    if (dom != null) {
+                        latestVer = obtainVersionFrom(dom);
+
+                        downloadUrl = obtainDownloadUrlFrom(dom);
+                    }
+                }
+            }
+            catch (Exception e) {
+                if (log.isDebugEnabled())
+                    log.debug("Unexpected exception in update checker. " + e.getMessage());
+            }
+        }
+
+        /**
+         * Gets the version from the current {@code node}, if one exists.
+         *
+         * @param node W3C DOM node.
+         * @return Version or {@code null} if one's not found.
+         */
+        @Nullable private String obtainMeta(String metaName, Node node) {
+            assert node != null;
+
+            if (node instanceof Element && "meta".equals(node.getNodeName().toLowerCase())) {
+                Element meta = (Element)node;
+
+                String name = meta.getAttribute("name");
+
+                if (metaName.equals(name)) {
+                    String content = meta.getAttribute("content");
+
+                    if (content != null && !content.isEmpty())
+                        return content;
+                }
+            }
+
+            NodeList childNodes = node.getChildNodes();
+
+            for (int i = 0; i < childNodes.getLength(); i++) {
+                String ver = obtainMeta(metaName, childNodes.item(i));
+
+                if (ver != null)
+                    return ver;
+            }
+
+            return null;
+        }
+
+        /**
+         * Gets the version from the current {@code node}, if one exists.
+         *
+         * @param node W3C DOM node.
+         * @return Version or {@code null} if one's not found.
+         */
+        @Nullable private String obtainVersionFrom(Node node) {
+            return obtainMeta("version", node);
+        }
+
+        /**
+         * Gets the download url from the current {@code node}, if one exists.
+         *
+         * @param node W3C DOM node.
+         * @return download url or {@code null} if one's not found.
+         */
+        @Nullable private String obtainDownloadUrlFrom(Node node) {
+            return obtainMeta("downloadUrl", node);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
deleted file mode 100644
index 93fd916..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal;
-
-import java.util.Collections;
-import java.util.Properties;
-import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteProductVersion;
-import org.apache.ignite.plugin.PluginProvider;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.testframework.junits.common.GridCommonTest;
-
-/**
- * Update notifier test.
- */
-@GridCommonTest(group = "Kernal Self")
-public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest {
-    /** */
-    private String updateStatusParams;
-
-    /** {@inheritDoc} */
-    @Override protected long getTestTimeout() {
-        return 30 * 1000;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, "true");
-
-        Properties props = U.field(IgniteProperties.class, "PROPS");
-
-        updateStatusParams = props.getProperty("ignite.update.status.params");
-
-        props.setProperty("ignite.update.status.params", "ver=" + IgniteProperties.get("ignite.version"));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
-
-        System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, "false");
-
-        Properties props = U.field(IgniteProperties.class, "PROPS");
-
-        props.setProperty("ignite.update.status.params", updateStatusParams);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNotifier() throws Exception {
-        String nodeVer = IgniteProperties.get("ignite.version");
-
-        GridUpdateNotifier ntf = new GridUpdateNotifier(null, nodeVer,
-            TEST_GATEWAY, Collections.<PluginProvider>emptyList(), false);
-
-        ntf.checkForNewVersion(log);
-
-        String ver = ntf.latestVersion();
-
-        // Wait 60 sec for response.
-        for (int i = 0; ver == null && i < 600; i++) {
-            Thread.sleep(100);
-
-            ver = ntf.latestVersion();
-        }
-
-        info("Latest version: " + ver);
-
-        assertNotNull("Ignite latest version has not been detected.", ver);
-
-        byte nodeMaintenance = IgniteProductVersion.fromString(nodeVer).maintenance();
-
-        byte lastMaintenance = IgniteProductVersion.fromString(ver).maintenance();
-
-        assertTrue("Wrong latest version.", (nodeMaintenance == 0 && lastMaintenance == 0) ||
-            (nodeMaintenance > 0 && lastMaintenance > 0));
-
-        ntf.reportStatus(log);
-    }
-
-    /**
-     * Test kernal gateway that always return uninitialized user stack trace.
-     */
-    private static final GridKernalGateway TEST_GATEWAY = new GridKernalGateway() {
-        @Override public void readLock() throws IllegalStateException {}
-
-        @Override public void readLockAnyway() {}
-
-        @Override public void setState(GridKernalState state) {}
-
-        @Override public GridKernalState getState() {
-            return null;
-        }
-
-        @Override public void readUnlock() {}
-
-        @Override public void writeLock() {}
-
-        @Override public void writeUnlock() {}
-
-        @Override public String userStackTrace() {
-            return null;
-        }
-
-        @Override public boolean tryWriteLock(long timeout) {
-            return false;
-        }
-
-        @Override public GridFutureAdapter<?> onDisconnected() {
-            return null;
-        }
-
-        @Override public void onReconnected() {
-            // No-op.
-        }
-    };
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java
new file mode 100644
index 0000000..a255f15
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteUpdateNotifierPerClusterSettingSelfTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
+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;
+
+/**
+ */
+public class IgniteUpdateNotifierPerClusterSettingSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private String backup;
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        backup = System.getProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, backup);
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNotifierEnabledForCluster() throws Exception {
+        checkNotifierStatusForCluster(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNotifierDisabledForCluster() throws Exception {
+        checkNotifierStatusForCluster(false);
+    }
+
+    /**
+     * @param enabled Notifier status.
+     * @throws Exception If failed.
+     */
+    private void checkNotifierStatusForCluster(boolean enabled) throws Exception {
+        System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, String.valueOf(enabled));
+
+        IgniteEx grid1 = startGrid(0);
+
+        checkNotifier(grid1, enabled);
+
+        System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, String.valueOf(!enabled));
+
+        IgniteEx grid2 = startGrid(1);
+
+        checkNotifier(grid2, enabled);
+
+        client = true;
+
+        IgniteEx grid3 = startGrid(2);
+
+        checkNotifier(grid3, enabled);
+
+        // Failover.
+        stopGrid(0); // Kill oldest.
+
+        client = false;
+
+        IgniteEx grid4 = startGrid(3);
+
+        checkNotifier(grid4, enabled);
+
+        client = true;
+
+        IgniteEx grid5 = startGrid(4);
+
+        checkNotifier(grid5, enabled);
+    }
+
+    /**
+     * @param ignite Node.
+     * @param expEnabled Expected notifier status.
+     */
+    private void checkNotifier(Ignite ignite, boolean expEnabled) {
+        ClusterProcessor proc = ((IgniteKernal)ignite).context().cluster();
+
+        if (expEnabled)
+            assertNotNull(GridTestUtils.getFieldValue(proc, "updateNtfTimer"));
+        else
+            assertNull(GridTestUtils.getFieldValue(proc, "updateNtfTimer"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifierSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifierSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifierSelfTest.java
new file mode 100644
index 0000000..309399c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cluster/GridUpdateNotifierSelfTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.cluster;
+
+import java.util.Collections;
+import java.util.Properties;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.GridKernalGateway;
+import org.apache.ignite.internal.GridKernalState;
+import org.apache.ignite.internal.IgniteProperties;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteProductVersion;
+import org.apache.ignite.plugin.PluginProvider;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.testframework.junits.common.GridCommonTest;
+
+/**
+ * Update notifier test.
+ */
+@GridCommonTest(group = "Kernal Self")
+public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest {
+    /** */
+    private String updateStatusParams;
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 30 * 1000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, "true");
+
+        Properties props = U.field(IgniteProperties.class, "PROPS");
+
+        updateStatusParams = props.getProperty("ignite.update.status.params");
+
+        props.setProperty("ignite.update.status.params", "ver=" + IgniteProperties.get("ignite.version"));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        System.setProperty(IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER, "false");
+
+        Properties props = U.field(IgniteProperties.class, "PROPS");
+
+        props.setProperty("ignite.update.status.params", updateStatusParams);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNotifier() throws Exception {
+        String nodeVer = IgniteProperties.get("ignite.version");
+
+        GridUpdateNotifier ntf = new GridUpdateNotifier(null, nodeVer,
+            TEST_GATEWAY, Collections.<PluginProvider>emptyList(), false);
+
+        ntf.checkForNewVersion(log);
+
+        String ver = ntf.latestVersion();
+
+        // Wait 60 sec for response.
+        for (int i = 0; ver == null && i < 600; i++) {
+            Thread.sleep(100);
+
+            ver = ntf.latestVersion();
+        }
+
+        info("Notifier version [ver=" + ver + ", nodeVer=" + nodeVer + ']');
+
+        assertNotNull("Ignite latest version has not been detected.", ver);
+
+        byte nodeMaintenance = IgniteProductVersion.fromString(nodeVer).maintenance();
+
+        byte lastMaintenance = IgniteProductVersion.fromString(ver).maintenance();
+
+        assertTrue("Wrong latest version [nodeVer=" + nodeMaintenance + ", lastVer=" + lastMaintenance + ']',
+            (nodeMaintenance == 0 && lastMaintenance == 0) || (nodeMaintenance > 0 && lastMaintenance > 0));
+
+        ntf.reportStatus(log);
+    }
+
+    /**
+     * Test kernal gateway that always return uninitialized user stack trace.
+     */
+    private static final GridKernalGateway TEST_GATEWAY = new GridKernalGateway() {
+        @Override public void readLock() throws IllegalStateException {}
+
+        @Override public void readLockAnyway() {}
+
+        @Override public void setState(GridKernalState state) {}
+
+        @Override public GridKernalState getState() {
+            return null;
+        }
+
+        @Override public void readUnlock() {}
+
+        @Override public void writeLock() {}
+
+        @Override public void writeUnlock() {}
+
+        @Override public String userStackTrace() {
+            return null;
+        }
+
+        @Override public boolean tryWriteLock(long timeout) {
+            return false;
+        }
+
+        @Override public GridFutureAdapter<?> onDisconnected() {
+            return null;
+        }
+
+        @Override public void onReconnected() {
+            // No-op.
+        }
+    };
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86c2ba2a/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 deb49b7..7197eb8 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
@@ -33,8 +33,9 @@ import org.apache.ignite.internal.GridNodeVisorAttributesSelfTest;
 import org.apache.ignite.internal.GridRuntimeExceptionSelfTest;
 import org.apache.ignite.internal.GridSameVmStartupSelfTest;
 import org.apache.ignite.internal.GridSpiExceptionSelfTest;
-import org.apache.ignite.internal.GridUpdateNotifierSelfTest;
+import org.apache.ignite.internal.processors.cluster.GridUpdateNotifierSelfTest;
 import org.apache.ignite.internal.GridVersionSelfTest;
+import org.apache.ignite.internal.IgniteUpdateNotifierPerClusterSettingSelfTest;
 import org.apache.ignite.internal.managers.GridManagerStopSelfTest;
 import org.apache.ignite.internal.managers.communication.GridCommunicationSendMessageSelfTest;
 import org.apache.ignite.internal.managers.deployment.GridDeploymentManagerStopSelfTest;
@@ -107,6 +108,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridNodeLocalSelfTest.class);
         suite.addTestSuite(GridKernalConcurrentAccessStopSelfTest.class);
         suite.addTestSuite(GridUpdateNotifierSelfTest.class);
+        suite.addTestSuite(IgniteUpdateNotifierPerClusterSettingSelfTest.class);
         suite.addTestSuite(GridLocalEventListenerSelfTest.class);
         suite.addTestSuite(IgniteTopologyPrintFormatSelfTest.class);