You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@apex.apache.org by vr...@apache.org on 2017/01/28 05:23:21 UTC

apex-core git commit: APEXCORE-611 Added log levels for Stram Events

Repository: apex-core
Updated Branches:
  refs/heads/master 42c695c6f -> a8b60f68e


APEXCORE-611 Added log levels for Stram Events


Project: http://git-wip-us.apache.org/repos/asf/apex-core/repo
Commit: http://git-wip-us.apache.org/repos/asf/apex-core/commit/a8b60f68
Tree: http://git-wip-us.apache.org/repos/asf/apex-core/tree/a8b60f68
Diff: http://git-wip-us.apache.org/repos/asf/apex-core/diff/a8b60f68

Branch: refs/heads/master
Commit: a8b60f68e5711a9d365d2cf749a13f84aef18b0a
Parents: 42c695c
Author: ajaygit158 <aj...@gmail.com>
Authored: Wed Jan 25 15:11:23 2017 +0530
Committer: ajaygit158 <aj...@gmail.com>
Committed: Sat Jan 28 08:30:26 2017 +0530

----------------------------------------------------------------------
 .../com/datatorrent/stram/api/StramEvent.java   | 106 ++++++++++++++++---
 1 file changed, 94 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/apex-core/blob/a8b60f68/engine/src/main/java/com/datatorrent/stram/api/StramEvent.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/api/StramEvent.java b/engine/src/main/java/com/datatorrent/stram/api/StramEvent.java
index ba9b55d..e9c5f13 100644
--- a/engine/src/main/java/com/datatorrent/stram/api/StramEvent.java
+++ b/engine/src/main/java/com/datatorrent/stram/api/StramEvent.java
@@ -34,12 +34,14 @@ public abstract class StramEvent
   private final long id;
   private long timestamp = System.currentTimeMillis();
   private String reason;
+  private LogLevel logLevel;
 
   public abstract String getType();
 
-  protected StramEvent()
+  protected StramEvent(LogLevel logLevel)
   {
     id = nextId.getAndIncrement();
+    this.logLevel = logLevel;
   }
 
   public long getId()
@@ -67,12 +69,28 @@ public abstract class StramEvent
     this.reason = reason;
   }
 
