[FLINK-1372] Log memory usage at start up and fix log level mismatch for scheduled memory log message
This closes #321. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ce8acc49 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ce8acc49 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ce8acc49 Branch: refs/heads/master Commit: ce8acc4920bfa1980d5b046cb858a28a6fe055fb Parents: 8d73d45 Author: Ufuk Celebi <[email protected]> Authored: Tue Jan 20 20:09:30 2015 +0100 Committer: Ufuk Celebi <[email protected]> Committed: Tue Jan 20 20:30:13 2015 +0100 ---------------------------------------------------------------------- .../flink/runtime/taskmanager/TaskManager.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/ce8acc49/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala ---------------------------------------------------------------------- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index d808e8e..6dca14a 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -115,6 +115,10 @@ import scala.collection.JavaConverters._ None } + if (log.isInfoEnabled) { + log.info(TaskManager.getMemoryUsageStatsAsString(ManagementFactory.getMemoryMXBean())); + } + var libraryCacheManager: LibraryCacheManager = null var networkEnvironment: Option[NetworkEnvironment] = None var registrationScheduler: Option[Cancellable] = None @@ -124,12 +128,10 @@ import scala.collection.JavaConverters._ var instanceID: InstanceID = null var heartbeatScheduler: Option[Cancellable] = None - if (log.isDebugEnabled) { - memoryLogggingIntervalMs.foreach { - interval => - val d = FiniteDuration(interval, TimeUnit.MILLISECONDS) - context.system.scheduler.schedule(d, d, self, LogMemoryUsage) - } + memoryLogggingIntervalMs.foreach { + interval => + val d = FiniteDuration(interval, TimeUnit.MILLISECONDS) + context.system.scheduler.schedule(d, d, self, LogMemoryUsage) } override def preStart(): Unit = {
