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 2017/12/21 09:17:20 UTC

ignite git commit: IGNITE-6976 Visor CMD: Implemented simple cache put/get/remove operations.

Repository: ignite
Updated Branches:
  refs/heads/master bed5c3e9a -> 35622cbba


IGNITE-6976 Visor CMD: Implemented simple cache put/get/remove operations.


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

Branch: refs/heads/master
Commit: 35622cbba97e72750b9d9cffd960933d545539ba
Parents: bed5c3e
Author: vsisko <vs...@gridgain.com>
Authored: Thu Dec 21 16:17:14 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Dec 21 16:17:14 2017 +0700

----------------------------------------------------------------------
 .../rest/request/RestQueryRequest.java          |  26 +-
 .../ignite/visor/commands/VisorConsole.scala    |   1 +
 .../visor/commands/ack/VisorAckCommand.scala    |   5 +-
 .../commands/alert/VisorAlertCommand.scala      |   4 +-
 .../commands/cache/VisorCacheCommand.scala      |  29 +-
 .../cache/VisorCacheModifyCommand.scala         | 413 +++++++++++++++++++
 .../commands/common/VisorConsoleCommand.scala   |  45 +-
 .../visor/commands/common/VisorTextTable.scala  |  26 +-
 .../config/VisorConfigurationCommand.scala      |  57 ++-
 .../commands/disco/VisorDiscoveryCommand.scala  |   4 +-
 .../commands/events/VisorEventsCommand.scala    |   9 +-
 .../visor/commands/gc/VisorGcCommand.scala      |   4 +-
 .../visor/commands/kill/VisorKillCommand.scala  |   4 +-
 .../visor/commands/node/VisorNodeCommand.scala  |  12 +-
 .../visor/commands/open/VisorOpenCommand.scala  |  28 +-
 .../visor/commands/ping/VisorPingCommand.scala  |   4 +-
 .../commands/start/VisorStartCommand.scala      |   4 +-
 .../commands/tasks/VisorTasksCommand.scala      |   4 +-
 .../commands/top/VisorTopologyCommand.scala     |   4 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |   4 +-
 .../scala/org/apache/ignite/visor/visor.scala   | 174 ++++----
 21 files changed, 632 insertions(+), 229 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
