[GEARPUMP-368] Add dedicated trait for commands

Author: karol-brejna-i <karol.brejna@intel.com>

Closes #241 from karol-brejna-i/GEARPUMP-368_submit_app_via_ui.
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/main/AppSubmitter.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/AppSubmitter.scala
index 508448f..defd86e 100644
--- a/core/src/main/scala/org/apache/gearpump/cluster/main/AppSubmitter.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/AppSubmitter.scala
@@ -21,12 +21,13 @@
 import java.net.{URL, URLClassLoader}
 import java.util.jar.JarFile
 
-import org.apache.gearpump.util.{AkkaApp, Constants, LogUtil, Util}
+import org.apache.gearpump.cluster.client.RuntimeEnvironment
+import org.apache.gearpump.util.{Constants, LogUtil, MasterClientCommand, Util}
 
 import scala.util.{Failure, Success, Try}
 
 /** Tool to submit an application jar to cluster */
-object AppSubmitter extends AkkaApp with ArgumentsParser {
+object AppSubmitter extends MasterClientCommand with ArgumentsParser {
 
   override val ignoreUnknownArgument = true
 
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/main/Gear.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/Gear.scala
index 7301819..7d6181f 100644
--- a/core/src/main/scala/org/apache/gearpump/cluster/main/Gear.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/Gear.scala
@@ -39,9 +39,7 @@
   private def executeCommand(command: String, commandArgs: Array[String]) = {
     commands.get(command) match {
       case Some(runner) =>
-        val akkaConfig = ClusterConfig.default()
-        LogUtil.loadConfiguration(akkaConfig, ProcessType.CLIENT)
-        runner.main(akkaConfig, commandArgs)
+        runner.main(commandArgs)
       case None =>
         printUsage()
     }
@@ -54,8 +52,6 @@
       System.setProperty(Constants.GEARPUMP_CUSTOM_CONFIG_FILE, configFile)
     }
 
-    RuntimeEnvironment.setRuntimeEnv(new RemoteRuntimeEnvironment)
-
     if (args.length == 0) {
       printUsage()
     } else {
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/main/Info.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/Info.scala
index fa2d429..8f8290c 100644
--- a/core/src/main/scala/org/apache/gearpump/cluster/main/Info.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/Info.scala
@@ -19,10 +19,10 @@
 
 import org.apache.gearpump.cluster.MasterToAppMaster.AppMastersData
 import org.apache.gearpump.cluster.client.ClientContext
-import org.apache.gearpump.util.AkkaApp
+import org.apache.gearpump.util.MasterClientCommand
 
 /** Tool to query master info */
-object Info extends AkkaApp with ArgumentsParser {
+object Info extends MasterClientCommand with ArgumentsParser {
 
   override val options: Array[(String, CLIOption[Any])] = Array(
     // For document purpose only, OPTION_CONFIG option is not used here.
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/main/Kill.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/Kill.scala
index d5a3520..d7016a3 100644
--- a/core/src/main/scala/org/apache/gearpump/cluster/main/Kill.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/Kill.scala
@@ -19,10 +19,10 @@
 package org.apache.gearpump.cluster.main
 
 import org.apache.gearpump.cluster.client.ClientContext
-import org.apache.gearpump.util.AkkaApp
+import org.apache.gearpump.util.MasterClientCommand
 
 /** Tool to kill an App */
-object Kill extends AkkaApp with ArgumentsParser {
+object Kill extends MasterClientCommand with ArgumentsParser {
 
   override val options: Array[(String, CLIOption[Any])] = Array(
     "appid" -> CLIOption("<application id>", required = true),
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/main/Local.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/Local.scala
index db2cd8a..64a26c4 100644
--- a/core/src/main/scala/org/apache/gearpump/cluster/main/Local.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/Local.scala
@@ -25,14 +25,14 @@
 import org.apache.gearpump.cluster.worker.{Worker => WorkerActor}
 import org.apache.gearpump.util.Constants._
 import org.apache.gearpump.util.LogUtil.ProcessType
-import org.apache.gearpump.util.{ActorUtil, AkkaApp, Constants, LogUtil, Util}
+import org.apache.gearpump.util.{ActorUtil, Constants, LogUtil, MasterClientCommand, Util}
 import org.slf4j.Logger
 
 import scala.collection.JavaConverters._
 import scala.concurrent.Await
 import scala.concurrent.duration.Duration
 
-object Local extends AkkaApp with ArgumentsParser {
+object Local extends MasterClientCommand with ArgumentsParser {
   override def akkaConfig: Config = ClusterConfig.master()
 
   var LOG: Logger = LogUtil.getLogger(getClass)
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/main/MainRunner.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/MainRunner.scala
index 11b7239..acc8e56 100644
--- a/core/src/main/scala/org/apache/gearpump/cluster/main/MainRunner.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/MainRunner.scala
@@ -18,10 +18,10 @@
 
 package org.apache.gearpump.cluster.main
 
-import org.apache.gearpump.util.AkkaApp
+import org.apache.gearpump.util.MasterClientCommand
 
 /** Tool to run any main class by providing a jar */
-object MainRunner extends AkkaApp with ArgumentsParser {
+object MainRunner extends MasterClientCommand with ArgumentsParser {
 
   override val options: Array[(String, CLIOption[Any])] = Array(
     // For document purpose only, OPTION_CONFIG option is not used here.
diff --git a/core/src/main/scala/org/apache/gearpump/cluster/main/Replay.scala b/core/src/main/scala/org/apache/gearpump/cluster/main/Replay.scala
index 8c2d7ef..3a9cac1 100644
--- a/core/src/main/scala/org/apache/gearpump/cluster/main/Replay.scala
+++ b/core/src/main/scala/org/apache/gearpump/cluster/main/Replay.scala
@@ -18,10 +18,10 @@
 package org.apache.gearpump.cluster.main
 
 import org.apache.gearpump.cluster.client.ClientContext
-import org.apache.gearpump.util.AkkaApp
+import org.apache.gearpump.util.MasterClientCommand
 
 // Internal tool to restart an application
-object Replay extends AkkaApp with ArgumentsParser {
+object Replay extends MasterClientCommand with ArgumentsParser {
 
   override val options: Array[(String, CLIOption[Any])] = Array(
     "appid" -> CLIOption("<application id>", required = true),
diff --git a/core/src/main/scala/org/apache/gearpump/util/MasterClientCommand.scala b/core/src/main/scala/org/apache/gearpump/util/MasterClientCommand.scala
new file mode 100644
index 0000000..4d46eb5
--- /dev/null
+++ b/core/src/main/scala/org/apache/gearpump/util/MasterClientCommand.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.gearpump.util
+
+import org.apache.gearpump.cluster.client.{RemoteRuntimeEnvironment, RuntimeEnvironment}
+import org.apache.gearpump.util.LogUtil.ProcessType
+
+import scala.util.Try
+
+trait MasterClientCommand extends AkkaApp {
+
+  override def main(args: Array[String]): Unit = {
+    RuntimeEnvironment.setRuntimeEnv(new RemoteRuntimeEnvironment)
+    LogUtil.loadConfiguration(akkaConfig, ProcessType.CLIENT)
+
+    Try {
+      main(akkaConfig, args)
+    }.failed.foreach { ex => help(); throw ex }
+  }
+
+}
diff --git a/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/main/GearpumpNimbus.scala b/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/main/GearpumpNimbus.scala
index c66159e..1a0b9ff 100644
--- a/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/main/GearpumpNimbus.scala
+++ b/experiments/storm/src/main/scala/org/apache/gearpump/experiments/storm/main/GearpumpNimbus.scala
@@ -42,9 +42,9 @@
 import org.apache.gearpump.experiments.storm.util.TimeCacheMapWrapper.Callback
 import org.apache.gearpump.experiments.storm.util.{GraphBuilder, StormConstants, StormUtil, TimeCacheMapWrapper}
 import org.apache.gearpump.streaming.StreamApplication
-import org.apache.gearpump.util.{AkkaApp, Constants, LogUtil}
+import org.apache.gearpump.util.{Constants, LogUtil, MasterClientCommand}
 
-object GearpumpNimbus extends AkkaApp with ArgumentsParser {
+object GearpumpNimbus extends MasterClientCommand with ArgumentsParser {
   private val THRIFT_PORT = StormUtil.getThriftPort()
   private val OUTPUT = "output"
   private val LOG: Logger = LogUtil.getLogger(classOf[GearpumpNimbus])