[GitHub] incubator-gearpump pull request #134: [GEARPUMP-252] return application stat...

2017-01-23 Thread huafengw
Github user huafengw commented on a diff in the pull request:

https://github.com/apache/incubator-gearpump/pull/134#discussion_r97287804
  
--- Diff: 
core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala 
---
@@ -18,17 +18,21 @@
 package org.apache.gearpump.cluster.client
 
 import akka.pattern.ask
-import akka.actor.ActorRef
+import akka.actor.{Actor, ActorRef, ActorSystem, Props}
 import akka.util.Timeout
-import org.apache.gearpump.cluster.ClientToMaster.{ResolveAppId, 
ShutdownApplication}
-import org.apache.gearpump.cluster.MasterToClient.{ResolveAppIdResult, 
ShutdownApplicationResult}
+import 
org.apache.gearpump.cluster.ClientToMaster.{RegisterAppResultListener, 
ResolveAppId, ShutdownApplication}
+import org.apache.gearpump.cluster.MasterToClient._
+import 
org.apache.gearpump.cluster.client.RunningApplication.{AppResultListener, 
WaitUntilFinish}
 import org.apache.gearpump.util.ActorUtil
+import org.apache.gearpump.cluster.client.RunningApplication._
 
+import scala.concurrent.duration._
 import scala.concurrent.Future
 import scala.util.{Failure, Success}
 import scala.concurrent.ExecutionContext.Implicits.global
 
