Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id BE04518BA4 for ; Mon, 6 Jul 2015 08:50:00 +0000 (UTC) Received: (qmail 44551 invoked by uid 500); 6 Jul 2015 08:50:00 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 44483 invoked by uid 500); 6 Jul 2015 08:50:00 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 44467 invoked by uid 99); 6 Jul 2015 08:50:00 -0000 Received: from Unknown (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 06 Jul 2015 08:50:00 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 39669C2130 for ; Mon, 6 Jul 2015 08:50:00 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.771 X-Spam-Level: * X-Spam-Status: No, score=1.771 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, T_RP_MATCHES_RCVD=-0.01, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-us-west.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id e7DXJxIu4ng5 for ; Mon, 6 Jul 2015 08:49:49 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-us-west.apache.org (ASF Mail Server at mx1-us-west.apache.org) with SMTP id 186EE29520 for ; Mon, 6 Jul 2015 08:49:41 +0000 (UTC) Received: (qmail 43006 invoked by uid 99); 6 Jul 2015 08:49:41 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 06 Jul 2015 08:49:41 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CA540E00CC; Mon, 6 Jul 2015 08:49:40 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.incubator.apache.org Date: Mon, 06 Jul 2015 08:50:07 -0000 Message-Id: In-Reply-To: <60262fe8c46c4de081929dbe6fc643c7@git.apache.org> References: <60262fe8c46c4de081929dbe6fc643c7@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [28/50] incubator-ignite git commit: # ignite-gg-10416 Extract opne command from visor cmd. # ignite-gg-10416 Extract opne command from visor cmd. Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/fcff50e3 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/fcff50e3 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/fcff50e3 Branch: refs/heads/ignite-752 Commit: fcff50e386d7f8e7ab1c00b86921f667d646d819 Parents: ad3e6af Author: Andrey Authored: Thu Jul 2 11:31:31 2015 +0700 Committer: Andrey Committed: Thu Jul 2 13:53:00 2015 +0700 ---------------------------------------------------------------------- .../ignite/visor/commands/VisorConsole.scala | 3 +- .../visor/commands/open/VisorOpenCommand.scala | 319 +++++++++++++++++++ .../scala/org/apache/ignite/visor/visor.scala | 231 +------------- .../ignite/visor/VisorRuntimeBaseSpec.scala | 2 + .../commands/kill/VisorKillCommandSpec.scala | 1 + .../commands/start/VisorStartCommandSpec.scala | 1 + .../commands/tasks/VisorTasksCommandSpec.scala | 1 + .../commands/vvm/VisorVvmCommandSpec.scala | 1 + 8 files changed, 340 insertions(+), 219 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcff50e3/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 d4ac39d..bcfc6e0 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 @@ -71,6 +71,7 @@ class VisorConsole { org.apache.ignite.visor.commands.gc.VisorGcCommand org.apache.ignite.visor.commands.kill.VisorKillCommand org.apache.ignite.visor.commands.node.VisorNodeCommand + org.apache.ignite.visor.commands.open.VisorOpenCommand org.apache.ignite.visor.commands.ping.VisorPingCommand org.apache.ignite.visor.commands.start.VisorStartCommand org.apache.ignite.visor.commands.tasks.VisorTasksCommand @@ -178,7 +179,7 @@ class VisorConsole { buf.append(line.dropRight(1)) } else { - if (buf.size != 0) { + if (buf.nonEmpty) { buf.append(line) line = buf.toString() http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcff50e3/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 new file mode 100644 index 0000000..6498baf --- /dev/null +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala @@ -0,0 +1,319 @@ +/* + * + * * 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.open + +import org.apache.ignite.IgniteSystemProperties._ +import org.apache.ignite.configuration.IgniteConfiguration +import org.apache.ignite.internal.IgniteComponentType._ +import org.apache.ignite.internal.IgniteEx +import org.apache.ignite.internal.util.scala.impl +import org.apache.ignite.internal.util.spring.IgniteSpringHelper +import org.apache.ignite.internal.util.{IgniteUtils => U} +import org.apache.ignite.logger.NullLogger +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi +import org.apache.ignite.visor.commands.common.{VisorConsoleCommand, VisorTextTable} +import org.apache.ignite.visor.visor._ +import org.apache.ignite.visor.{VisorTag, visor} +import org.apache.ignite.{IgniteException, IgniteSystemProperties, Ignition} + +import java.net.URL + +import scala.language.{implicitConversions, reflectiveCalls} + +/** + * ==Overview== + * Contains Visor command `node` implementation. + * + * ==Help== + * {{{ + * +--------------------------------+ + * | node | Prints node statistics. | + * +--------------------------------+ + * }}} + * + * ====Specification==== + * {{{ + * node "{-id8=|-id=} {-a}" + * node + * }}} + * + * ====Arguments==== + * {{{ + * -id8= + * ID8 of node. Either '-id8' or '-id' can be specified. + * If neither specified - command starts in interactive mode. + * -id= + * Full ID of node. Either '-id8' or '-id' can be specified. + * If neither specified - command starts in interactive mode. + * -a + * Print extended information. + * By default - only abbreviated statistics is printed. + * }}} + * + * ====Examples==== + * {{{ + * node + * Starts command in interactive mode. + * node "-id8=12345678" + * Prints statistics for specified node. + * node "-id8=12345678 -a" + * Prints full statistics for specified node. + * }}} + */ +class VisorOpenCommand extends VisorConsoleCommand { + @impl protected val name = "open" + + /** Default configuration path relative to Ignite home. */ + private final val DFLT_CFG = "config/default-config.xml" + + /** + * ==Command== + * Connects Visor console to the default grid. + * + * ==Example== + * open + * Connects to the default grid. + */ + def open() { + open("") + } + + /** + * ==Command== + * Connects Visor console to default or named grid. + * + * ==Examples== + * open -g=mygrid + * Connects to 'mygrid' grid. + * + * @param args Command arguments. + */ + def open(args: String) { + assert(args != null) + + if (isConnected) { + warn("Visor is already connected. Disconnect first.") + + return + } + + try { + def configuration(path: String): IgniteConfiguration = { + assert(path != null) + + val url = + try + new URL(path) + catch { + case e: Exception => + val url = U.resolveIgniteUrl(path) + + if (url == null) + throw new IgniteException("Ignite configuration path is invalid: " + path, e) + + url + } + + // Add no-op logger to remove no-appender warning. + val log4jTup = + if (classOf[Ignition].getClassLoader.getResource("org/apache/log4j/Appender.class") != null) + U.addLog4jNoOpLogger() + else + null + + val spring: IgniteSpringHelper = SPRING.create(false) + + val cfgs = + try + // Cache, IGFS, indexing SPI configurations should be excluded from daemon node config. + spring.loadConfigurations(url, "cacheConfiguration", "fileSystemConfiguration", + "indexingSpi").get1() + finally { + if (log4jTup != null) + U.removeLog4jNoOpLogger(log4jTup) + } + + if (cfgs == null || cfgs.isEmpty) + throw new IgniteException("Can't find grid configuration in: " + url) + + if (cfgs.size > 1) + throw new IgniteException("More than one grid configuration found in: " + url) + + val cfg = cfgs.iterator().next() + + // Setting up 'Config URL' for properly print in console. + System.setProperty(IgniteSystemProperties.IGNITE_CONFIG_URL, url.getPath) + + var cpuCnt = Runtime.getRuntime.availableProcessors + + if (cpuCnt < 4) + cpuCnt = 4 + + cfg.setConnectorConfiguration(null) + + // All thread pools are overridden to have size equal to number of CPUs. + cfg.setPublicThreadPoolSize(cpuCnt) + cfg.setSystemThreadPoolSize(cpuCnt) + cfg.setPeerClassLoadingThreadPoolSize(cpuCnt) + + var ioSpi = cfg.getCommunicationSpi + + if (ioSpi == null) + ioSpi = new TcpCommunicationSpi() + + cfg + } + + val argLst = parseArgs(args) + + val path = argValue("cpath", argLst) + val dflt = hasArgFlag("d", argLst) + + val (cfg, cfgPath) = + if (path.isDefined) + (configuration(path.get), path.get) + else if (dflt) + (configuration(DFLT_CFG), "") + else { + // If configuration file is not defined in arguments, + // ask to choose from the list + askConfigFile() match { + case Some(p) => + nl() + + (VisorTextTable() +=("Using configuration", p)) render() + + nl() + + (configuration(p), p) + case None => + return + } + } + + open(cfg, cfgPath) + } + catch { + case e: IgniteException => + warn(e.getMessage) + warn("Type 'help open' to see how to use this command.") + + status("q") + } + } + + /** + * Connects Visor console to configuration with path. + * + * @param cfg Configuration. + * @param cfgPath Configuration path. + */ + def open(cfg: IgniteConfiguration, cfgPath: String) = { + val daemon = Ignition.isDaemon + + val shutdownHook = IgniteSystemProperties.getString(IGNITE_NO_SHUTDOWN_HOOK, "false") + + // Make sure Visor console starts as daemon node. + Ignition.setDaemon(true) + + // Make sure visor starts without shutdown hook. + System.setProperty(IGNITE_NO_SHUTDOWN_HOOK, "true") + + // Set NullLoger in quite mode. + if ("true".equalsIgnoreCase(sys.props.getOrElse(IGNITE_QUIET, "true"))) + cfg.setGridLogger(new NullLogger) + + val startedGridName = try { + Ignition.start(cfg).name + } + finally { + Ignition.setDaemon(daemon) + + System.setProperty(IGNITE_NO_SHUTDOWN_HOOK, shutdownHook) + } + + ignite = + try + Ignition.ignite(startedGridName).asInstanceOf[IgniteEx] + catch { + case _: IllegalStateException => + throw new IgniteException("Named grid unavailable: " + startedGridName) + } + + visor.open(startedGridName, cfgPath) + } +} + +/** + * Companion object that does initialization of the command. + */ +object VisorOpenCommand { + /** Singleton command. */ + private val cmd = new VisorOpenCommand + + // Adds command's help to visor. + addHelp( + name = "open", + shortInfo = "Connects Visor console to the grid.", + longInfo = Seq( + "Connects Visor console to the grid. Note that P2P class loading", + "should be enabled on all nodes.", + " ", + "If neither '-cpath' or '-d' are provided, command will ask", + "user to select Ignite configuration file in interactive mode." + ), + spec = Seq( + "open -cpath=", + "open -d" + ), + args = Seq( + "-cpath=" -> Seq( + "Ignite configuration path.", + "Can be absolute, relative to Ignite home folder or any well formed URL." + ), + "-d" -> Seq( + "Flag forces the command to connect to grid using default Ignite configuration file.", + "without interactive mode." + ) + ), + examples = Seq( + "open" -> + "Prompts user to select Ignite configuration file in interactive mode.", + "open -d" -> + "Connects Visor console to grid using default Ignite configuration file.", + "open -cpath=/gg/config/mycfg.xml" -> + "Connects Visor console to grid using Ignite configuration from provided file." + ), + emptyArgs = cmd.open, + withArgs = cmd.open + ) + + /** + * Singleton. + */ + def apply() = cmd + + /** + * Implicit converter from visor to commands "pimp". + * + * @param vs Visor tagging trait. + */ + implicit def fromNode2Visor(vs: VisorTag): VisorOpenCommand = cmd +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcff50e3/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 5f63f23..67e3d70 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,23 +17,17 @@ package org.apache.ignite.visor -import org.apache.ignite.IgniteSystemProperties._ import org.apache.ignite._ import org.apache.ignite.cluster.{ClusterGroup, ClusterGroupEmptyException, ClusterMetrics, ClusterNode} -import org.apache.ignite.configuration.IgniteConfiguration import org.apache.ignite.events.EventType._ import org.apache.ignite.events.{DiscoveryEvent, Event} -import org.apache.ignite.internal.IgniteComponentType._ import org.apache.ignite.internal.IgniteEx import org.apache.ignite.internal.IgniteNodeAttributes._ import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException import org.apache.ignite.internal.util.lang.{GridFunc => F} -import org.apache.ignite.internal.util.spring.IgniteSpringHelper 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.logger.NullLogger -import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi import org.apache.ignite.thread.IgniteThreadPoolExecutor import org.apache.ignite.visor.commands.common.VisorTextTable @@ -213,9 +207,6 @@ object visor extends VisorTag { */ private final val DFLT_LOG_PATH = "visor/visor-log" - /** Default configuration path relative to Ignite home. */ - private final val DFLT_CFG = "config/default-config.xml" - /** Log file. */ private var logFile: File = null @@ -429,42 +420,6 @@ object visor extends VisorTag { withArgs = status ) - addHelp( - name = "open", - shortInfo = "Connects Visor console to the grid.", - longInfo = Seq( - "Connects Visor console to the grid. Note that P2P class loading", - "should be enabled on all nodes.", - " ", - "If neither '-cpath' or '-d' are provided, command will ask", - "user to select Ignite configuration file in interactive mode." - ), - spec = Seq( - "open -cpath=", - "open -d" - ), - args = Seq( - "-cpath=" -> Seq( - "Ignite configuration path.", - "Can be absolute, relative to Ignite home folder or any well formed URL." - ), - "-d" -> Seq( - "Flag forces the command to connect to grid using default Ignite configuration file.", - "without interactive mode." - ) - ), - examples = Seq( - "open" -> - "Prompts user to select Ignite configuration file in interactive mode.", - "open -d" -> - "Connects Visor console to grid using default Ignite configuration file.", - "open -cpath=/gg/config/mycfg.xml" -> - "Connects Visor console to grid using Ignite configuration from provided file." - ), - emptyArgs = open, - withArgs = open - ) - /** * @param name - command name. */ @@ -980,7 +935,7 @@ object visor extends VisorTag { val sb = new StringBuilder() - for (i <- 0 until lst.size if lst(i).nonEmpty || sb.size != 0) { + for (i <- 0 until lst.size if lst(i).nonEmpty || sb.nonEmpty) { val arg = sb.toString + lst(i) arg match { @@ -1007,7 +962,7 @@ object visor extends VisorTag { def hasArgValue(@Nullable v: String, args: ArgList): Boolean = { assert(args != null) - args.find(_._2 == v).nonEmpty + args.exists(_._2 == v) } /** @@ -1019,7 +974,7 @@ object visor extends VisorTag { def hasArgName(@Nullable n: String, args: ArgList): Boolean = { assert(args != null) - args.find(_._1 == n).nonEmpty + args.exists(_._1 == n) } /** @@ -1032,7 +987,7 @@ object visor extends VisorTag { def hasArgFlag(n: String, args: ArgList): Boolean = { assert(n != null && args != null) - args.find((a) => a._1 == n && a._2 == null).nonEmpty + args.exists((a) => a._1 == n && a._2 == null) } /** @@ -1525,170 +1480,22 @@ object visor extends VisorTag { private def blank(len: Int) = new String().padTo(len, ' ') /** - * ==Command== - * Connects Visor console to default or named grid. - * - * ==Examples== - * open -g=mygrid - * Connects to 'mygrid' grid. - * - * @param args Command arguments. - */ - def open(args: String) { - assert(args != null) - - if (isConnected) { - warn("Visor is already connected. Disconnect first.") - - return - } - - try { - def configuration(path: String): IgniteConfiguration = { - assert(path != null) - - val url = - try - new URL(path) - catch { - case e: Exception => - val url = U.resolveIgniteUrl(path) - - if (url == null) - throw new IgniteException("Ignite configuration path is invalid: " + path, e) - - url - } - - // Add no-op logger to remove no-appender warning. - val log4jTup = - if (classOf[Ignition].getClassLoader.getResource("org/apache/log4j/Appender.class") != null) - U.addLog4jNoOpLogger() - else - null - - val spring: IgniteSpringHelper = SPRING.create(false) - - val cfgs = - try - // Cache, IGFS, streamer and DR configurations should be excluded from daemon node config. - spring.loadConfigurations(url, "cacheConfiguration", "fileSystemConfiguration", - "streamerConfiguration", "drSenderConfiguration", "drReceiverConfiguration", - "interopConfiguration", "indexingSpi").get1() - finally { - if (log4jTup != null) - U.removeLog4jNoOpLogger(log4jTup) - } - - if (cfgs == null || cfgs.isEmpty) - throw new IgniteException("Can't find grid configuration in: " + url) - - if (cfgs.size > 1) - throw new IgniteException("More than one grid configuration found in: " + url) - - val cfg = cfgs.iterator().next() - - // Setting up 'Config URL' for properly print in console. - System.setProperty(IgniteSystemProperties.IGNITE_CONFIG_URL, url.getPath) - - var cpuCnt = Runtime.getRuntime.availableProcessors - - if (cpuCnt < 4) - cpuCnt = 4 - - cfg.setConnectorConfiguration(null) - - // All thread pools are overridden to have size equal to number of CPUs. - cfg.setPublicThreadPoolSize(cpuCnt) - cfg.setSystemThreadPoolSize(cpuCnt) - cfg.setPeerClassLoadingThreadPoolSize(cpuCnt) - - var ioSpi = cfg.getCommunicationSpi - - if (ioSpi == null) - ioSpi = new TcpCommunicationSpi() - - cfg - } - - val argLst = parseArgs(args) - - val path = argValue("cpath", argLst) - val dflt = hasArgFlag("d", argLst) - - val (cfg, cfgPath) = - if (path.isDefined) - (configuration(path.get), path.get) - else if (dflt) - (configuration(DFLT_CFG), "") - else { - // If configuration file is not defined in arguments, - // ask to choose from the list - askConfigFile() match { - case Some(p) => - nl() - - (VisorTextTable() +=("Using configuration", p)) render() - - nl() - - (configuration(p), p) - case None => - return - } - } - - open(cfg, cfgPath) - } - catch { - case e: IgniteException => - warn(e.getMessage) - warn("Type 'help open' to see how to use this command.") - - status("q") - } - } - - /** * Connects Visor console to configuration with path. * - * @param cfg Configuration. + * @param gridName Name of grid instance. * @param cfgPath Configuration path. */ - def open(cfg: IgniteConfiguration, cfgPath: String) { - val daemon = Ignition.isDaemon - - val shutdownHook = IgniteSystemProperties.getString(IGNITE_NO_SHUTDOWN_HOOK, "false") - - // Make sure Visor console starts as daemon node. - Ignition.setDaemon(true) - - // Make sure visor starts without shutdown hook. - System.setProperty(IGNITE_NO_SHUTDOWN_HOOK, "true") - - // Set NullLoger in quite mode. - if ("true".equalsIgnoreCase(sys.props.getOrElse(IGNITE_QUIET, "true"))) - cfg.setGridLogger(new NullLogger) - - val startedGridName = try { - Ignition.start(cfg).name - } - finally { - Ignition.setDaemon(daemon) - - System.setProperty(IGNITE_NO_SHUTDOWN_HOOK, shutdownHook) - } - + def open(gridName: String, cfgPath: String) { this.cfgPath = cfgPath ignite = try - Ignition.ignite(startedGridName).asInstanceOf[IgniteEx] + Ignition.ignite(gridName).asInstanceOf[IgniteEx] catch { case _: IllegalStateException => this.cfgPath = null - throw new IgniteException("Named grid unavailable: " + startedGridName) + throw new IgniteException("Named grid unavailable: " + gridName) } assert(cfgPath != null) @@ -1824,18 +1631,6 @@ object visor extends VisorTag { } /** - * ==Command== - * Connects Visor console to the default grid. - * - * ==Example== - * open - * Connects to the default grid. - */ - def open() { - open("") - } - - /** * Returns string with node id8, its memory variable, if available, and its * IP address (first internal address), if node is alive. * @@ -2016,7 +1811,7 @@ object visor extends VisorTag { else if (nodes.size == 1) Some(nodes.head.id) else { - (0 until nodes.size) foreach (i => { + nodes.indices foreach (i => { val n = nodes(i) val m = n.metrics @@ -2080,7 +1875,7 @@ object visor extends VisorTag { None } else { - (0 until neighborhood.size) foreach (i => { + neighborhood.indices foreach (i => { val neighbors = neighborhood(i) var ips = immutable.Set.empty[String] @@ -2229,7 +2024,7 @@ object visor extends VisorTag { val ids = ignite.cluster.forRemotes().nodes().map(nid8).toList - (0 until ids.size).foreach(i => println((i + 1) + ": " + ids(i))) + ids.indices.foreach(i => println((i + 1) + ": " + ids(i))) nl() @@ -2758,7 +2553,7 @@ object visor extends VisorTag { help() } - lazy val commands = cmdLst.map(_.name) ++ cmdLst.map(_.aliases).flatten + lazy val commands = cmdLst.map(_.name) ++ cmdLst.flatMap(_.aliases) def searchCmd(cmd: String) = cmdLst.find(c => c.name.equals(cmd) || (c.aliases != null && c.aliases.contains(cmd))) @@ -2851,7 +2646,7 @@ object visor extends VisorTag { var dec = BigDecimal.valueOf(0L) - for (i <- 0 until octets.length) dec += octets(i).toLong * math.pow(256, octets.length - 1 - i).toLong + for (i <- octets.indices) dec += octets(i).toLong * math.pow(256, octets.length - 1 - i).toLong dec } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcff50e3/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala ---------------------------------------------------------------------- diff --git a/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala b/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala index f27bae3..1c5e232 100644 --- a/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala +++ b/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala @@ -19,6 +19,8 @@ package org.apache.ignite.visor import org.apache.ignite.Ignition import org.apache.ignite.configuration.IgniteConfiguration +import org.apache.ignite.visor.commands.open.VisorOpenCommand._ + import org.scalatest._ /** http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcff50e3/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/kill/VisorKillCommandSpec.scala ---------------------------------------------------------------------- diff --git a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/kill/VisorKillCommandSpec.scala b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/kill/VisorKillCommandSpec.scala index 2c659b5..4719606 100644 --- a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/kill/VisorKillCommandSpec.scala +++ b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/kill/VisorKillCommandSpec.scala @@ -20,6 +20,7 @@ package org.apache.ignite.visor.commands.kill import org.apache.ignite.visor.visor import org.scalatest._ +import org.apache.ignite.visor.commands.open.VisorOpenCommand._ import org.apache.ignite.visor.commands.kill.VisorKillCommand._ /** http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcff50e3/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/start/VisorStartCommandSpec.scala ---------------------------------------------------------------------- diff --git a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/start/VisorStartCommandSpec.scala b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/start/VisorStartCommandSpec.scala index c6404b5..49a861c 100644 --- a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/start/VisorStartCommandSpec.scala +++ b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/start/VisorStartCommandSpec.scala @@ -20,6 +20,7 @@ package org.apache.ignite.visor.commands.start import org.apache.ignite.visor.visor import org.scalatest._ +import org.apache.ignite.visor.commands.open.VisorOpenCommand._ import org.apache.ignite.visor.commands.start.VisorStartCommand._ import org.apache.ignite.visor.commands.top.VisorTopologyCommand._ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcff50e3/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommandSpec.scala ---------------------------------------------------------------------- diff --git a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommandSpec.scala b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommandSpec.scala index db07543..fe364bc 100644 --- a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommandSpec.scala +++ b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/tasks/VisorTasksCommandSpec.scala @@ -26,6 +26,7 @@ import org.scalatest._ import java.util +import org.apache.ignite.visor.commands.open.VisorOpenCommand._ import org.apache.ignite.visor.commands.tasks.VisorTasksCommand._ import scala.collection.JavaConversions._ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fcff50e3/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommandSpec.scala ---------------------------------------------------------------------- diff --git a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommandSpec.scala b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommandSpec.scala index 1a4bc3e..022f6d6 100644 --- a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommandSpec.scala +++ b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/vvm/VisorVvmCommandSpec.scala @@ -20,6 +20,7 @@ package org.apache.ignite.visor.commands.vvm import org.apache.ignite.visor.visor import org.scalatest._ +import org.apache.ignite.visor.commands.open.VisorOpenCommand._ import org.apache.ignite.visor.commands.vvm.VisorVvmCommand._ /**