You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gearpump.apache.org by ma...@apache.org on 2016/04/28 04:37:21 UTC

incubator-gearpump git commit: fix GEARPUMP-41 check akka.scheduler.tick-duration on Windows

Repository: incubator-gearpump
Updated Branches:
  refs/heads/master abe6f1047 -> ccf318f3a


fix GEARPUMP-41 check akka.scheduler.tick-duration on Windows


Project: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/commit/ccf318f3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/tree/ccf318f3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/diff/ccf318f3

Branch: refs/heads/master
Commit: ccf318f3ab812ba3a363d52d1f67e1705b08014c
Parents: abe6f10
Author: huafengw <fv...@gmail.com>
Authored: Wed Apr 27 09:26:19 2016 +0800
Committer: huafengw <fv...@gmail.com>
Committed: Thu Apr 28 09:42:41 2016 +0800

----------------------------------------------------------------------
 .../scala/io/gearpump/cluster/ClusterConfig.scala  |  2 +-
 .../main/scala/io/gearpump/util/Constants.scala    |  2 ++
 .../scala/io/gearpump/cluster/worker/Worker.scala  | 17 +++++++++++++----
 3 files changed, 16 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/ccf318f3/core/src/main/scala/io/gearpump/cluster/ClusterConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/io/gearpump/cluster/ClusterConfig.scala b/core/src/main/scala/io/gearpump/cluster/ClusterConfig.scala
index 7bae6d6..5cc49e7 100644
--- a/core/src/main/scala/io/gearpump/cluster/ClusterConfig.scala
+++ b/core/src/main/scala/io/gearpump/cluster/ClusterConfig.scala
@@ -142,7 +142,7 @@ object ClusterConfig {
 
     if (!akka.util.Helpers.isWindows) {
 
-      // Change the akka.scheduler.tick-duration to 10ms for Linux or Mac
+      // Change the akka.scheduler.tick-duration to 1 ms for Linux or Mac
       basic = linux.withFallback(basic)
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/ccf318f3/core/src/main/scala/io/gearpump/util/Constants.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/io/gearpump/util/Constants.scala b/core/src/main/scala/io/gearpump/util/Constants.scala
index 65b7023..342cd87 100644
--- a/core/src/main/scala/io/gearpump/util/Constants.scala
+++ b/core/src/main/scala/io/gearpump/util/Constants.scala
@@ -172,4 +172,6 @@ object Constants {
   val PREFER_IPV4 = "java.net.preferIPv4Stack"
 
   val APPLICATION_EXECUTOR_NUMBER = "gearpump.application.executor-num"
+
+  val AKKA_SCHEDULER_TICK_DURATION = "akka.scheduler.tick-duration"
 }

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/ccf318f3/daemon/src/main/scala/io/gearpump/cluster/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/daemon/src/main/scala/io/gearpump/cluster/worker/Worker.scala b/daemon/src/main/scala/io/gearpump/cluster/worker/Worker.scala
index f5e6919..0568641 100644
--- a/daemon/src/main/scala/io/gearpump/cluster/worker/Worker.scala
+++ b/daemon/src/main/scala/io/gearpump/cluster/worker/Worker.scala
@@ -28,7 +28,7 @@ import scala.util.{Failure, Success, Try}
 
 import akka.actor.SupervisorStrategy.Stop
 import akka.actor._
-import com.typesafe.config.{Config, ConfigFactory}
+import com.typesafe.config.{ConfigValueFactory, Config, ConfigFactory}
 import org.slf4j.Logger
 
 import io.gearpump.cluster.AppMasterToMaster.{GetWorkerData, WorkerData}
@@ -341,6 +341,8 @@ private[cluster] object Worker {
       procLauncher: ExecutorProcessLauncher) extends Actor {
     import launch.{appId, executorId, resource}
 
+    private val LOG: Logger = LogUtil.getLogger(getClass, app = appId, executor = executorId)
+
     val executorConfig: Config = {
       val workerConfig = context.system.settings.config
 
@@ -362,12 +364,19 @@ private[cluster] object Worker {
         // Falls back to workerConfig
         .withFallback(workerConfig)
 
+      // Minimum supported akka.scheduler.tick-duration on Windows is 10ms
+      val duration = config.getInt(AKKA_SCHEDULER_TICK_DURATION)
+      val updatedConf = if (akka.util.Helpers.isWindows && duration < 10) {
+        LOG.warn(s"$AKKA_SCHEDULER_TICK_DURATION on Windows must be larger than 10ms, set to 10ms")
+        config.withValue(AKKA_SCHEDULER_TICK_DURATION, ConfigValueFactory.fromAnyRef(10))
+      } else {
+        config
+      }
+
       // Excludes reference.conf, and JVM properties..
-      ClusterConfig.filterOutDefaultConfig(config)
+      ClusterConfig.filterOutDefaultConfig(updatedConf)
     }
 
-    private val LOG: Logger = LogUtil.getLogger(getClass, app = appId, executor = executorId)
-
     implicit val executorService = ioPool
 
     private val executorHandler = {