This is an automated email from the ASF dual-hosted git repository.

markusthoemmes pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-openwhisk.git


The following commit(s) were added to refs/heads/master by this push:
     new 31c5f21  Add LogStore interface.  (#2695)
31c5f21 is described below

commit 31c5f21e473b98fde4021083d3cef388ad206bf6
Author: tysonnorris <tysonnor...@gmail.com>
AuthorDate: Fri Nov 3 05:37:58 2017 -0700

    Add LogStore interface.  (#2695)
    
    Adds an interface to gather logs after the activation, define their way of 
storage and fetch them from that storage upon user request.
    
    Lifecycle wise, log-handling runs through two steps:
    1. Collecting logs after an activation has run to store them in the 
database.
    2. Fetching logs from the API to use them (as a user).
    
    This interface facilitates writing exchangeable pieces to both store logs 
in a different location and/or fetch them from some different system.
---
 common/scala/src/main/resources/reference.conf     |  1 +
 .../containerpool/logging/DockerLogStore.scala     | 52 +++++++++++++++
 .../core/containerpool/logging/LogStore.scala      | 76 ++++++++++++++++++++++
 .../scala/whisk/core/entity/ActivationLogs.scala   | 29 ++-------
 .../scala/whisk/core/controller/Activations.scala  |  9 ++-
 .../scala/whisk/core/controller/ApiUtils.scala     |  2 +-
 .../scala/whisk/core/controller/Controller.scala   |  3 +
 .../scala/whisk/core/controller/RestAPIs.scala     |  8 ++-
 .../whisk/core/containerpool/ContainerProxy.scala  |  8 ++-
 .../scala/whisk/core/invoker/InvokerReactive.scala |  7 +-
 .../containerpool/test/ContainerProxyTests.scala   | 39 +++++++----
 .../controller/test/ControllerTestCommon.scala     |  3 +
 12 files changed, 191 insertions(+), 46 deletions(-)

diff --git a/common/scala/src/main/resources/reference.conf 
b/common/scala/src/main/resources/reference.conf
index 50a36a5..bf2e694 100644
--- a/common/scala/src/main/resources/reference.conf
+++ b/common/scala/src/main/resources/reference.conf
@@ -2,4 +2,5 @@ whisk.spi{
   ArtifactStoreProvider = whisk.core.database.CouchDbStoreProvider
   MessagingProvider = whisk.connector.kafka.KafkaMessagingProvider
   ContainerFactoryProvider = 
whisk.core.containerpool.docker.DockerContainerFactoryProvider
+  LogStoreProvider = whisk.core.containerpool.logging.DockerLogStoreProvider
 }
\ No newline at end of file
diff --git 
a/common/scala/src/main/scala/whisk/core/containerpool/logging/DockerLogStore.scala
 
b/common/scala/src/main/scala/whisk/core/containerpool/logging/DockerLogStore.scala
new file mode 100644
index 0000000..ed93934
--- /dev/null
+++ 
b/common/scala/src/main/scala/whisk/core/containerpool/logging/DockerLogStore.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 whisk.core.containerpool.logging
+
+import akka.actor.ActorSystem
+import whisk.common.TransactionId
+import whisk.core.containerpool.Container
+import whisk.core.entity.{ActivationLogs, ExecutableWhiskAction, 
WhiskActivation}
+
+import scala.concurrent.{ExecutionContext, Future}
+
+/**
+ * Docker based implementation of a LogStore.
+ *
+ * Relies on docker's implementation details with regards to the JSON 
log-driver. When using the JSON log-driver
+ * docker writes stdout/stderr to a JSON formatted file which is read by this 
store. Logs are written in the
+ * activation record itself and thus stored in CouchDB.
+ */
+class DockerLogStore(system: ActorSystem) extends LogStore {
+  implicit val ec: ExecutionContext = system.dispatcher
+
+  /* "json-file" is the log-driver that writes out to file */
+  override val containerParameters = Map("--log-driver" -> Set("json-file"))
+
+  /* As logs are already part of the activation record, just return that bit 
of it */
+  override def fetchLogs(activation: WhiskActivation): Future[ActivationLogs] 
= Future.successful(activation.logs)
+
+  override def collectLogs(transid: TransactionId,
+                           container: Container,
+                           action: ExecutableWhiskAction): 
Future[ActivationLogs] = {
+    container.logs(action.limits.logs.asMegaBytes, 
action.exec.sentinelledLogs)(transid).map(ActivationLogs(_))
+  }
+}
+
+object DockerLogStoreProvider extends LogStoreProvider {
+  override def logStore(actorSystem: ActorSystem): LogStore = new 
DockerLogStore(actorSystem)
+}
diff --git 
a/common/scala/src/main/scala/whisk/core/containerpool/logging/LogStore.scala 
b/common/scala/src/main/scala/whisk/core/containerpool/logging/LogStore.scala
new file mode 100644
index 0000000..bb7f8f5
--- /dev/null
+++ 
b/common/scala/src/main/scala/whisk/core/containerpool/logging/LogStore.scala
@@ -0,0 +1,76 @@
+/*
+ * 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 whisk.core.containerpool.logging
+
+import akka.actor.ActorSystem
+import whisk.common.TransactionId
+import whisk.core.containerpool.Container
+import whisk.core.entity.{ActivationLogs, ExecutableWhiskAction, 
WhiskActivation}
+import whisk.spi.Spi
+
+import scala.concurrent.Future
+
+/**
+ * Interface to gather logs after the activation, define their way of storage 
and fetch them from that storage upon
+ * user request.
+ *
+ * Lifecycle wise, log-handling runs through two steps:
+ * 1. Collecting logs after an activation has run to store them in the 
database.
+ * 2. Fetching logs from the API to use them (as a user).
+ *
+ * Both of those lifecycle steps can independently implemented via {@code 
collectLogs} and {@code fetchLogs}
+ * respectively.
+ *
+ * The implementation can choose to not fetch logs at all but use the 
underlying container orchestrator to handle
+ * log-storage/forwarding (via {@code containerParameters}). In this case, 
{@code collectLogs} would be implemented
+ * as a stub, only returning a line of log hinting that logs are stored 
elsewhere. {@code fetchLogs} though can
+ * implement the API of the backend system to be able to still show the logs 
of a specific activation via the OpenWhisk
+ * API.
+ */
+trait LogStore {
+
+  /** Additional parameters to pass to container creation */
+  def containerParameters: Map[String, Set[String]]
+
+  /**
+   * Collect logs after the activation has finished.
+   *
+   * This method is called after an activation has finished. The logs gathered 
here are stored along the activation
+   * record in the database.
+   *
+   * @param transid transaction the activation ran in
+   * @param container container used by the activation
+   * @param action action that was activated
+   * @return logs for the given activation
+   */
+  def collectLogs(transid: TransactionId, container: Container, action: 
ExecutableWhiskAction): Future[ActivationLogs]
+
+  /**
+   * Fetch relevant logs for the given activation from the store.
+   *
+   * This method is called when a user requests logs via the API.
+   *
+   * @param activation activation to fetch the logs for
+   * @return the relevant logs
+   */
+  def fetchLogs(activation: WhiskActivation): Future[ActivationLogs]
+}
+
+trait LogStoreProvider extends Spi {
+  def logStore(actorSystem: ActorSystem): LogStore
+}
diff --git a/common/scala/src/main/scala/whisk/core/entity/ActivationLogs.scala 
b/common/scala/src/main/scala/whisk/core/entity/ActivationLogs.scala
index e2c9e07..bc0b6d1 100644
--- a/common/scala/src/main/scala/whisk/core/entity/ActivationLogs.scala
+++ b/common/scala/src/main/scala/whisk/core/entity/ActivationLogs.scala
@@ -17,36 +17,19 @@
 
 package whisk.core.entity
 
-import scala.Vector
-import scala.util.Try
+import spray.json._
+import spray.json.DefaultJsonProtocol._
 
-import spray.json.DefaultJsonProtocol.StringJsonFormat
-import spray.json.JsArray
-import spray.json.JsObject
-import spray.json.JsString
-import spray.json.JsValue
-import spray.json.RootJsonFormat
-import spray.json.deserializationError
-import spray.json.pimpAny
-
-protected[core] case class ActivationLogs(val logs: Vector[String] = Vector()) 
extends AnyVal {
+protected[core] case class ActivationLogs(logs: Vector[String] = Vector.empty) 
extends AnyVal {
   def toJsonObject = JsObject("logs" -> toJson)
-  def toJson = JsArray(logs map { _.toJson })
+  def toJson = logs.toJson
 
-  override def toString = logs mkString ("[", ", ", "]")
+  override def toString = logs.mkString("[", ", ", "]")
 }
 
 protected[core] object ActivationLogs {
   protected[core] implicit val serdes = new RootJsonFormat[ActivationLogs] {
     def write(l: ActivationLogs) = l.toJson
-
-    def read(value: JsValue) =
-      Try {
-        val JsArray(logs) = value
-        ActivationLogs(logs map {
-          case JsString(s) => s
-          case _           => deserializationError("activation logs malformed")
-        })
-      } getOrElse deserializationError("activation logs malformed")
+    def read(value: JsValue) = ActivationLogs(value.convertTo[Vector[String]])
   }
 }
diff --git 
a/core/controller/src/main/scala/whisk/core/controller/Activations.scala 
b/core/controller/src/main/scala/whisk/core/controller/Activations.scala
index 284b7c4..0d7dbbc 100644
--- a/core/controller/src/main/scala/whisk/core/controller/Activations.scala
+++ b/core/controller/src/main/scala/whisk/core/controller/Activations.scala
@@ -27,10 +27,12 @@ import 
akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport.sprayJsonMarsha
 import akka.http.scaladsl.model.StatusCodes.BadRequest
 import akka.http.scaladsl.server.Directives
 import akka.http.scaladsl.unmarshalling._
+import scala.concurrent.Future
 import spray.json._
 import spray.json.DefaultJsonProtocol.RootJsObjectFormat
 import spray.json.DeserializationException
 import whisk.common.TransactionId
+import whisk.core.containerpool.logging.LogStore
 import whisk.core.database.StaleParameter
 import whisk.core.entitlement.{Collection, Privilege, Resource}
 import whisk.core.entitlement.Privilege.READ
@@ -54,6 +56,9 @@ trait WhiskActivationsApi extends Directives with 
AuthenticatedRouteProvider wit
   /** Database service to GET activations. */
   protected val activationStore: ActivationStore
 
+  /** LogStore for retrieving activation logs */
+  protected val logStore: LogStore
+
   /** Path to Actions REST API. */
   protected val activationsPath = "activations"
 
@@ -192,7 +197,7 @@ trait WhiskActivationsApi extends Directives with 
AuthenticatedRouteProvider wit
       WhiskActivation,
       activationStore,
       docid,
-      (activation: WhiskActivation) => activation.response.toExtendedJson)
+      (activation: WhiskActivation) => 
Future.successful(activation.response.toExtendedJson))
   }
 
   /**
@@ -208,7 +213,7 @@ trait WhiskActivationsApi extends Directives with 
AuthenticatedRouteProvider wit
       WhiskActivation,
       activationStore,
       docid,
-      (activation: WhiskActivation) => activation.logs.toJsonObject)
+      (activation: WhiskActivation) => 
logStore.fetchLogs(activation).map(_.toJsonObject))
   }
 
   /** Custom unmarshaller for query parameters "name" into valid entity name. 
*/
diff --git 
a/core/controller/src/main/scala/whisk/core/controller/ApiUtils.scala 
b/core/controller/src/main/scala/whisk/core/controller/ApiUtils.scala
index b6aa0aa..c97ae76 100644
--- a/core/controller/src/main/scala/whisk/core/controller/ApiUtils.scala
+++ b/core/controller/src/main/scala/whisk/core/controller/ApiUtils.scala
@@ -200,7 +200,7 @@ trait ReadOps extends Directives {
     factory: DocumentFactory[A],
     datastore: ArtifactStore[Au],
     docid: DocId,
-    project: A => JsObject)(implicit transid: TransactionId, format: 
RootJsonFormat[A], ma: Manifest[A]) = {
+    project: A => Future[JsObject])(implicit transid: TransactionId, format: 
RootJsonFormat[A], ma: Manifest[A]) = {
     onComplete(factory.get(datastore, docid)) {
       case Success(entity) =>
         logging.info(this, s"[PROJECT] entity success")
diff --git 
a/core/controller/src/main/scala/whisk/core/controller/Controller.scala 
b/core/controller/src/main/scala/whisk/core/controller/Controller.scala
index 6c8bef5..bb723bc 100644
--- a/core/controller/src/main/scala/whisk/core/controller/Controller.scala
+++ b/core/controller/src/main/scala/whisk/core/controller/Controller.scala
@@ -48,6 +48,8 @@ import whisk.core.entity.ExecManifest.Runtimes
 import whisk.core.loadBalancer.{LoadBalancerService}
 import whisk.http.BasicHttpService
 import whisk.http.BasicRasService
+import whisk.spi.SpiLoader
+import whisk.core.containerpool.logging.LogStoreProvider
 
 /**
  * The Controller is the service that provides the REST API for OpenWhisk.
@@ -115,6 +117,7 @@ class Controller(val instance: InstanceId,
   private implicit val loadBalancer = new LoadBalancerService(whiskConfig, 
instance, entityStore)
   private implicit val entitlementProvider = new 
LocalEntitlementProvider(whiskConfig, loadBalancer)
   private implicit val activationIdFactory = new ActivationIdGenerator {}
+  private implicit val logStore = 
SpiLoader.get[LogStoreProvider].logStore(actorSystem)
 
   // register collections
   Collection.initialize(entityStore)
diff --git 
a/core/controller/src/main/scala/whisk/core/controller/RestAPIs.scala 
b/core/controller/src/main/scala/whisk/core/controller/RestAPIs.scala
index acda09d..8015f60 100644
--- a/core/controller/src/main/scala/whisk/core/controller/RestAPIs.scala
+++ b/core/controller/src/main/scala/whisk/core/controller/RestAPIs.scala
@@ -22,7 +22,6 @@ import scala.concurrent.ExecutionContext
 import akka.actor.ActorSystem
 import akka.http.scaladsl.model.StatusCodes._
 import akka.http.scaladsl.model.Uri
-import akka.http.scaladsl.model.headers._
 import akka.http.scaladsl.server.Directives
 import akka.http.scaladsl.server.Route
 import akka.http.scaladsl.model.headers._
@@ -31,12 +30,13 @@ import akka.stream.ActorMaterializer
 
 import spray.json._
 import spray.json.DefaultJsonProtocol._
-import whisk.common.Logging
-import whisk.common.TransactionId
 import whisk.core.database.CacheChangeNotification
 import whisk.core.WhiskConfig
 import whisk.core.WhiskConfig.whiskVersionBuildno
 import whisk.core.WhiskConfig.whiskVersionDate
+import whisk.common.Logging
+import whisk.common.TransactionId
+import whisk.core.containerpool.logging.LogStore
 import whisk.core.entitlement._
 import whisk.core.entity._
 import whisk.core.entity.ActivationId.ActivationIdGenerator
@@ -143,6 +143,7 @@ class RestAPIVersion(config: WhiskConfig, apiPath: String, 
apiVersion: String)(
   implicit val loadBalancer: LoadBalancerService,
   implicit val cacheChangeNotification: Some[CacheChangeNotification],
   implicit val activationStore: ActivationStore,
+  implicit val logStore: LogStore,
   implicit val whiskConfig: WhiskConfig)
     extends SwaggerDocs(Uri.Path(apiPath) / apiVersion, "apiv1swagger.json")
     with Authenticate
@@ -234,6 +235,7 @@ class RestAPIVersion(config: WhiskConfig, apiPath: String, 
apiVersion: String)(
 
   class ActivationsApi(val apiPath: String, val apiVersion: String)(
     implicit override val activationStore: ActivationStore,
+    override val logStore: LogStore,
     override val entitlementProvider: EntitlementProvider,
     override val executionContext: ExecutionContext,
     override val logging: Logging)
diff --git 
a/core/invoker/src/main/scala/whisk/core/containerpool/ContainerProxy.scala 
b/core/invoker/src/main/scala/whisk/core/containerpool/ContainerProxy.scala
index 8df292d..a1b9658 100644
--- a/core/invoker/src/main/scala/whisk/core/containerpool/ContainerProxy.scala
+++ b/core/invoker/src/main/scala/whisk/core/containerpool/ContainerProxy.scala
@@ -94,6 +94,7 @@ case object ContainerRemoved
 class ContainerProxy(factory: (TransactionId, String, ImageName, Boolean, 
ByteSize) => Future[Container],
                      sendActiveAck: (TransactionId, WhiskActivation, Boolean, 
InstanceId) => Future[Any],
                      storeActivation: (TransactionId, WhiskActivation) => 
Future[Any],
+                     collectLogs: (TransactionId, Container, 
ExecutableWhiskAction) => Future[ActivationLogs],
                      instance: InstanceId,
                      unusedTimeout: FiniteDuration,
                      pauseGrace: FiniteDuration)
@@ -370,8 +371,8 @@ class ContainerProxy(factory: (TransactionId, String, 
ImageName, Boolean, ByteSi
         case Success(ack) => sendActiveAck(tid, ack, job.msg.blocking, 
job.msg.rootControllerIndex)
       }
       .flatMap { activation =>
-        container.logs(job.action.limits.logs.asMegaBytes, 
job.action.exec.sentinelledLogs).map { logs =>
-          activation.withLogs(ActivationLogs(logs.toVector))
+        collectLogs(tid, container, job.action).map { logs =>
+          activation.withLogs(logs)
         }
       }
       .andThen {
@@ -390,10 +391,11 @@ object ContainerProxy {
   def props(factory: (TransactionId, String, ImageName, Boolean, ByteSize) => 
Future[Container],
             ack: (TransactionId, WhiskActivation, Boolean, InstanceId) => 
Future[Any],
             store: (TransactionId, WhiskActivation) => Future[Any],
+            collectLogs: (TransactionId, Container, ExecutableWhiskAction) => 
Future[ActivationLogs],
             instance: InstanceId,
             unusedTimeout: FiniteDuration = 10.minutes,
             pauseGrace: FiniteDuration = 50.milliseconds) =
-    Props(new ContainerProxy(factory, ack, store, instance, unusedTimeout, 
pauseGrace))
+    Props(new ContainerProxy(factory, ack, store, collectLogs, instance, 
unusedTimeout, pauseGrace))
 
   // Needs to be thread-safe as it's used by multiple proxies concurrently.
   private val containerCount = new Counter
diff --git 
a/core/invoker/src/main/scala/whisk/core/invoker/InvokerReactive.scala 
b/core/invoker/src/main/scala/whisk/core/invoker/InvokerReactive.scala
index 76b9c3e..51246c8 100644
--- a/core/invoker/src/main/scala/whisk/core/invoker/InvokerReactive.scala
+++ b/core/invoker/src/main/scala/whisk/core/invoker/InvokerReactive.scala
@@ -47,6 +47,7 @@ import whisk.core.containerpool.ContainerPool
 import whisk.core.containerpool.ContainerProxy
 import whisk.core.containerpool.PrewarmingConfig
 import whisk.core.containerpool.Run
+import whisk.core.containerpool.logging.LogStoreProvider
 import whisk.core.database.NoDocumentException
 import whisk.core.entity._
 import whisk.core.entity.size._
@@ -61,6 +62,8 @@ class InvokerReactive(config: WhiskConfig, instance: 
InstanceId, producer: Messa
   implicit val ec = actorSystem.dispatcher
   implicit val cfg = config
 
+  private val logsProvider = 
SpiLoader.get[LogStoreProvider].logStore(actorSystem)
+
   /**
    * Factory used by the ContainerProxy to physically create a new container.
    *
@@ -80,7 +83,7 @@ class InvokerReactive(config: WhiskConfig, instance: 
InstanceId, producer: Messa
           "--cap-drop" -> Set("NET_RAW", "NET_ADMIN"),
           "--ulimit" -> Set("nofile=1024:1024"),
           "--pids-limit" -> Set("1024"),
-          "--dns" -> config.invokerContainerDns.toSet))
+          "--dns" -> config.invokerContainerDns.toSet) ++ 
logsProvider.containerParameters)
   containerFactory.init()
   sys.addShutdownHook(containerFactory.cleanup())
 
@@ -139,7 +142,7 @@ class InvokerReactive(config: WhiskConfig, instance: 
InstanceId, producer: Messa
 
   /** Creates a ContainerProxy Actor when being called. */
   val childFactory = (f: ActorRefFactory) =>
-    f.actorOf(ContainerProxy.props(containerFactory.createContainer _, ack, 
store, instance))
+    f.actorOf(ContainerProxy.props(containerFactory.createContainer, ack, 
store, logsProvider.collectLogs, instance))
 
   val prewarmKind = "nodejs:6"
   val prewarmExec = ExecManifest.runtimesManifest
diff --git 
a/tests/src/test/scala/whisk/core/containerpool/test/ContainerProxyTests.scala 
b/tests/src/test/scala/whisk/core/containerpool/test/ContainerProxyTests.scala
index ca9d6f4..d3484e9 100644
--- 
a/tests/src/test/scala/whisk/core/containerpool/test/ContainerProxyTests.scala
+++ 
b/tests/src/test/scala/whisk/core/containerpool/test/ContainerProxyTests.scala
@@ -44,6 +44,7 @@ import whisk.common.Logging
 import whisk.common.TransactionId
 import whisk.core.connector.ActivationMessage
 import whisk.core.containerpool._
+import whisk.core.containerpool.logging.DockerLogStore
 import whisk.core.entity._
 import whisk.core.entity.ExecManifest.RuntimeManifest
 import whisk.core.entity.ExecManifest.ImageName
@@ -148,6 +149,8 @@ class ContainerProxyTests
 
   val store = stubFunction[TransactionId, WhiskActivation, Future[Any]]
 
+  val collectLogs = new DockerLogStore(system).collectLogs _
+
   behavior of "ContainerProxy"
 
   /*
@@ -157,7 +160,8 @@ class ContainerProxyTests
     val container = new TestContainer
     val factory = createFactory(Future.successful(container))
 
-    val machine = childActorOf(ContainerProxy.props(factory, createAcker, 
store, InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, createAcker, store, 
collectLogs, InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     preWarm(machine)
 
@@ -174,7 +178,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
 
     preWarm(machine)
@@ -206,7 +211,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     preWarm(machine)
 
@@ -230,7 +236,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     preWarm(machine)
 
@@ -256,7 +263,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     run(machine, Uninitialized)
 
@@ -278,7 +286,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.failed(new Exception()))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     machine ! Run(action, message)
     expectMsg(Transition(machine, Uninitialized, Running))
@@ -307,7 +316,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     machine ! Run(action, message)
     expectMsg(Transition(machine, Uninitialized, Running))
@@ -336,7 +346,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     machine ! Run(action, message)
     expectMsg(Transition(machine, Uninitialized, Running))
@@ -364,7 +375,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     run(machine, Uninitialized) // first run an activation
     timeout(machine) // times out Ready state so container suspends
@@ -396,7 +408,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     run(machine, Uninitialized)
     timeout(machine) // times out Ready state so container suspends
@@ -428,7 +441,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
 
     // Start running the action
@@ -475,7 +489,8 @@ class ContainerProxyTests
     val factory = createFactory(Future.successful(container))
     val acker = createAcker
 
-    val machine = childActorOf(ContainerProxy.props(factory, acker, store, 
InstanceId(0), pauseGrace = timeout))
+    val machine =
+      childActorOf(ContainerProxy.props(factory, acker, store, collectLogs, 
InstanceId(0), pauseGrace = timeout))
     registerCallback(machine)
     run(machine, Uninitialized)
     timeout(machine)
diff --git 
a/tests/src/test/scala/whisk/core/controller/test/ControllerTestCommon.scala 
b/tests/src/test/scala/whisk/core/controller/test/ControllerTestCommon.scala
index cb79804..1daa2a4 100644
--- a/tests/src/test/scala/whisk/core/controller/test/ControllerTestCommon.scala
+++ b/tests/src/test/scala/whisk/core/controller/test/ControllerTestCommon.scala
@@ -34,6 +34,7 @@ import whisk.common.TransactionCounter
 import whisk.common.TransactionId
 import whisk.core.WhiskConfig
 import whisk.core.connector.ActivationMessage
+import whisk.core.containerpool.logging.LogStoreProvider
 import whisk.core.controller.RestApiCommons
 import whisk.core.controller.WhiskServices
 import whisk.core.database.DocumentFactory
@@ -43,6 +44,7 @@ import whisk.core.entitlement._
 import whisk.core.entity._
 import whisk.core.entity.test.ExecHelpers
 import whisk.core.loadBalancer.LoadBalancer
+import whisk.spi.SpiLoader
 
 protected trait ControllerTestCommon
     extends FlatSpec
@@ -91,6 +93,7 @@ protected trait ControllerTestCommon
   val entityStore = WhiskEntityStore.datastore(whiskConfig)
   val activationStore = WhiskActivationStore.datastore(whiskConfig)
   val authStore = WhiskAuthStore.datastore(whiskConfig)
+  val logStore = SpiLoader.get[LogStoreProvider].logStore(actorSystem)
 
   def deleteAction(doc: DocId)(implicit transid: TransactionId) = {
     Await.result(WhiskAction.get(entityStore, doc) flatMap { doc =>

-- 
To stop receiving notification emails like this one, please contact
['"commits@openwhisk.apache.org" <commits@openwhisk.apache.org>'].

Reply via email to