-class RunningApplication(val appId: Int, master: ActorRef, timeout: 
Timeout) {
+class RunningApplication(val appId: Int, master: ActorRef, timeout: 
Timeout,
--- End diff --

We can do it in following pr.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-gearpump pull request #134: [GEARPUMP-252] return application stat...

2017-01-23 Thread manuzhang
Github user manuzhang commented on a diff in the pull request:

https://github.com/apache/incubator-gearpump/pull/134#discussion_r97293869
  
--- Diff: 
core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala 
---
@@ -18,17 +18,21 @@
 package org.apache.gearpump.cluster.client
 
 import akka.pattern.ask
-import akka.actor.ActorRef
+import akka.actor.{Actor, ActorRef, ActorSystem, Props}
 import akka.util.Timeout
-import org.apache.gearpump.cluster.ClientToMaster.{ResolveAppId, 
ShutdownApplication}
-import org.apache.gearpump.cluster.MasterToClient.{ResolveAppIdResult, 
ShutdownApplicationResult}
+import 
org.apache.gearpump.cluster.ClientToMaster.{RegisterAppResultListener, 
ResolveAppId, ShutdownApplication}
+import org.apache.gearpump.cluster.MasterToClient._
+import 
org.apache.gearpump.cluster.client.RunningApplication.{AppResultListener, 
WaitUntilFinish}
 import org.apache.gearpump.util.ActorUtil
+import org.apache.gearpump.cluster.client.RunningApplication._
 
+import scala.concurrent.duration._
 import scala.concurrent.Future
 import scala.util.{Failure, Success}
 import scala.concurrent.ExecutionContext.Implicits.global
 
-class RunningApplication(val appId: Int, master: ActorRef, timeout: 
Timeout) {
+class RunningApplication(val appId: Int, master: ActorRef, timeout: 
Timeout,
--- End diff --

sure


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-gearpump pull request #134: [GEARPUMP-252] return application stat...

2017-01-23 Thread huafengw
Github user huafengw commented on a diff in the pull request:

https://github.com/apache/incubator-gearpump/pull/134#discussion_r97287682
  
--- Diff: 
core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala 
---
@@ -40,6 +44,16 @@ class RunningApplication(val appId: Int, master: 
ActorRef, timeout: Timeout) {
 }
   }
 
+  def waitUnilFinish(): Unit = {
+val delegator = system.actorOf(Props(new AppResultListener(appId, 
master)))
+val result = ActorUtil.askActor[ApplicationResult](delegator, 
WaitUntilFinish, INF_TIMEOUT)
+result match {
+  case failed: ApplicationFailed =>
+throw failed.error
+  case _ =>
+}
+  }
+
   def askAppMaster[T](msg: Any): Future[T] = {
--- End diff --

The ActorUtil's askAppMaster will first `ResolveAppId` which is unnecessary 
here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-gearpump pull request #134: [GEARPUMP-252] return application stat...

2017-01-23 Thread manuzhang
Github user manuzhang commented on a diff in the pull request:

https://github.com/apache/incubator-gearpump/pull/134#discussion_r97273193
  
--- Diff: 
core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala 
---
@@ -18,17 +18,21 @@
 package org.apache.gearpump.cluster.client
 
 import akka.pattern.ask
-import akka.actor.ActorRef
+import akka.actor.{Actor, ActorRef, ActorSystem, Props}
 import akka.util.Timeout
-import org.apache.gearpump.cluster.ClientToMaster.{ResolveAppId, 
ShutdownApplication}
-import org.apache.gearpump.cluster.MasterToClient.{ResolveAppIdResult, 
ShutdownApplicationResult}
+import 
org.apache.gearpump.cluster.ClientToMaster.{RegisterAppResultListener, 
ResolveAppId, ShutdownApplication}
+import org.apache.gearpump.cluster.MasterToClient._
+import 
org.apache.gearpump.cluster.client.RunningApplication.{AppResultListener, 
WaitUntilFinish}
 import org.apache.gearpump.util.ActorUtil
+import org.apache.gearpump.cluster.client.RunningApplication._
 
+import scala.concurrent.duration._
 import scala.concurrent.Future
 import scala.util.{Failure, Success}
 import scala.concurrent.ExecutionContext.Implicits.global
 
-class RunningApplication(val appId: Int, master: ActorRef, timeout: 
Timeout) {
+class RunningApplication(val appId: Int, master: ActorRef, timeout: 
Timeout,
--- End diff --

I think we need a user interface here. `RunningApplication` could be the 
default implementation. Plus, please add guide on how to use the interface. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-gearpump pull request #134: [GEARPUMP-252] return application stat...

2017-01-23 Thread manuzhang
Github user manuzhang commented on a diff in the pull request:

https://github.com/apache/incubator-gearpump/pull/134#discussion_r97272651
  
--- Diff: 
core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala 
---
@@ -40,6 +44,16 @@ class RunningApplication(val appId: Int, master: 
ActorRef, timeout: Timeout) {
 }
   }
 
+  def waitUnilFinish(): Unit = {
+val delegator = system.actorOf(Props(new AppResultListener(appId, 
master)))
+val result = ActorUtil.askActor[ApplicationResult](delegator, 
WaitUntilFinish, INF_TIMEOUT)
+result match {
+  case failed: ApplicationFailed =>
+throw failed.error
+  case _ =>
+}
+  }
+
   def askAppMaster[T](msg: Any): Future[T] = {
--- End diff --

We already have one in `ActorUtil`. Why not reuse it ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-gearpump pull request #134: [GEARPUMP-252] return application stat...

2017-01-23 Thread manuzhang
Github user manuzhang commented on a diff in the pull request:

https://github.com/apache/incubator-gearpump/pull/134#discussion_r97268263
  
--- Diff: 
core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala 
---
@@ -40,6 +44,16 @@ class RunningApplication(val appId: Int, master: 
ActorRef, timeout: Timeout) {
 }
   }
 
+  def waitUnilFinish(): Unit = {
--- End diff --

typo. `Unil` -> `Until`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-gearpump pull request #134: [GEARPUMP-252] return application stat...

2017-01-23 Thread manuzhang
Github user manuzhang commented on a diff in the pull request:

https://github.com/apache/incubator-gearpump/pull/134#discussion_r97263936
  
--- Diff: 
core/src/main/scala/org/apache/gearpump/cluster/client/RunningApplication.scala 
---
@@ -40,6 +44,16 @@ class RunningApplication(val appId: Int, master: 
ActorRef, timeout: Timeout) {
 }
   }
 
+  def waitUnilFinish(): Unit = {
+val delegator = system.actorOf(Props(new AppResultListener(appId, 
master)))
+val result = ActorUtil.askActor[ApplicationResult](delegator, 
WaitUntilFinish, INF_TIMEOUT)
+result match {
+  case failed: ApplicationFailed =>
+throw failed.error
+  case _ =>
--- End diff --

Better add some warning log for unmatched cases


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-gearpump pull request #134: [GEARPUMP-252] return application stat...

2017-01-23 Thread manuzhang
Github user manuzhang commented on a diff in the pull request:

https://github.com/apache/incubator-gearpump/pull/134#discussion_r97263412
  
--- Diff: 
core/src/main/scala/org/apache/gearpump/cluster/AppDescription.scala ---
@@ -142,4 +139,26 @@ case class ExecutorContext(
 case class ExecutorJVMConfig(
 classPath: Array[String], jvmArguments: Array[String], mainClass: 
String,
 arguments: Array[String], jar: Option[AppJar], username: String,
-executorAkkaConfig: Config = ConfigFactory.empty())
\ No newline at end of file
+executorAkkaConfig: Config = ConfigFactory.empty())
+
+sealed abstract class ApplicationStatus(val status: String)
+  extends Serializable{
+  override def toString: String = status
+}
+
+sealed abstract class ApplicationTerminalStatus(override val status: 
String)
+  extends ApplicationStatus(status)
+
+object ApplicationStatus {
+  case object PENDING extends ApplicationStatus("pending")
+
+  case object ACTIVE extends ApplicationStatus("active")
+
+  case object SUCCEEDED extends ApplicationTerminalStatus("succeeded")
+
+  case object FAILED extends ApplicationTerminalStatus("failed")
+
+  case object TERMINATED extends ApplicationTerminalStatus("terminated")
+
+  case object NOEXIST extends ApplicationStatus("nonexist")
--- End diff --

`noexist` or `nonexist` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] incubator-gearpump pull request #134: [GEARPUMP-252] return application stat...

2017-01-23 Thread manuzhang
Github user manuzhang commented on a diff in the pull request:

https://github.com/apache/incubator-gearpump/pull/134#discussion_r97263634
  
--- Diff: 
core/src/main/scala/org/apache/gearpump/cluster/appmaster/ApplicationRuntimeInfo.scala
 ---
@@ -0,0 +1,52 @@
+/*
+ * 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.cluster.appmaster
+
+import akka.actor.ActorRef
+import com.typesafe.config.{Config, ConfigFactory}
+import org.apache.gearpump.TimeStamp
+import org.apache.gearpump.cluster.{ApplicationStatus, 
ApplicationTerminalStatus}
+
+/** Run time info of Application */
+case class ApplicationRuntimeInfo(
+appId: Int,
+// AppName is the unique Id for an application
+appName: String,
+appMaster: ActorRef = ActorRef.noSender,
+worker: ActorRef = ActorRef.noSender,
+user: String = "",
+submissionTime: TimeStamp = 0,
+startTime: TimeStamp = 0,
+finishTime: TimeStamp = 0,
+config: Config = ConfigFactory.empty(),
+status: ApplicationStatus = ApplicationStatus.NOEXIST) {
+
+  def onAppMasterRegistered(appMaster: ActorRef, worker: ActorRef): 
ApplicationRuntimeInfo = {
+this.copy(appMaster = appMaster, worker = worker)
+  }
+
+  def onAppMasterActivated(timeStamp: TimeStamp): ApplicationRuntimeInfo = 
{
+this.copy(startTime = timeStamp, status = ApplicationStatus.ACTIVE)
+  }
+
+  def onTerminalStatus(timeStamp: TimeStamp, finalStatus: 
ApplicationTerminalStatus):
--- End diff --

`terminal` or `final` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---