+  public LogLevel getLogLevel()
+  {
+    return logLevel;
+  }
+
+  public static enum LogLevel
+  {
+    TRACE,
+    DEBUG,
+    INFO,
+    WARN,
+    ERROR,
+    FATAL;
+  }
+
   public abstract static class OperatorEvent extends StramEvent
   {
     private String operatorName;
 
-    public OperatorEvent(String operatorName)
+    public OperatorEvent(String operatorName, LogLevel logLevel)
     {
+      super(logLevel);
       this.operatorName = operatorName;
     }
 
@@ -95,7 +113,12 @@ public abstract class StramEvent
 
     public SetOperatorPropertyEvent(String operatorName, String propertyName, String propertyValue)
     {
-      super(operatorName);
+      this(operatorName, propertyName, propertyValue, LogLevel.INFO);
+    }
+
+    public SetOperatorPropertyEvent(String operatorName, String propertyName, String propertyValue, LogLevel logLevel)
+    {
+      super(operatorName, logLevel);
       this.propertyName = propertyName;
       this.propertyValue = propertyValue;
     }
@@ -135,7 +158,12 @@ public abstract class StramEvent
 
     public PartitionEvent(String operatorName, int oldNumPartitions, int newNumPartitions)
     {
-      super(operatorName);
+      this(operatorName, oldNumPartitions, newNumPartitions, LogLevel.INFO);
+    }
+
+    public PartitionEvent(String operatorName, int oldNumPartitions, int newNumPartitions, LogLevel logLevel)
+    {
+      super(operatorName, logLevel);
       this.oldNumPartitions = oldNumPartitions;
       this.newNumPartitions = newNumPartitions;
     }
@@ -172,9 +200,9 @@ public abstract class StramEvent
   {
     private final int operatorId;
 
-    public PhysicalOperatorEvent(String operatorName, int operatorId)
+    public PhysicalOperatorEvent(String operatorName, int operatorId, LogLevel logLevel)
     {
-      super(operatorName);
+      super(operatorName, logLevel);
       this.operatorId = operatorId;
     }
 
@@ -189,7 +217,12 @@ public abstract class StramEvent
   {
     public CreateOperatorEvent(String operatorName, int operatorId)
     {
-      super(operatorName, operatorId);
+      this(operatorName, operatorId, LogLevel.INFO);
+    }
+
+    public CreateOperatorEvent(String operatorName, int operatorId, LogLevel logLevel)
+    {
+      super(operatorName, operatorId, logLevel);
     }
 
     @Override
@@ -204,7 +237,12 @@ public abstract class StramEvent
   {
     public RemoveOperatorEvent(String operatorName, int operatorId)
     {
-      super(operatorName, operatorId);
+      this(operatorName, operatorId, LogLevel.INFO);
+    }
+
+    public RemoveOperatorEvent(String operatorName, int operatorId, LogLevel logLevel)
+    {
+      super(operatorName, operatorId, logLevel);
     }
 
     @Override
@@ -221,7 +259,12 @@ public abstract class StramEvent
 
     public StartOperatorEvent(String operatorName, int operatorId, String containerId)
     {
-      super(operatorName, operatorId);
+      this(operatorName, operatorId, containerId, LogLevel.INFO);
+    }
+
+    public StartOperatorEvent(String operatorName, int operatorId, String containerId, LogLevel logLevel)
+    {
+      super(operatorName, operatorId, logLevel);
       this.containerId = containerId;
     }
 
@@ -249,7 +292,12 @@ public abstract class StramEvent
 
     public StopOperatorEvent(String operatorName, int operatorId, String containerId)
     {
-      super(operatorName, operatorId);
+      this(operatorName, operatorId, containerId, LogLevel.WARN);
+    }
+
+    public StopOperatorEvent(String operatorName, int operatorId, String containerId, LogLevel logLevel)
+    {
+      super(operatorName, operatorId, logLevel);
       this.containerId = containerId;
     }
 
@@ -278,7 +326,12 @@ public abstract class StramEvent
 
     public SetPhysicalOperatorPropertyEvent(String operatorName, int operatorId, String propertyName, String propertyValue)
     {
-      super(operatorName, operatorId);
+      this(operatorName, operatorId, propertyName, propertyValue, LogLevel.INFO);
+    }
+
+    public SetPhysicalOperatorPropertyEvent(String operatorName, int operatorId, String propertyName, String propertyValue, LogLevel logLevel)
+    {
+      super(operatorName, operatorId, logLevel);
       this.propertyName = propertyName;
       this.propertyValue = propertyValue;
     }
@@ -318,6 +371,12 @@ public abstract class StramEvent
 
     public StartContainerEvent(String containerId, String containerNodeId)
     {
+      this(containerId, containerNodeId, LogLevel.INFO);
+    }
+
+    public StartContainerEvent(String containerId, String containerNodeId, LogLevel logLevel)
+    {
+      super(logLevel);
       this.containerId = containerId;
       this.containerNodeId = containerNodeId;
     }
@@ -357,6 +416,12 @@ public abstract class StramEvent
 
     public StopContainerEvent(String containerId, int exitStatus)
     {
+      this(containerId, exitStatus, LogLevel.INFO);
+    }
+
+    public StopContainerEvent(String containerId, int exitStatus, LogLevel logLevel)
+    {
+      super(logLevel);
       this.containerId = containerId;
       this.exitStatus = exitStatus;
     }
@@ -395,6 +460,12 @@ public abstract class StramEvent
 
     public ChangeLogicalPlanEvent(LogicalPlanRequest request)
     {
+      this(request, LogLevel.INFO);
+    }
+
+    public ChangeLogicalPlanEvent(LogicalPlanRequest request, LogLevel logLevel)
+    {
+      super(logLevel);
       this.request = request;
     }
 
@@ -423,7 +494,12 @@ public abstract class StramEvent
 
     public OperatorErrorEvent(String operatorName, int operatorId, String containerId, String errorMessage)
     {
-      super(operatorName, operatorId);
+      this(operatorName, operatorId, containerId, errorMessage, LogLevel.ERROR);
+    }
+
+    public OperatorErrorEvent(String operatorName, int operatorId, String containerId, String errorMessage, LogLevel logLevel)
+    {
+      super(operatorName, operatorId, logLevel);
       this.containerId = containerId;
       this.errorMessage = errorMessage;
     }
@@ -463,6 +539,12 @@ public abstract class StramEvent
 
     public ContainerErrorEvent(String containerId, String errorMessage)
     {
+      this(containerId, errorMessage, LogLevel.ERROR);
+    }
+
+    public ContainerErrorEvent(String containerId, String errorMessage, LogLevel logLevel)
+    {
+      super(logLevel);
       this.containerId = containerId;
       this.errorMessage = errorMessage;
     }