index 75c74db..ca24a27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/request/RestQueryRequest.java
@@ -1,20 +1,18 @@
 /*
+ * 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
  *
- *  * 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.
+ *      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.rest.request;

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala
index a43f9ff..8bf64b7 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/VisorConsole.scala
@@ -65,6 +65,7 @@ class VisorConsole {
         org.apache.ignite.visor.commands.cache.VisorCacheResetCommand
         org.apache.ignite.visor.commands.cache.VisorCacheRebalanceCommand
         org.apache.ignite.visor.commands.cache.VisorCacheCommand
+        org.apache.ignite.visor.commands.cache.VisorCacheModifyCommand
         org.apache.ignite.visor.commands.config.VisorConfigurationCommand
         org.apache.ignite.visor.commands.deploy.VisorDeployCommand
         org.apache.ignite.visor.commands.disco.VisorDiscoveryCommand

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ack/VisorAckCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ack/VisorAckCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ack/VisorAckCommand.scala
index 13c343a..eadf64b 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ack/VisorAckCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ack/VisorAckCommand.scala
@@ -88,9 +88,7 @@ class VisorAckCommand extends VisorConsoleCommand {
      * @param msg Optional command argument. If `null` this function is no-op.
      */
     def ack(msg: String) {
-        if (!isConnected)
-            adviseToConnect()
-        else
+        if (checkConnected()) {
             try {
                 executeMulti(classOf[VisorAckTask], new VisorAckTaskArg(msg))
             }
@@ -98,6 +96,7 @@ class VisorAckCommand extends VisorConsoleCommand {
                 case _: ClusterGroupEmptyException => scold("Topology is empty.")
                 case e: Exception => scold("System error: " + e.getMessage)
             }
+        }
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
index 254dc2d..ffa6a94 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/alert/VisorAlertCommand.scala
@@ -260,9 +260,7 @@ class VisorAlertCommand extends VisorConsoleCommand {
         breakable {
             assert(args != null)
 
-            if (!isConnected)
-                adviseToConnect()
-            else {
+            if (checkConnected()) {
                 var name: Option[String] = None
                 var script: Option[String] = None
                 val conditions = mutable.ArrayBuffer.empty[VisorAlertCondition]

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index fec5a96..3571efb 100755
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -17,17 +17,18 @@
 
 package org.apache.ignite.visor.commands.cache
 
-import java.util.{Collection => JavaCollection, List => JavaList, Collections, UUID}
+import java.util.{Collections, UUID, Collection => JavaCollection, List => JavaList}
 
 import org.apache.ignite._
 import org.apache.ignite.cluster.ClusterNode
 import org.apache.ignite.internal.util.lang.{GridFunc => F}
+import org.apache.ignite.internal.util.scala.impl
 import org.apache.ignite.internal.util.typedef.X
 import org.apache.ignite.internal.visor.cache._
 import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.apache.ignite.visor.VisorTag
 import org.apache.ignite.visor.commands.cache.VisorCacheCommand._
-import org.apache.ignite.visor.commands.common.VisorTextTable
+import org.apache.ignite.visor.commands.common.{VisorConsoleCommand, VisorTextTable}
 import org.apache.ignite.visor.visor._
 import org.jetbrains.annotations._
 
@@ -161,18 +162,8 @@ import scala.language.{implicitConversions, reflectiveCalls}
  *
  * }}}
  */
-class VisorCacheCommand {
-    /**
-     * Prints error message and advise.
-     *
-     * @param errMsgs Error messages.
-     */
-    private def scold(errMsgs: Any*) {
-        assert(errMsgs != null)
-
-        warn(errMsgs: _*)
-        warn("Type 'help cache' to see how to use this command.")
-    }
+class VisorCacheCommand extends VisorConsoleCommand {
+    @impl protected val name: String = "cache"
 
     /**
      * ===Command===
@@ -214,14 +205,7 @@ class VisorCacheCommand {
      * @param args Command arguments.
      */
     def cache(args: String) {
-        if (!isConnected)
-            adviseToConnect()
-        else if (!isActive) {
-            warn("Can not perform the operation because the cluster is inactive.",
-                "Note, that the cluster is considered inactive by default if Ignite Persistent Store is used to let all the nodes join the cluster.",
-                "To activate the cluster execute following command: top -activate.")
-        }
-        else {
+        if (checkConnected() && checkActiveState()) {
             var argLst = parseArgs(args)
 
             if (hasArgFlag("i", argLst)) {
@@ -639,7 +623,6 @@ class VisorCacheCommand {
     def askForCache(title: String, node: Option[ClusterNode], showSystem: Boolean = false,
         aggrData: Seq[VisorCacheAggregatedMetrics]): Option[String] = {
         assert(title != null)
-        assert(visor.visor.isConnected)
 
         if (aggrData.isEmpty) {
             scold("No caches found.")

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheModifyCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheModifyCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheModifyCommand.scala
new file mode 100644
index 0000000..7461889
--- /dev/null
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheModifyCommand.scala
@@ -0,0 +1,413 @@
+/*
+ * 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.visor.commands.cache
+
+import java.util.{Calendar, Date, GregorianCalendar, UUID}
+
+import org.apache.ignite.internal.util.lang.{GridFunc => F}
+import org.apache.ignite.internal.util.scala.impl
+import org.apache.ignite.internal.visor.cache._
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
+import org.apache.ignite.visor.commands.cache.VisorCacheModifyCommand._
+import org.apache.ignite.visor.commands.common.VisorConsoleCommand
+import org.apache.ignite.visor.visor._
+
+/**
+ * ==Overview==
+ * Visor 'modify' command implementation.
+ *
+ * ==Help==
+ * {{{
+ * +-----------------------------------------------------------------------------------------+
+ * | modify -put    | Put custom value into cache.                                           |
+ * +-----------------------------------------------------------------------------------------+
+ * | modify -get    | Get value with specified key from cache.                               |
+ * +-----------------------------------------------------------------------------------------+
+ * | modify -remove | Remove value with specified key from cache.                            |
+ * +-----------------------------------------------------------------------------------------+
+ *
+ * }}}
+ *
+ * ====Specification====
+ * {{{
+ *     modify -put -c=<cache-name> {-kt=<key-type>} {-kv=<key>} {-vt=<value-type>} {-v=<value>}
+ *     modify -get -c=<cache-name> {-kt=<key-type>} {-kv=<key>}
+ *     modify -remove -c=<cache-name> {-kt=<key-type>} {-kv=<key>}
+ * }}}
+ *
+ * ====Arguments====
+ * {{{
+ *     -c=<cache-name>
+ *         Name of the cache.
+ *     -kt=<key-type>
+ *         Type of key. Default value is java.lang.String. Short type name can be specified.
+ *     -kv=<key>
+ *         Key. Asked in interactive mode when it is not specified.
+ *     -vt=<value-type>.
+ *         Type of value. Default value is java.lang.String. Short type name can be specified.
+ *         Value type is equals to key type when value is not specified.
+ *     -v=<value>
+ *         Value. Equals to key when it is not specified.
+ *         Asked in interactive mode when key and value are not specified.
+ * }}}
+ *
+ * ====Examples====
+ * {{{
+ *     modify -put -c=@c0
+ *         Put value into cache in interactive mode.
+ *     modify -get -c=@c0
+ *         Get value from cache in interactive mode.
+ *     modify -remove -c=@c0
+ *         Remove value from cache in interactive mode.
+ *     modify -put -c=cache -kv=key1
+ *         Put value into cache with name cache with key of default String type equal to key1
+ *         and value equal to key.
+ *     modify -put -c=cache -kt=java.lang.String -kv=key1 -vt=lava.lang.String -v=value1
+ *         Put value into cache with name cache with key of String type equal to key1
+ *         and value of String type equal to value1
+ *     modify -get -c=cache -kt=java.lang.String -kv=key1
+ *         Get value from cache with name cache with key of String type equal to key1
+ *     modify -remove -c=cache -kt=java.lang.String -kv=key1
+ *         Remove value from cache with name cache with key of String type equal to key1.
+ *
+ * }}}
+ */
+class VisorCacheModifyCommand extends VisorConsoleCommand {
+    @impl protected val name = "modify"
+
+    /**
+     * ===Command===
+     * Modify cache value in specified cache.
+     *
+     * ===Examples===
+     * <ex>modify -put -c=@c0</ex>
+     *     Put value into cache with name taken from 'c0' memory variable in interactive mode.
+     * <br>
+     * <ex>modify -get</ex>
+     *     Get value from cache with name taken from 'c0' memory variable in interactive mode.
+     * <br>
+     * <ex>modify -remove</ex>
+     *     Remove value from cache with name taken from 'c0' memory variable in interactive mode.
+     * <br>
+     * <ex>modify -put -c=cache -kt=java.lang.String -k=key1 -vt=lava.lang.String -v=value1</ex>
+     *     Put value into cache with name 'cache' with key of String type equal to 'key1'
+     *     and value of String type equal to 'value1'
+     * <br>
+     * <ex>modify -get -c=cache -kt=java.lang.String -k=key1</ex>
+     *     Get value from cache with name 'cache' with key of String type equal to 'key1'
+     * <br>
+     * <ex>modify -remove -c=cache -kt=java.lang.String -k=key1</ex>
+     *     Remove value from cache with name 'cache' with key of String type equal to 'key1'.
+     *
+     * @param args Command arguments.
+     */
+    def modify(args: String) {
+        if (checkConnected() && checkActiveState()) {
+            def argNonEmpty(argLst: ArgList, arg: Option[String], key: String): Boolean = {
+                if (hasArgName(key, argLst) && arg.forall((a) => F.isEmpty(a))) {
+                    warn(s"Argument $key is specified and can not be empty")
+
+                    false
+                }
+                else
+                    true
+            }
+
+            var argLst = parseArgs(args)
+
+            val put = hasArgFlag("put", argLst)
+            val get = hasArgFlag("get", argLst)
+            val remove = hasArgFlag("remove", argLst)
+
+            if (!put && !get && !remove) {
+                warn("Put, get, or remove operation should be specified")
+
+                return
+            }
+
+            if (put && get || get && remove || get && remove) {
+                warn("Only one operation put, get or remove allowed in one command invocation")
+
+                return
+            }
+
+            if (!hasArgName("c", argLst)) {
+                warn("Cache name should be specified")
+
+                return
+            }
+
+            val cacheName = argValue("c", argLst) match {
+                case Some(dfltName) if dfltName == DFLT_CACHE_KEY || dfltName == DFLT_CACHE_NAME =>
+                    argLst = argLst.filter(_._1 != "c") ++ Seq("c" -> null)
+
+                    Some(null)
+
+                case cn => cn
+            }
+
+            if (cacheName.isEmpty) {
+                warn("Cache with specified name is not found")
+
+                return
+            }
+
+            val keyTypeStr = argValue("kt", argLst)
+            val keyStr = argValue("k", argLst)
+            var key: Object = null
+
+            if (keyTypeStr.nonEmpty && keyStr.isEmpty) {
+                warn("Key should be specified when key type is specified")
+
+                return
+            }
+
+            val valueTypeStr = argValue("vt", argLst)
+            val valueStr = argValue("v", argLst)
+            var value: Object = null
+
+            if (valueTypeStr.nonEmpty && valueStr.isEmpty) {
+                warn("Value should be specified when value type is specified")
+
+                return
+            }
+
+            if (!argNonEmpty(argLst, keyTypeStr, "kt")
+                || !argNonEmpty(argLst, keyStr, "k")
+                || !argNonEmpty(argLst, valueTypeStr, "vt")
+                || !argNonEmpty(argLst, valueStr, "v"))
+                return
+
+            keyTypeStr match {
+                case Some(clsStr) =>
+                    try {
+                        INPUT_TYPES.find(_._3.getName.indexOf(clsStr) >= 0) match {
+                            case Some(t) => key = t._2(keyStr.get)
+                            case None =>
+                                warn("Specified type is not allowed")
+
+                                return
+                        }
+                    }
+                    catch {
+                        case e: Throwable =>
+                            warn("Failed to read key: " + e.getMessage)
+
+                            return
+                    }
+
+                case None if keyStr.nonEmpty =>
+                    key = keyStr.get
+
+                case None if put && valueStr.nonEmpty => // No-op.
+
+                case None =>
+                    askTypedValue("key") match {
+                        case Some(k) if k.toString.nonEmpty => key = k
+                        case _ =>
+                            warn("Key can not be empty.")
+
+                            return
+                    }
+            }
+
+            if (put) {
+                valueTypeStr match {
+                    case Some(clsStr) =>
+                        try {
+                            INPUT_TYPES.find(_._3.getName.indexOf(clsStr) >= 0) match {
+                                case Some(t) => value = t._2(valueStr.get)
+                                case None => warn("Specified type is not allowed")
+
+                                    return
+                            }
+                        }
+                        catch {
+                            case e: Throwable =>
+                                warn("Failed to read value: " + e.getMessage)
+
+                                return
+                        }
+                    case None if valueStr.nonEmpty =>
+                        value = valueStr.get
+
+                    case None =>
+                        askTypedValue("value") match {
+                            case Some(v) if v.toString.nonEmpty => value = v
+                            case _ =>
+                                warn("Value can not be empty.")
+
+                                return
+                        }
+                }
+
+                if (key == null)
+                    key = value
+            }
+
+            if ((get || remove) && valueTypeStr.nonEmpty)
+                warn("Specified value is not used by selected operation and will be ignored")
+
+            val arg = new VisorCacheModifyTaskArg(cacheName.get,
+                if (put) VisorModifyCacheMode.PUT else if (get) VisorModifyCacheMode.GET else VisorModifyCacheMode.REMOVE,
+                key, value
+            )
+
+            try {
+                val taskResult = executeRandom(classOf[VisorCacheModifyTask], arg)
+                val resultObj = taskResult.getResult match {
+                    case d: Date =>
+                        val cal = new GregorianCalendar()
+                        cal.setTime(d)
+
+                        if (cal.get(Calendar.HOUR_OF_DAY) == 0 && cal.get(Calendar.MINUTE) == 0
+                            && cal.get(Calendar.SECOND) == 0)
+                            formatDate(d)
+                        else
+                            formatDateTime(d)
+
+                    case v => v
+                }
+                val affinityNode = taskResult.getAffinityNode
+
+                if (put) {
+                    println("Put operation success" + "; Affinity node: " + nid8(affinityNode))
+
+                    if (resultObj != null)
+                        println("Previous value is: " + resultObj)
+                }
+
+                if (get) {
+                    if (resultObj != null)
+                        println("Value with specified key: " + resultObj + "; Affinity node: " + nid8(affinityNode))
+                    else
+                        println("Value with specified key not found")
+                }
+
+                if (remove) {
+                    if (resultObj != null)
+                        println("Removed value: " + resultObj + "; Affinity node: " + nid8(affinityNode))
+                    else
+                        println("Value with specified key not found")
+                }
+            }
+            catch {
+                case e: Throwable =>
+                    warn("Failed to execute cache modify operation: " + e.getMessage)
+            }
+        }
+    }
+
+    /**
+     * ===Command===
+     * Modify cache data by execution of put/get/remove command.
+     *
+     * ===Examples===
+     * <ex>modify -put -c=@c0</ex>
+     * Put entity in cache with name taken from 'c0' memory variable in interactive mode
+     */
+    def modify() {
+        this.modify("")
+    }
+}
+
+/**
+ * Companion object that does initialization of the command.
+ */
+object VisorCacheModifyCommand {
+    /** Singleton command */
+    private val cmd = new VisorCacheModifyCommand
+
+    /** Default cache name to show on screen. */
+    private final val DFLT_CACHE_NAME = escapeName(null)
+
+    /** Default cache key. */
+    protected val DFLT_CACHE_KEY: String = DFLT_CACHE_NAME + "-" + UUID.randomUUID().toString
+
+    addHelp(
+        name = "modify",
+        shortInfo = "Modify cache by put/get/remove value.",
+        longInfo = Seq(
+            "Execute modification of cache data:",
+            " ",
+            "Put new value into cache.",
+            " ",
+            "Get value from cache.",
+            " ",
+            "Remove value from cache."
+        ),
+        spec = Seq(
+            "modify -put -c=<cache-name> {-kt=<key-type>} {-k=<key>} {-vt=<value-type>} {-v=<value>}",
+            "modify -get -c=<cache-name> {-kt=<key-type>} {-k=<key>}",
+            "modify -remove -c=<cache-name> {-kt=<key-type>} {-k=<key>}"
+    ),
+        args = Seq(
+            "-c=<cache-name>" ->
+                "Name of the cache",
+            "-put" -> Seq(
+                "Put value into cache and show its affinity node.",
+                "If the cache previously contained a mapping for the key, the old value is shown",
+                "Key and value are asked in interactive mode when they are not specified.",
+                "Key is equals to value when key is not specified."
+            ),
+            "-get" -> Seq(
+                "Get value from cache and show its affinity node.",
+                "Key is asked in interactive mode when it is not specified."
+            ),
+            "-remove" -> Seq(
+                "Remove value from cache and show its affinity node.",
+                "Key is asked in interactive mode when it is not specified."
+            ),
+            "-kt=<key-type>" ->
+                "Type of key. Default type is java.lang.String. Type name can be specified without package.",
+            "-k=<key>" ->
+                "Key. Must be specified when key type is specified.",
+            "-vt=<value-type>" ->
+                "Type of value. Default type is java.lang.String. Type name can be specified without package.",
+            "-v=<value>" ->
+                "Value. Must be specified when value type is specified."
+        ),
+        examples = Seq(
+            "modify -put -c=@c0" ->
+                "Put value into cache with name taken from 'c0' memory variable in interactive mode.",
+            "modify -get -c=@c0" ->
+                "Get value from cache with name taken from 'c0' memory variable in interactive mode.",
+            "modify -remove -c=@c0" ->
+                "Remove value from cache with name taken from 'c0' memory variable in interactive mode.",
+            "modify -put -c=cache -v=value1" -> Seq(
+                "Put the value 'value1' into the cache 'cache'.",
+                "Other params have default values: -kt = java.lang.String , -k = value1, -vt = java.lang.String"
+            ),
+            "modify -put -c=@c0 -kt=java.lang.String -k=key1 -vt=lava.lang.String -v=value1" -> Seq(
+                "Put value into cache with name taken from 'c0' memory variable",
+                "with key of String type equal to 'key1' and value of String type equal to 'value1'"
+            ),
+            "modify -get -c=@c0 -kt=java.lang.String -k=key1" ->
+                "Get value from cache with name taken from 'c0' memory variable with key of String type equal to key1",
+            "modify -remove -c=@c0 -kt=java.lang.String -k=key1" ->
+                "Remove value from cache with name taken from 'c0' memory variable with key of String type equal to key1."
+        ),
+        emptyArgs = cmd.modify,
+        withArgs = cmd.modify
+    )
+
+    /**
+     * Singleton.
+     */
+    def apply(): VisorCacheModifyCommand = cmd
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorConsoleCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorConsoleCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorConsoleCommand.scala
index fb2b716..0658ad4 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorConsoleCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorConsoleCommand.scala
@@ -1,20 +1,18 @@
 /*
+ * 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
  *
- *  * 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.
+ *      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.visor.commands.common
@@ -58,6 +56,21 @@ trait VisorConsoleCommand {
     }
 
     /**
+     * Check cluster active state and show inform message when cluster has inactive state.
+     *
+     * @return `True` when cluster is active.
+     */
+    protected def checkActiveState(): Boolean = {
+        visor.isActive || {
+            warn("Can not perform the operation because the cluster is inactive.",
+                "Note, that the cluster is considered inactive by default if Ignite Persistent Store is used to let all the nodes join the cluster.",
+                "To activate the cluster execute following command: top -activate.")
+
+            false
+        }
+    }
+
+    /**
      * Prints warn message and advise.
      *
      * @param warnMsgs Warning messages.
@@ -75,14 +88,14 @@ trait VisorConsoleCommand {
      * @param lines Lines to join together.
      * @return Joined line.
      */
-    protected def join(lines: java.lang.Iterable[_ <: Any]) = {
+    protected def join(lines: java.lang.Iterable[_ <: Any]): String = {
         if (lines == null || lines.isEmpty)
             NA
         else
             lines.mkString("[", ", ", "]")
     }
 
-    protected def join(lines: Array[_ <: Any]) = {
+    protected def join(lines: Array[_ <: Any]): String = {
         if (lines == null || lines.isEmpty)
             NA
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorTextTable.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorTextTable.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorTextTable.scala
index e6fe35f..58e3f21 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorTextTable.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/common/VisorTextTable.scala
@@ -1,20 +1,18 @@
 /*
+ * 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
  *
- *  * 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.
+ *      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.visor.commands.common

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
index 0185228..67d9c14 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
@@ -83,13 +83,12 @@ class VisorConfigurationCommand extends VisorConsoleCommand {
       * Starts command in interactive mode.
      */
     def config() {
-        if (isConnected)
+        if (checkConnected()) {
             askForNode("Select node from:") match {
                 case Some(id) => config("-id=" + id)
                 case None => ()
             }
-        else
-            adviseToConnect()
+        }
     }
 
     /**
@@ -103,44 +102,40 @@ class VisorConfigurationCommand extends VisorConsoleCommand {
      * @param args Command arguments.
      */
     def config(args: String) {
-        if (!isConnected) {
-            adviseToConnect()
-
-            return
-        }
-
-        val argLst = parseArgs(args)
+        if (checkConnected()) {
+            val argLst = parseArgs(args)
 
-        val nid = parseNode(argLst) match {
-            case Left(msg) =>
-                scold(msg)
+            val nid = parseNode(argLst) match {
+                case Left(msg) =>
+                    scold(msg)
 
-                return
+                    return
 
-            case Right(None) =>
-                scold("One of -id8 or -id is required.")
+                case Right(None) =>
+                    scold("One of -id8 or -id is required.")
 
-                return
+                    return
 
-            case Right(Some(n)) =>
-                assert(n != null)
+                case Right(Some(n)) =>
+                    assert(n != null)
 
-                n.id()
-        }
+                    n.id()
+            }
 
-        try {
-            val cfg = collectConfiguration(nid)
+            try {
+                val cfg = collectConfiguration(nid)
 
-            printConfiguration(cfg)
+                printConfiguration(cfg)
 
-            cacheConfigurations(nid).foreach(ccfg => {
-                println()
+                cacheConfigurations(nid).foreach(ccfg => {
+                    println()
 
-                printCacheConfiguration(s"Cache '${escapeName(ccfg.getName)}':", ccfg)
-            })
-        }
-        catch {
-            case e: Throwable => scold(e)
+                    printCacheConfiguration(s"Cache '${escapeName(ccfg.getName)}':", ccfg)
+                })
+            }
+            catch {
+                case e: Throwable => scold(e)
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala
index de69cd7..3b4c90f 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/disco/VisorDiscoveryCommand.scala
@@ -117,9 +117,7 @@ class VisorDiscoveryCommand extends VisorConsoleCommand {
      * Prints discovery events fired during last two minutes.
      */
     def disco(args: String) {
-        if (!isConnected)
-            adviseToConnect()
-        else {
+        if (checkConnected()) {
             val argLst = parseArgs(args)
 
             val fs = argValue("t", argLst)

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala
index c41eca1..e1d36c7 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/events/VisorEventsCommand.scala
@@ -127,9 +127,7 @@ class VisorEventsCommand extends VisorConsoleCommand {
      * Starts command in interactive mode.
      */
     def events() {
-        if (!isConnected)
-            adviseToConnect()
-        else
+        if (checkConnected()) {
             askForNode("Select node from:") match {
                 case Some(id) => ask("Sort [c]ronologically or by [e]vent type (c/e) [c]: ", "c") match {
                     case "c" | "C" => nl(); events("-s=t -id=" + id)
@@ -138,6 +136,7 @@ class VisorEventsCommand extends VisorConsoleCommand {
                 }
                 case None => ()
             }
+        }
     }
 
     /**
@@ -208,7 +207,7 @@ class VisorEventsCommand extends VisorConsoleCommand {
      * @param args Command parameters.
      */
     def events(args: String) {
-        if (isConnected) {
+        if (checkConnected()) {
             val argLst = parseArgs(args)
 
             parseNode(argLst) match {
@@ -328,8 +327,6 @@ class VisorEventsCommand extends VisorConsoleCommand {
                     all.render()
             }
         }
-        else
-            adviseToConnect()
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala
index 5f642d7..6f2ddb7 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/gc/VisorGcCommand.scala
@@ -83,7 +83,7 @@ class VisorGcCommand extends VisorConsoleCommand {
     def gc(args: String) {
         assert(args != null)
 
-        if (isConnected) {
+        if (checkConnected()) {
             val argLst = parseArgs(args)
 
             try {
@@ -123,8 +123,6 @@ class VisorGcCommand extends VisorConsoleCommand {
                 case e: IgniteException => scold(e)
             }
         }
-        else
-            adviseToConnect()
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala
index c705e21..059c387 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/kill/VisorKillCommand.scala
@@ -127,9 +127,7 @@ class VisorKillCommand extends VisorConsoleCommand {
      * @param args Command arguments.
      */
     def kill(args: String) = breakable {
-        if (!isConnected)
-            adviseToConnect()
-        else {
+        if (checkConnected()) {
             val argLst = parseArgs(args)
 
             val iNodes = hasArgFlag("in", argLst)

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
index 93cf233..b240aa5 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
@@ -93,9 +93,7 @@ class VisorNodeCommand extends VisorConsoleCommand {
      * Starts command in interactive mode.
      */
     def node() {
-        if (!isConnected)
-            adviseToConnect()
-        else
+        if (checkConnected()) {
             askForNode("Select node from:") match {
                 case Some(id) => ask("Detailed statistics (y/n) [n]: ", "n") match {
                     case "n" | "N" => nl(); node("-id=" + id)
@@ -104,6 +102,7 @@ class VisorNodeCommand extends VisorConsoleCommand {
                 }
                 case None => ()
             }
+        }
     }
 
     /**
@@ -120,9 +119,7 @@ class VisorNodeCommand extends VisorConsoleCommand {
      * @param args Command arguments.
      */
     def node(@Nullable args: String) = breakable {
-        if (!isConnected)
-            adviseToConnect()
-        else
+        if (checkConnected()) {
             try {
                 val argLst = parseArgs(args)
 
@@ -264,7 +261,7 @@ class VisorNodeCommand extends VisorConsoleCommand {
                             t += ("Cur/avg CPU load %", formatDouble(m.getCurrentCpuLoad * 100) +
                                 "/" + formatDouble(m.getAverageCpuLoad * 100) + "%")
                             t += ("Heap memory used/max", formatMemory(m.getHeapMemoryUsed) +
-                                "/" +  formatMemory(m.getHeapMemoryMaximum))
+                                "/" + formatMemory(m.getHeapMemoryMaximum))
                         }
 
                         println("Time of the snapshot: " + formatDateTime(System.currentTimeMillis))
@@ -279,6 +276,7 @@ class VisorNodeCommand extends VisorConsoleCommand {
             catch {
                 case e: Exception => scold(e)
             }
+        }
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
index 949aa00..f62ba3c 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
@@ -1,20 +1,18 @@
 /*
+ * 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
  *
- *  * 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.
- *  
+ *      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.visor.commands.open

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ping/VisorPingCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ping/VisorPingCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ping/VisorPingCommand.scala
index 97be127..307b78d 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ping/VisorPingCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/ping/VisorPingCommand.scala
@@ -113,9 +113,7 @@ class VisorPingCommand extends VisorConsoleCommand {
      * @param args List of node ID8s. If empty or null - pings all nodes in the topology.
      */
     def ping(args: String) = breakable {
-        if (!isConnected)
-            adviseToConnect()
-        else {
+        if (checkConnected()) {
             val argLst = parseArgs(args)
 
             val res = new Result()

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala
index 9365592..619ab5d 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/start/VisorStartCommand.scala
@@ -151,9 +151,7 @@ class VisorStartCommand extends VisorConsoleCommand {
     def start(args: String) = breakable {
         assert(args != null)
 
-        if (!isConnected)
-            adviseToConnect()
-        else {
+        if (checkConnected()) {
             val argLst = parseArgs(args)
 
             val fileOpt = argValue("f", argLst)

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
index 966bd64..9186866 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommand.scala
@@ -380,9 +380,7 @@ class VisorTasksCommand extends VisorConsoleCommand {
      * @param args Command arguments.
      */
     def tasks(args: String) {
-        if (!isConnected)
-            adviseToConnect()
-        else {
+        if (checkConnected()) {
             val argLst = parseArgs(args)
 
             if (hasArgFlag("l", argLst)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
index b75afc6..79858ee 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
@@ -145,9 +145,7 @@ class VisorTopologyCommand extends VisorConsoleCommand {
     def top(args: String) = breakable {
         assert(args != null)
 
-        if (!isConnected)
-            adviseToConnect()
-        else {
+        if (checkConnected()) {
             val argLst = parseArgs(args)
 
             if (hasArgFlag("activate", argLst))

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommand.scala
index e935256..350adc3 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommand.scala
@@ -101,9 +101,7 @@ class VisorVvmCommand extends VisorConsoleCommand {
      * @param args Command parameters.
      */
     def vvm(@Nullable args: String) = breakable {
-        if (!isConnected)
-            adviseToConnect()
-        else {
+        if (checkConnected()) {
             val argLst = parseArgs(args)
 
             val vvmHome = argValue("home", argLst) getOrElse IgniteSystemProperties.getString("VVM_HOME")

http://git-wip-us.apache.org/repos/asf/ignite/blob/35622cbb/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
index 5765579..c24bc25 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
@@ -17,6 +17,13 @@
 
 package org.apache.ignite.visor
 
+import java.io._
+import java.lang.{Boolean => JavaBoolean, Byte => JavaByte, Character => JavaCharacter, Double => JavaDouble, Float => JavaFloat, Integer => JavaInteger, Long => JavaLong, Short => JavaShort}
+import java.text._
+import java.util.concurrent._
+import java.util.{Collection => JavaCollection, HashSet => JavaHashSet, _}
+
+import jline.console.ConsoleReader
 import org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER
 import org.apache.ignite._
 import org.apache.ignite.cluster.{ClusterGroup, ClusterGroupEmptyException, ClusterMetrics, ClusterNode}
@@ -28,23 +35,15 @@ import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException
 import org.apache.ignite.internal.util.lang.{GridFunc => F}
 import org.apache.ignite.internal.util.typedef._
 import org.apache.ignite.internal.util.{GridConfigurationFinder, IgniteUtils => U}
-import org.apache.ignite.lang._
-import org.apache.ignite.thread.{IgniteThreadFactory, IgniteThreadPoolExecutor}
-import org.apache.ignite.visor.commands.common.VisorTextTable
-import jline.console.ConsoleReader
-import org.jetbrains.annotations.Nullable
-import java.io._
-import java.lang.{Boolean => JavaBoolean}
-import java.text._
-import java.util.concurrent._
-import java.util.{Collection => JavaCollection, HashSet => JavaHashSet, _}
-
 import org.apache.ignite.internal.visor.cache._
-import org.apache.ignite.internal.visor.node.VisorNodeEventsCollectorTaskArg
-import org.apache.ignite.internal.visor.node._
+import org.apache.ignite.internal.visor.node.{VisorNodeEventsCollectorTaskArg, _}
 import org.apache.ignite.internal.visor.util.VisorEventMapper
 import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.apache.ignite.internal.visor.{VisorMultiNodeTask, VisorTaskArgument}
+import org.apache.ignite.lang._
+import org.apache.ignite.thread.{IgniteThreadFactory, IgniteThreadPoolExecutor}
+import org.apache.ignite.visor.commands.common.VisorTextTable
+import org.jetbrains.annotations.Nullable
 
 import scala.collection.JavaConversions._
 import scala.collection.immutable
@@ -133,6 +132,32 @@ object visor extends VisorTag {
     /** Type alias for general event filter. */
     type EventFilter = Event => Boolean
 
+    private final val LOC = Locale.US
+
+    /** Date format. */
+    private final val dateFmt = new SimpleDateFormat("yyyy-MM-dd", LOC)
+
+    /** Date time format. */
+    private final val timeFmt = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", LOC)
+
+    final val INPUT_TYPES: Seq[(String, (String => Object), Class[_])] = Seq(
+        ("java.lang.String", (value: String) => value, classOf[String]),
+        ("java.lang.Character", (value: String) => JavaCharacter.valueOf(value.head), classOf[JavaCharacter]),
+        ("java.lang.Integer", (value: String) => JavaInteger.valueOf(value), classOf[JavaInteger]),
+        ("java.lang.Long", (value: String) => JavaLong.valueOf(value), classOf[JavaLong]),
+        ("java.lang.Short", (value: String) => JavaShort.valueOf(value), classOf[JavaShort]),
+        ("java.lang.Byte", (value: String) => JavaByte.valueOf(value), classOf[JavaByte]),
+        ("java.lang.Float", (value: String) => JavaFloat.valueOf(value), classOf[JavaFloat]),
+        ("java.lang.Double", (value: String) => JavaDouble.valueOf(value), classOf[JavaDouble]),
+        ("java.lang.Boolean", (value: String) => JavaBoolean.valueOf(value), classOf[JavaBoolean]),
+        ("java.util.Date - Value in format yyyy-MM-dd {HH:mm:ss}",
+            (value: String) => try
+                timeFmt.parse(value)
+            catch {
+                case e: ParseException => dateFmt.parse(value)
+            }, classOf[Date]),
+        ("java.util.UUID - Value like this: CC03C3B0-C03D-4B02-82AF-3E0F85414BA6", (value: String) => UUID.fromString(value), classOf[UUID]))
+
     /** `Nil` is for empty list, `Til` is for empty tuple. */
     val Til: Arg = (null, null)
 
@@ -172,14 +197,6 @@ object visor extends VisorTag {
     /** */
     @volatile private var conTs: Long = 0
 
-    private final val LOC = Locale.US
-
-    /** Date time format. */
-    private final val dtFmt = new SimpleDateFormat("MM/dd/yy, HH:mm:ss", LOC)
-
-    /** Date format. */
-    private final val dFmt = new SimpleDateFormat("dd MMMM yyyy", LOC)
-
     private final val DEC_FMT_SYMS = new DecimalFormatSymbols(LOC)
 
     /** Number format. */
@@ -651,7 +668,7 @@ object visor extends VisorTag {
     private def clearNamespace(namespace: String) {
         assert(namespace != null)
 
-        mem.keySet.foreach(k => {
+        mem.keys().foreach(k => {
             if (k.matches(s"$namespace\\d+"))
                 mem.remove(k)
         })
@@ -1165,7 +1182,7 @@ object visor extends VisorTag {
      * @param ts Timestamp.
      */
     def formatDateTime(ts: Long): String =
-        dtFmt.format(ts)
+        timeFmt.format(ts)
 
     /**
      * Returns string representation of the date provided. Result formatted using
@@ -1174,7 +1191,7 @@ object visor extends VisorTag {
      * @param date Date.
      */
     def formatDateTime(date: Date): String =
-        dtFmt.format(date)
+        timeFmt.format(date)
 
     /**
      * Returns string representation of the timestamp provided. Result formatted
@@ -1183,7 +1200,7 @@ object visor extends VisorTag {
      * @param ts Timestamp.
      */
     def formatDate(ts: Long): String =
-        dFmt.format(ts)
+        dateFmt.format(ts)
 
     /**
      * Returns string representation of the date provided. Result formatted using
@@ -1192,7 +1209,7 @@ object visor extends VisorTag {
      * @param date Date.
      */
     def formatDate(date: Date): String =
-        dFmt.format(date)
+        dateFmt.format(date)
 
     /**
      * Base class for memory units.
@@ -1328,13 +1345,19 @@ object visor extends VisorTag {
     }
 
     /**
-     * Prints standard 'not connected' error message.
+     * Check connection state and show inform message when Visor console is not connected to cluster.
+     *
+     * @return `True` when Visor console is connected to cluster.
      */
-    def adviseToConnect() {
-        warn(
-            "Visor is disconnected.",
-            "Type 'open' to connect Visor console or 'help open' to get help."
-        )
+    def checkConnected(): Boolean = {
+        isCon || {
+            warn(
+                "Visor is disconnected.",
+                "Type 'open' to connect Visor console or 'help open' to get help."
+            )
+
+            false
+        }
     }
 
     /**
@@ -2075,6 +2098,50 @@ object visor extends VisorTag {
         }
     }
 
+    def askTypedValue(name: String): Option[Object] = {
+        val t = VisorTextTable()
+
+        t #= ("#", "Type description")
+
+        INPUT_TYPES.indices.foreach(i => t += (i, INPUT_TYPES(i)._1))
+
+        println("Available " + name + " types:")
+
+        t.render()
+
+        val a = ask("\nChoose " + name + " type ('c' to cancel) [0]: ", "0")
+
+        if (a.toLowerCase == "c")
+            None
+        else {
+            try {
+                val parser = INPUT_TYPES(a.toInt)._2
+
+                try {
+                    val input = readLineOpt("Input " + name + ": ")
+
+                    input.map(parser)
+                }
+                catch {
+                    case e: Throwable =>
+                        nl()
+
+                        warn("Failed to parse value to specified type")
+
+                        None
+                }
+            }
+            catch {
+                case e: Throwable =>
+                    nl()
+
+                    warn("Invalid selection: " + a)
+
+                    None
+            }
+        }
+    }
+
     /**
      * Safe `readLine` version.
      *
@@ -2093,41 +2160,6 @@ object visor extends VisorTag {
     }
 
     /**
-     * Asks user to choose node id8.
-     *
-     * @return `Option` for node id8.
-     */
-    def askNodeId(): Option[String] = {
-        assert(isConnected)
-
-        val ids = ignite.cluster.forRemotes().nodes().map(nid8).toList
-
-        ids.indices.foreach(i => println((i + 1) + ": " + ids(i)))
-
-        nl()
-
-        println("C: Cancel")
-
-        nl()
-
-        readLineOpt("Choose node: ") match {
-            case Some("c") | Some("C") | None => None
-            case Some(idx) =>
-                try
-                    Some(ids(idx.toInt - 1))
-                catch {
-                    case _: Throwable =>
-                        if (idx.isEmpty)
-                            warn("Index can't be empty.")
-                        else
-                            warn("Invalid index: " + idx + ".")
-
-                        None
-                }
-        }
-    }
-
-    /**
      * Adds close callback. Added function will be called every time
      * command `close` is called.
      *
@@ -2183,9 +2215,7 @@ object visor extends VisorTag {
      * Disconnects from the grid.
      */
     def close() {
-        if (!isConnected)
-            adviseToConnect()
-        else {
+        if (checkConnected()) {
             if (pool != null) {
                 pool.shutdown()
 
@@ -2320,9 +2350,7 @@ object visor extends VisorTag {
     def log(args: String) {
         assert(args != null)
 
-        if (!isConnected)
-            adviseToConnect()
-        else {
+        if (checkConnected()) {
             def scold(errMsgs: Any*) {
                 assert(errMsgs != null)