You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@apex.apache.org by th...@apache.org on 2016/01/29 08:21:20 UTC

[01/50] incubator-apex-core git commit: Merge branch 'ilooner-APEX-83_pull' into devel-3 [Forced Update!]

Repository: incubator-apex-core
Updated Branches:
  refs/heads/master 712138ac4 -> 9ffbc73e7 (forced update)


Merge branch 'ilooner-APEX-83_pull' into devel-3


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

Branch: refs/heads/master
Commit: 38a6f4e3bec2412f0b98ce18dd727c2d335c4d1f
Parents: c30dd95 2c4254f
Author: Gaurav Gupta <ga...@apache.org>
Authored: Tue Dec 1 00:57:53 2015 -0800
Committer: Gaurav Gupta <ga...@apache.org>
Committed: Tue Dec 1 00:57:53 2015 -0800

----------------------------------------------------------------------
 .../java/com/datatorrent/stram/engine/Node.java |  27 ++-
 .../stram/engine/GenericNodeTest.java           | 211 ++++++++++++++++++-
 2 files changed, 229 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[45/50] incubator-apex-core git commit: APEXCORE-306 Update checkpoints for strongly connected operators as group.

Posted by th...@apache.org.
APEXCORE-306 Update checkpoints for strongly connected operators as group.


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

Branch: refs/heads/master
Commit: b3402be5a45728515f4a8328fec5a76ddede0350
Parents: 4d5828c
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Jan 21 16:39:55 2016 -0800
Committer: David Yan <da...@datatorrent.com>
Committed: Fri Jan 22 19:24:31 2016 -0800

----------------------------------------------------------------------
 .../stram/StreamingContainerManager.java        | 161 +++++++++++++------
 .../com/datatorrent/stram/api/Checkpoint.java   |  11 ++
 .../stram/plan/logical/LogicalPlan.java         |  91 +++++++----
 .../com/datatorrent/stram/CheckpointTest.java   |   3 +-
 .../stram/plan/logical/DelayOperatorTest.java   |  88 +++++++++-
 .../stram/plan/logical/LogicalPlanTest.java     | 131 ++++++++++-----
 6 files changed, 358 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b3402be5/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
index 6233697..a687a37 100644
--- a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
+++ b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
@@ -35,6 +35,7 @@ import com.esotericsoftware.kryo.KryoException;
 import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
@@ -159,6 +160,7 @@ public class StreamingContainerManager implements PlanContext
   private long lastResourceRequest = 0;
   private final Map<String, StreamingContainerAgent> containers = new ConcurrentHashMap<String, StreamingContainerAgent>();
   private final List<Pair<PTOperator, Long>> purgeCheckpoints = new ArrayList<Pair<PTOperator, Long>>();
+  private Map<OperatorMeta, Set<OperatorMeta>> checkpointGroups;
   private final Map<Long, Set<PTOperator>> shutdownOperators = new HashMap<>();
   private CriticalPathInfo criticalPathInfo;
   private final ConcurrentMap<PTOperator, PTOperator> reportStats = Maps.newConcurrentMap();
@@ -812,6 +814,7 @@ public class StreamingContainerManager implements PlanContext
     Collection<OperatorMeta> logicalOperators = getLogicalPlan().getAllOperators();
     //for backward compatibility
     for (OperatorMeta operatorMeta : logicalOperators) {
+      @SuppressWarnings("deprecation")
       Context.CountersAggregator aggregator = operatorMeta.getValue(OperatorContext.COUNTERS_AGGREGATOR);
       if (aggregator == null) {
         continue;
@@ -825,6 +828,7 @@ public class StreamingContainerManager implements PlanContext
         }
       }
       if (counters.size() > 0) {
+        @SuppressWarnings("deprecation")
         Object aggregate = aggregator.aggregate(counters);
         latestLogicalCounters.put(operatorMeta.getName(), aggregate);
       }
@@ -857,6 +861,8 @@ public class StreamingContainerManager implements PlanContext
         if (windowMetrics == null) {
           windowMetrics = new LinkedBlockingQueue<Pair<Long, Map<String, Object>>>(METRIC_QUEUE_SIZE)
           {
+            private static final long serialVersionUID = 1L;
+
             @Override
             public boolean add(Pair<Long, Map<String, Object>> longMapPair)
             {
@@ -1134,7 +1140,7 @@ public class StreamingContainerManager implements PlanContext
     cs.container.setAllocatedVCores(0);
 
     // resolve dependencies
-    UpdateCheckpointsContext ctx = new UpdateCheckpointsContext(clock);
+    UpdateCheckpointsContext ctx = new UpdateCheckpointsContext(clock, false, getCheckpointGroups());
     for (PTOperator oper : cs.container.getOperators()) {
       updateRecoveryCheckpoints(oper, ctx);
     }
@@ -1881,31 +1887,18 @@ public class StreamingContainerManager implements PlanContext
     public final Set<PTOperator> blocked = new LinkedHashSet<PTOperator>();
     public final long currentTms;
     public final boolean recovery;
+    public final Map<OperatorMeta, Set<OperatorMeta>> checkpointGroups;
 
     public UpdateCheckpointsContext(Clock clock)
     {
-      this.currentTms = clock.getTime();
-      this.recovery = false;
+      this(clock, false, Collections.<OperatorMeta, Set<OperatorMeta>>emptyMap());
     }
 
-    public UpdateCheckpointsContext(Clock clock, boolean recovery)
+    public UpdateCheckpointsContext(Clock clock, boolean recovery, Map<OperatorMeta, Set<OperatorMeta>> checkpointGroups)
     {
       this.currentTms = clock.getTime();
       this.recovery = recovery;
-    }
-
-  }
-
-  private void addVisited(PTOperator operator, UpdateCheckpointsContext ctx)
-  {
-    ctx.visited.add(operator);
-    for (PTOperator.PTOutput out : operator.getOutputs()) {
-      for (PTOperator.PTInput sink : out.sinks) {
-        PTOperator sinkOperator = sink.target;
-        if (!ctx.visited.contains(sinkOperator)) {
-          addVisited(sinkOperator, ctx);
-        }
-      }
+      this.checkpointGroups = checkpointGroups;
     }
   }
 
@@ -1933,20 +1926,55 @@ public class StreamingContainerManager implements PlanContext
       }
     }
 
-    long maxCheckpoint = operator.getRecentCheckpoint().windowId;
-    if (ctx.recovery && maxCheckpoint == Stateless.WINDOW_ID && operator.isOperatorStateLess()) {
-      long currentWindowId = WindowGenerator.getWindowId(ctx.currentTms, this.vars.windowStartMillis, this.getLogicalPlan().getValue(LogicalPlan.STREAMING_WINDOW_SIZE_MILLIS));
-      maxCheckpoint = currentWindowId;
+    // the most recent checkpoint eligible for recovery based on downstream state
+    Checkpoint maxCheckpoint = Checkpoint.INITIAL_CHECKPOINT;
+
+    Set<OperatorMeta> checkpointGroup = ctx.checkpointGroups.get(operator.getOperatorMeta());
+    if (checkpointGroup == null) {
+      checkpointGroup = Collections.singleton(operator.getOperatorMeta());
+    }
+    // find intersection of checkpoints that group can collectively move to
+    TreeSet<Checkpoint> commonCheckpoints = new TreeSet<>(new Checkpoint.CheckpointComparator());
+    synchronized (operator.checkpoints) {
+      commonCheckpoints.addAll(operator.checkpoints);
+    }
+    Set<PTOperator> groupOpers = new HashSet<>(checkpointGroup.size());
+    if (checkpointGroup.size() > 1) {
+      for (OperatorMeta om : checkpointGroup) {
+        Collection<PTOperator> operators = plan.getAllOperators(om);
+        for (PTOperator groupOper : operators) {
+          synchronized (groupOper.checkpoints) {
+            commonCheckpoints.retainAll(groupOper.checkpoints);
+          }
+          // visit all downstream operators of the group
+          ctx.visited.add(groupOper);
+          groupOpers.add(groupOper);
+        }
+      }
+      // highest common checkpoint
+      if (!commonCheckpoints.isEmpty()) {
+        maxCheckpoint = commonCheckpoints.last();
+      }
+    } else {
+      // without logical grouping, treat partitions as independent
+      // this is especially important for parallel partitioning
+      ctx.visited.add(operator);
+      groupOpers.add(operator);
+      maxCheckpoint = operator.getRecentCheckpoint();
+      if (ctx.recovery && maxCheckpoint.windowId == Stateless.WINDOW_ID && operator.isOperatorStateLess()) {
+        long currentWindowId = WindowGenerator.getWindowId(ctx.currentTms, this.vars.windowStartMillis, this.getLogicalPlan().getValue(LogicalPlan.STREAMING_WINDOW_SIZE_MILLIS));
+        maxCheckpoint = new Checkpoint(currentWindowId, 0, 0);
+      }
     }
-    ctx.visited.add(operator);
 
     // DFS downstream operators
-    if (operator.getOperatorMeta().getOperator() instanceof Operator.DelayOperator) {
-      addVisited(operator, ctx);
-    } else {
-      for (PTOperator.PTOutput out : operator.getOutputs()) {
+    for (PTOperator groupOper : groupOpers) {
+      for (PTOperator.PTOutput out : groupOper.getOutputs()) {
         for (PTOperator.PTInput sink : out.sinks) {
           PTOperator sinkOperator = sink.target;
+          if (groupOpers.contains(sinkOperator)) {
+            continue; // downstream operator within group
+          }
           if (!ctx.visited.contains(sinkOperator)) {
             // downstream traversal
             updateRecoveryCheckpoints(sinkOperator, ctx);
@@ -1954,7 +1982,7 @@ public class StreamingContainerManager implements PlanContext
           // recovery window id cannot move backwards
           // when dynamically adding new operators
           if (sinkOperator.getRecoveryCheckpoint().windowId >= operator.getRecoveryCheckpoint().windowId) {
-            maxCheckpoint = Math.min(maxCheckpoint, sinkOperator.getRecoveryCheckpoint().windowId);
+            maxCheckpoint = Checkpoint.min(maxCheckpoint, sinkOperator.getRecoveryCheckpoint());
           }
 
           if (ctx.blocked.contains(sinkOperator)) {
@@ -1967,33 +1995,43 @@ public class StreamingContainerManager implements PlanContext
       }
     }
 
-    // checkpoint frozen during deployment
-    if (ctx.recovery || operator.getState() != PTOperator.State.PENDING_DEPLOY) {
-      // remove previous checkpoints
-      Checkpoint c1 = Checkpoint.INITIAL_CHECKPOINT;
-      synchronized (operator.checkpoints) {
-        if (!operator.checkpoints.isEmpty() && (operator.checkpoints.getFirst()).windowId <= maxCheckpoint) {
-          c1 = operator.checkpoints.getFirst();
-          Checkpoint c2;
-          while (operator.checkpoints.size() > 1 && ((c2 = operator.checkpoints.get(1)).windowId) <= maxCheckpoint) {
-            operator.checkpoints.removeFirst();
-            //LOG.debug("Checkpoint to delete: operator={} windowId={}", operator.getName(), c1);
-            this.purgeCheckpoints.add(new Pair<PTOperator, Long>(operator, c1.windowId));
-            c1 = c2;
+    // find the common checkpoint that is <= downstream recovery checkpoint
+    if (!commonCheckpoints.contains(maxCheckpoint)) {
+      if (!commonCheckpoints.isEmpty()) {
+        maxCheckpoint = Objects.firstNonNull(commonCheckpoints.floor(maxCheckpoint), maxCheckpoint);
+      }
+    }
+
+    for (PTOperator groupOper : groupOpers) {
+      // checkpoint frozen during deployment
+      if (ctx.recovery || groupOper.getState() != PTOperator.State.PENDING_DEPLOY) {
+        // remove previous checkpoints
+        Checkpoint c1 = Checkpoint.INITIAL_CHECKPOINT;
+        LinkedList<Checkpoint> checkpoints = groupOper.checkpoints;
+        synchronized (checkpoints) {
+          if (!checkpoints.isEmpty() && (checkpoints.getFirst()).windowId <= maxCheckpoint.windowId) {
+            c1 = checkpoints.getFirst();
+            Checkpoint c2;
+            while (checkpoints.size() > 1 && ((c2 = checkpoints.get(1)).windowId) <= maxCheckpoint.windowId) {
+              checkpoints.removeFirst();
+              //LOG.debug("Checkpoint to delete: operator={} windowId={}", operator.getName(), c1);
+              this.purgeCheckpoints.add(new Pair<PTOperator, Long>(groupOper, c1.windowId));
+              c1 = c2;
+            }
           }
-        }
-        else {
-          if (ctx.recovery && operator.checkpoints.isEmpty() && operator.isOperatorStateLess()) {
-            LOG.debug("Adding checkpoint for stateless operator {} {}", operator, Codec.getStringWindowId(maxCheckpoint));
-            c1 = operator.addCheckpoint(maxCheckpoint, this.vars.windowStartMillis);
+          else {
+            if (ctx.recovery && checkpoints.isEmpty() && groupOper.isOperatorStateLess()) {
+              LOG.debug("Adding checkpoint for stateless operator {} {}", groupOper, Codec.getStringWindowId(maxCheckpoint.windowId));
+              c1 = groupOper.addCheckpoint(maxCheckpoint.windowId, this.vars.windowStartMillis);
+            }
           }
         }
+        //LOG.debug("Operator {} checkpoints: commit {} recent {}", new Object[] {operator.getName(), c1, operator.checkpoints});
+        groupOper.setRecoveryCheckpoint(c1);
+      }
+      else {
+        LOG.debug("Skipping checkpoint update {} during {}", groupOper, groupOper.getState());
       }
-      //LOG.debug("Operator {} checkpoints: commit {} recent {}", new Object[] {operator.getName(), c1, operator.checkpoints});
-      operator.setRecoveryCheckpoint(c1);
-    }
-    else {
-      LOG.debug("Skipping checkpoint update {} during {}", operator, operator.getState());
     }
 
   }
@@ -2009,13 +2047,32 @@ public class StreamingContainerManager implements PlanContext
     return this.vars.windowStartMillis;
   }
 
+  private Map<OperatorMeta, Set<OperatorMeta>> getCheckpointGroups()
+  {
+    if (this.checkpointGroups == null) {
+      this.checkpointGroups = new HashMap<>();
+      LogicalPlan dag = this.plan.getLogicalPlan();
+      dag.resetNIndex();
+      LogicalPlan.ValidationContext vc = new LogicalPlan.ValidationContext();
+      for (OperatorMeta om : dag.getRootOperators()) {
+        this.plan.getLogicalPlan().findStronglyConnected(om, vc);
+      }
+      for (Set<OperatorMeta> checkpointGroup : vc.stronglyConnected) {
+        for (OperatorMeta om : checkpointGroup) {
+          this.checkpointGroups.put(om, checkpointGroup);
+        }
+      }
+    }
+    return checkpointGroups;
+  }
+
   /**
    * Visit all operators to update current checkpoint based on updated downstream state.
    * Purge older checkpoints that are no longer needed.
    */
   private long updateCheckpoints(boolean recovery)
   {
-    UpdateCheckpointsContext ctx = new UpdateCheckpointsContext(clock, recovery);
+    UpdateCheckpointsContext ctx = new UpdateCheckpointsContext(clock, recovery, getCheckpointGroups());
     for (OperatorMeta logicalOperator : plan.getLogicalPlan().getRootOperators()) {
       //LOG.debug("Updating checkpoints for operator {}", logicalOperator.getName());
       List<PTOperator> operators = plan.getOperators(logicalOperator);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b3402be5/engine/src/main/java/com/datatorrent/stram/api/Checkpoint.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/api/Checkpoint.java b/engine/src/main/java/com/datatorrent/stram/api/Checkpoint.java
index 5ec4a7e..d24b17a 100644
--- a/engine/src/main/java/com/datatorrent/stram/api/Checkpoint.java
+++ b/engine/src/main/java/com/datatorrent/stram/api/Checkpoint.java
@@ -18,6 +18,8 @@
  */
 package com.datatorrent.stram.api;
 
+import java.util.Comparator;
+
 import com.datatorrent.api.annotation.Stateless;
 import com.datatorrent.bufferserver.util.Codec;
 
@@ -102,6 +104,15 @@ public class Checkpoint implements com.datatorrent.api.Stats.Checkpoint
     return windowId;
   }
 
+  public static class CheckpointComparator implements Comparator<Checkpoint>
+  {
+    @Override
+    public int compare(Checkpoint o1, Checkpoint o2)
+    {
+      return Long.compare(o1.windowId, o2.windowId);
+    }
+  }
+
   @SuppressWarnings("FieldNameHidesFieldInSuperclass")
   public static final Checkpoint INITIAL_CHECKPOINT = new Checkpoint(Stateless.WINDOW_ID, 0, 0);
   private static final long serialVersionUID = 201402152116L;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b3402be5/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index 883ad71..6d7ebe1 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -159,8 +159,6 @@ public class LogicalPlan implements Serializable, DAG
   public final Map<String, ModuleMeta> modules = new LinkedHashMap<>();
   private final List<OperatorMeta> rootOperators = new ArrayList<OperatorMeta>();
   private final Attribute.AttributeMap attributes = new DefaultAttributeMap();
-  private transient int nodeIndex = 0; // used for cycle validation
-  private transient Stack<OperatorMeta> stack = new Stack<OperatorMeta>(); // used for cycle validation
   private transient Map<String, ArrayListMultimap<OutputPort<?>, InputPort<?>>> streamLinks = new HashMap<>();
 
   @Override
@@ -1540,6 +1538,7 @@ public class LogicalPlan implements Serializable, DAG
     return this.operators.get(operatorName);
   }
 
+  @Override
   public ModuleMeta getModuleMeta(String moduleName)
   {
     return this.modules.get(moduleName);
@@ -1557,6 +1556,7 @@ public class LogicalPlan implements Serializable, DAG
     throw new IllegalArgumentException("Operator not associated with the DAG: " + operator);
   }
 
+  @Override
   public ModuleMeta getMeta(Module module)
   {
     for (ModuleMeta m : getAllModules()) {
@@ -1626,6 +1626,24 @@ public class LogicalPlan implements Serializable, DAG
     return classNames;
   }
 
+  public static class ValidationContext
+  {
+    public int nodeIndex = 0;
+    public Stack<OperatorMeta> stack = new Stack<OperatorMeta>();
+    public Stack<OperatorMeta> path = new Stack<OperatorMeta>();
+    public List<Set<OperatorMeta>> stronglyConnected = new ArrayList<>();
+    public OperatorMeta invalidLoopAt;
+    public List<Set<OperatorMeta>> invalidCycles = new ArrayList<>();
+  }
+
+  public void resetNIndex()
+  {
+    for (OperatorMeta om : getAllOperators()) {
+      om.lowlink = null;
+      om.nindex = null;
+    }
+  }
+
   /**
    * Validate the plan. Includes checks that required ports are connected,
    * required configuration parameters specified, graph free of cycles etc.
@@ -1752,21 +1770,20 @@ public class LogicalPlan implements Serializable, DAG
         throw new ValidationException("At least one output port must be connected: " + n.name);
       }
     }
-    stack = new Stack<OperatorMeta>();
 
-    List<List<String>> cycles = new ArrayList<List<String>>();
+    ValidationContext validatonContext = new ValidationContext();
     for (OperatorMeta n: operators.values()) {
       if (n.nindex == null) {
-        findStronglyConnected(n, cycles);
+        findStronglyConnected(n, validatonContext);
       }
     }
-    if (!cycles.isEmpty()) {
-      throw new ValidationException("Loops in graph: " + cycles);
+    if (!validatonContext.invalidCycles.isEmpty()) {
+      throw new ValidationException("Loops in graph: " + validatonContext.invalidCycles);
     }
 
     List<List<String>> invalidDelays = new ArrayList<>();
     for (OperatorMeta n : rootOperators) {
-      findInvalidDelays(n, invalidDelays);
+      findInvalidDelays(n, invalidDelays, new Stack<OperatorMeta>());
     }
     if (!invalidDelays.isEmpty()) {
       throw new ValidationException("Invalid delays in graph: " + invalidDelays);
@@ -1908,59 +1925,72 @@ public class LogicalPlan implements Serializable, DAG
    * @param om
    * @param cycles
    */
-  public void findStronglyConnected(OperatorMeta om, List<List<String>> cycles)
+  public void findStronglyConnected(OperatorMeta om, ValidationContext ctx)
   {
-    om.nindex = nodeIndex;
-    om.lowlink = nodeIndex;
-    nodeIndex++;
-    stack.push(om);
+    om.nindex = ctx.nodeIndex;
+    om.lowlink = ctx.nodeIndex;
+    ctx.nodeIndex++;
+    ctx.stack.push(om);
+    ctx.path.push(om);
 
     // depth first successors traversal
     for (StreamMeta downStream: om.outputStreams.values()) {
       for (InputPortMeta sink: downStream.sinks) {
-        if (om.getOperator() instanceof Operator.DelayOperator) {
-          // this is an iteration loop, do not treat it as downstream when detecting cycles
-          sink.attributes.put(IS_CONNECTED_TO_DELAY_OPERATOR, true);
-          continue;
-        }
         OperatorMeta successor = sink.getOperatorWrapper();
         if (successor == null) {
           continue;
         }
         // check for self referencing node
         if (om == successor) {
-          cycles.add(Collections.singletonList(om.name));
+          ctx.invalidCycles.add(Collections.singleton(om));
         }
         if (successor.nindex == null) {
           // not visited yet
-          findStronglyConnected(successor, cycles);
+          findStronglyConnected(successor, ctx);
           om.lowlink = Math.min(om.lowlink, successor.lowlink);
         }
-        else if (stack.contains(successor)) {
+        else if (ctx.stack.contains(successor)) {
           om.lowlink = Math.min(om.lowlink, successor.nindex);
+          boolean isDelayLoop = false;
+          for (int i=ctx.path.size(); i>0; i--) {
+            OperatorMeta om2 = ctx.path.get(i-1);
+            if (om2.getOperator() instanceof Operator.DelayOperator) {
+              isDelayLoop = true;
+            }
+            if (om2 == successor) {
+              break;
+            }
+          }
+          if (!isDelayLoop) {
+            ctx.invalidLoopAt = successor;
+          }
         }
       }
     }
 
     // pop stack for all root operators
     if (om.lowlink.equals(om.nindex)) {
-      List<String> connectedIds = new ArrayList<String>();
-      while (!stack.isEmpty()) {
-        OperatorMeta n2 = stack.pop();
-        connectedIds.add(n2.name);
+      Set<OperatorMeta> connectedSet = new LinkedHashSet<>(ctx.stack.size());
+      while (!ctx.stack.isEmpty()) {
+        OperatorMeta n2 = ctx.stack.pop();
+        connectedSet.add(n2);
         if (n2 == om) {
           break; // collected all connected operators
         }
       }
       // strongly connected (cycle) if more than one node in stack
-      if (connectedIds.size() > 1) {
-        LOG.debug("detected cycle from node {}: {}", om.name, connectedIds);
-        cycles.add(connectedIds);
+      if (connectedSet.size() > 1) {
+        ctx.stronglyConnected.add(connectedSet);
+        if (connectedSet.contains(ctx.invalidLoopAt)) {
+          ctx.invalidCycles.add(connectedSet);
+        }
       }
     }
+    ctx.path.pop();
+
   }
 
-  public void findInvalidDelays(OperatorMeta om, List<List<String>> invalidDelays)
+  public void findInvalidDelays(OperatorMeta om, List<List<String>> invalidDelays, Stack<OperatorMeta> stack)
   {
     stack.push(om);
 
@@ -1977,6 +2007,7 @@ public class LogicalPlan implements Serializable, DAG
       for (InputPortMeta sink : downStream.sinks) {
         OperatorMeta successor = sink.getOperatorWrapper();
         if (isDelayOperator) {
+          sink.attributes.put(IS_CONNECTED_TO_DELAY_OPERATOR, true);
           // Check whether all downstream operators are already visited in the path
           if (successor != null && !stack.contains(successor)) {
             LOG.debug("detected DelayOperator does not immediately output to a visited operator {}.{}->{}.{}",
@@ -1984,7 +2015,7 @@ public class LogicalPlan implements Serializable, DAG
             invalidDelays.add(Arrays.asList(om.getName(), successor.getName()));
           }
         } else {
-          findInvalidDelays(successor, invalidDelays);
+          findInvalidDelays(successor, invalidDelays, stack);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b3402be5/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java b/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java
index ee3cbc3..5675b53 100644
--- a/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java
@@ -56,6 +56,7 @@ import com.datatorrent.stram.api.StreamingContainerUmbilicalProtocol.OperatorHea
 import com.datatorrent.stram.engine.GenericTestOperator;
 import com.datatorrent.stram.engine.OperatorContext;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
+import com.datatorrent.stram.plan.logical.LogicalPlan.OperatorMeta;
 import com.datatorrent.stram.plan.physical.PTContainer;
 import com.datatorrent.stram.plan.physical.PTOperator;
 import com.datatorrent.stram.plan.physical.PhysicalPlan;
@@ -314,7 +315,7 @@ public class CheckpointTest
     o4p1.checkpoints.add(leafCheckpoint);
 
     UpdateCheckpointsContext ctx;
-    dnm.updateRecoveryCheckpoints(o1p1, ctx = new UpdateCheckpointsContext(clock, true));
+    dnm.updateRecoveryCheckpoints(o1p1, ctx = new UpdateCheckpointsContext(clock, true, Collections.<OperatorMeta, Set<OperatorMeta>>emptyMap()));
     Assert.assertEquals("initial checkpoint " + o1p1, Checkpoint.INITIAL_CHECKPOINT, o1p1.getRecoveryCheckpoint());
     Assert.assertEquals("initial checkpoint " + o2SLp1, leafCheckpoint, o2SLp1.getRecoveryCheckpoint());
     Assert.assertEquals("initial checkpoint " + o3SLp1, new Checkpoint(clock.getTime(), 0, 0), o3SLp1.getRecoveryCheckpoint());

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b3402be5/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
index 359da17..06f184f 100644
--- a/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
@@ -20,7 +20,11 @@ package com.datatorrent.stram.plan.logical;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Stack;
 import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.locks.Lock;
@@ -32,8 +36,14 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Ignore;
+import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+
+import com.google.common.collect.Sets;
+
 import com.datatorrent.api.Context;
 import com.datatorrent.api.DAG;
 import com.datatorrent.api.DefaultInputPort;
@@ -42,8 +52,17 @@ import com.datatorrent.api.Operator;
 import com.datatorrent.common.util.BaseOperator;
 import com.datatorrent.common.util.DefaultDelayOperator;
 import com.datatorrent.stram.StramLocalCluster;
+import com.datatorrent.stram.StreamingContainerManager;
+import com.datatorrent.stram.StreamingContainerManager.UpdateCheckpointsContext;
+import com.datatorrent.stram.api.Checkpoint;
 import com.datatorrent.stram.engine.GenericTestOperator;
 import com.datatorrent.stram.engine.TestGeneratorInputOperator;
+import com.datatorrent.stram.plan.logical.LogicalPlan.OperatorMeta;
+import com.datatorrent.stram.plan.physical.PTOperator;
+import com.datatorrent.stram.plan.physical.PhysicalPlan;
+import com.datatorrent.stram.support.StramTestSupport;
+import com.datatorrent.stram.support.StramTestSupport.MemoryStorageAgent;
+import com.datatorrent.stram.support.StramTestSupport.TestMeta;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
@@ -75,7 +94,7 @@ public class DelayOperatorTest
     GenericTestOperator opB = dag.addOperator("B", GenericTestOperator.class);
     GenericTestOperator opC = dag.addOperator("C", GenericTestOperator.class);
     GenericTestOperator opD = dag.addOperator("D", GenericTestOperator.class);
-    DefaultDelayOperator opDelay = dag.addOperator("opDelay", DefaultDelayOperator.class);
+    DefaultDelayOperator<Object> opDelay = dag.addOperator("opDelay", DefaultDelayOperator.class);
 
     dag.addStream("BtoC", opB.outport1, opC.inport1);
     dag.addStream("CtoD", opC.outport1, opD.inport1);
@@ -83,7 +102,7 @@ public class DelayOperatorTest
     dag.addStream("DelayToD", opDelay.output, opD.inport2);
 
     List<List<String>> invalidDelays = new ArrayList<>();
-    dag.findInvalidDelays(dag.getMeta(opB), invalidDelays);
+    dag.findInvalidDelays(dag.getMeta(opB), invalidDelays, new Stack<OperatorMeta>());
     assertEquals("operator invalid delay", 1, invalidDelays.size());
 
     try {
@@ -106,7 +125,7 @@ public class DelayOperatorTest
     dag.addStream("DelayToC", opDelay.output, opC.inport2);
 
     invalidDelays = new ArrayList<>();
-    dag.findInvalidDelays(dag.getMeta(opB), invalidDelays);
+    dag.findInvalidDelays(dag.getMeta(opB), invalidDelays, new Stack<OperatorMeta>());
     assertEquals("operator invalid delay", 1, invalidDelays.size());
 
     try {
@@ -373,5 +392,68 @@ public class DelayOperatorTest
         Arrays.copyOfRange(new TreeSet<>(FibonacciOperator.results).toArray(), 0, 20));
   }
 
+  @Rule
+  public TestMeta testMeta = new TestMeta();
+
+  @Test
+  public void testCheckpointUpdate()
+  {
+    LogicalPlan dag = StramTestSupport.createDAG(testMeta);
+
+    TestGeneratorInputOperator opA = dag.addOperator("A", TestGeneratorInputOperator.class);
+    GenericTestOperator opB = dag.addOperator("B", GenericTestOperator.class);
+    GenericTestOperator opC = dag.addOperator("C", GenericTestOperator.class);
+    GenericTestOperator opD = dag.addOperator("D", GenericTestOperator.class);
+    DefaultDelayOperator<Object> opDelay = dag.addOperator("opDelay", new DefaultDelayOperator<>());
+
+    dag.addStream("AtoB", opA.outport, opB.inport1);
+    dag.addStream("BtoC", opB.outport1, opC.inport1);
+    dag.addStream("CtoD", opC.outport1, opD.inport1);
+    dag.addStream("CtoDelay", opC.outport2, opDelay.input);
+    dag.addStream("DelayToB", opDelay.output, opB.inport2);
+    dag.validate();
+
+    dag.setAttribute(com.datatorrent.api.Context.OperatorContext.STORAGE_AGENT, new MemoryStorageAgent());
+    StreamingContainerManager scm = new StreamingContainerManager(dag);
+    PhysicalPlan plan = scm.getPhysicalPlan();
+    // set all operators as active to enable recovery window id update
+    for (PTOperator oper : plan.getAllOperators().values()) {
+      oper.setState(PTOperator.State.ACTIVE);
+    }
+
+    Clock clock = new SystemClock();
+
+    PTOperator opA1 = plan.getOperators(dag.getMeta(opA)).get(0);
+    PTOperator opB1 = plan.getOperators(dag.getMeta(opB)).get(0);
+    PTOperator opC1 = plan.getOperators(dag.getMeta(opC)).get(0);
+    PTOperator opDelay1 = plan.getOperators(dag.getMeta(opDelay)).get(0);
+    PTOperator opD1 = plan.getOperators(dag.getMeta(opD)).get(0);
+
+    Checkpoint cp3 = new Checkpoint(3L, 0, 0);
+    Checkpoint cp5 = new Checkpoint(5L, 0, 0);
+    Checkpoint cp4 = new Checkpoint(4L, 0, 0);
+
+    opB1.checkpoints.add(cp3);
+    opC1.checkpoints.add(cp3);
+    opC1.checkpoints.add(cp4);
+    opDelay1.checkpoints.add(cp3);
+    opDelay1.checkpoints.add(cp5);
+    opD1.checkpoints.add(cp5);
+    // construct grouping that would be supplied through LogicalPlan
+    Set<OperatorMeta> stronglyConnected = Sets.newHashSet(dag.getMeta(opB), dag.getMeta(opC), dag.getMeta(opDelay));
+    Map<OperatorMeta, Set<OperatorMeta>> groups = new HashMap<>();
+    for (OperatorMeta om : stronglyConnected) {
+      groups.put(om, stronglyConnected);
+    }
+
+    UpdateCheckpointsContext ctx = new UpdateCheckpointsContext(clock, false, groups);
+    scm.updateRecoveryCheckpoints(opB1, ctx);
+
+    Assert.assertEquals("checkpoint " + opA1, Checkpoint.INITIAL_CHECKPOINT, opA1.getRecoveryCheckpoint());
+    Assert.assertEquals("checkpoint " + opB1, cp3, opC1.getRecoveryCheckpoint());
+    Assert.assertEquals("checkpoint " + opC1, cp3, opC1.getRecoveryCheckpoint());
+    Assert.assertEquals("checkpoint " + opD1, cp5, opD1.getRecoveryCheckpoint());
+
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b3402be5/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanTest.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanTest.java
index a4ac488..9383f12 100644
--- a/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanTest.java
@@ -19,6 +19,7 @@
 package com.datatorrent.stram.plan.logical;
 
 import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.common.util.DefaultDelayOperator;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -35,6 +36,7 @@ import javax.validation.constraints.Pattern;
 import com.esotericsoftware.kryo.DefaultSerializer;
 import com.esotericsoftware.kryo.serializers.JavaSerializer;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -43,7 +45,6 @@ import static org.junit.Assert.*;
 
 import com.datatorrent.common.partitioner.StatelessPartitioner;
 import com.datatorrent.api.*;
-import com.datatorrent.api.Context.DAGContext;
 import com.datatorrent.api.Context.OperatorContext;
 import com.datatorrent.api.Context.PortContext;
 import com.datatorrent.api.DAG.Locality;
@@ -61,10 +62,12 @@ import com.datatorrent.stram.plan.logical.LogicalPlan.StreamMeta;
 import com.datatorrent.stram.support.StramTestSupport.MemoryStorageAgent;
 import com.datatorrent.stram.support.StramTestSupport.RegexMatcher;
 
-public class LogicalPlanTest {
+public class LogicalPlanTest
+{
 
   @Test
-  public void testCycleDetection() {
+  public void testCycleDetection()
+  {
      LogicalPlan dag = new LogicalPlan();
 
      //NodeConf operator1 = b.getOrAddNode("operator1");
@@ -91,20 +94,20 @@ public class LogicalPlanTest {
        // expected, stream can have single input/output only
      }
 
-     List<List<String>> cycles = new ArrayList<List<String>>();
-     dag.findStronglyConnected(dag.getMeta(operator7), cycles);
-     assertEquals("operator self reference", 1, cycles.size());
-     assertEquals("operator self reference", 1, cycles.get(0).size());
-     assertEquals("operator self reference", dag.getMeta(operator7).getName(), cycles.get(0).get(0));
+     LogicalPlan.ValidationContext vc = new LogicalPlan.ValidationContext();
+     dag.findStronglyConnected(dag.getMeta(operator7), vc);
+     assertEquals("operator self reference", 1, vc.invalidCycles.size());
+     assertEquals("operator self reference", 1, vc.invalidCycles.get(0).size());
+     assertEquals("operator self reference", dag.getMeta(operator7), vc.invalidCycles.get(0).iterator().next());
 
      // 3 operator cycle
-     cycles.clear();
-     dag.findStronglyConnected(dag.getMeta(operator4), cycles);
-     assertEquals("3 operator cycle", 1, cycles.size());
-     assertEquals("3 operator cycle", 3, cycles.get(0).size());
-     assertTrue("operator2", cycles.get(0).contains(dag.getMeta(operator2).getName()));
-     assertTrue("operator3", cycles.get(0).contains(dag.getMeta(operator3).getName()));
-     assertTrue("operator4", cycles.get(0).contains(dag.getMeta(operator4).getName()));
+     vc = new LogicalPlan.ValidationContext();
+     dag.findStronglyConnected(dag.getMeta(operator4), vc);
+     assertEquals("3 operator cycle", 1, vc.invalidCycles.size());
+     assertEquals("3 operator cycle", 3, vc.invalidCycles.get(0).size());
+     assertTrue("operator2", vc.invalidCycles.get(0).contains(dag.getMeta(operator2)));
+     assertTrue("operator3", vc.invalidCycles.get(0).contains(dag.getMeta(operator3)));
+     assertTrue("operator4", vc.invalidCycles.get(0).contains(dag.getMeta(operator4)));
 
      try {
        dag.validate();
@@ -115,13 +118,44 @@ public class LogicalPlanTest {
 
   }
 
-  public static class ValidationOperator extends BaseOperator {
+  @Test
+  public void testCycleDetectionWithDelay()
+  {
+    LogicalPlan dag = new LogicalPlan();
+
+    TestGeneratorInputOperator opA = dag.addOperator("A", TestGeneratorInputOperator.class);
+    GenericTestOperator opB = dag.addOperator("B", GenericTestOperator.class);
+    GenericTestOperator opC = dag.addOperator("C", GenericTestOperator.class);
+    GenericTestOperator opD = dag.addOperator("D", GenericTestOperator.class);
+    DefaultDelayOperator<Object> opDelay = dag.addOperator("opDelay", new DefaultDelayOperator<>());
+    DefaultDelayOperator<Object> opDelay2 = dag.addOperator("opDelay2", new DefaultDelayOperator<>());
+
+    dag.addStream("AtoB", opA.outport, opB.inport1);
+    dag.addStream("BtoC", opB.outport1, opC.inport1);
+    dag.addStream("CtoD", opC.outport1, opD.inport1);
+    dag.addStream("CtoDelay", opC.outport2, opDelay.input);
+    dag.addStream("DtoDelay", opD.outport1, opDelay2.input);
+    dag.addStream("DelayToB", opDelay.output, opB.inport2);
+    dag.addStream("Delay2ToC", opDelay2.output, opC.inport2);
+
+    LogicalPlan.ValidationContext vc = new LogicalPlan.ValidationContext();
+    dag.findStronglyConnected(dag.getMeta(opA), vc);
+
+    Assert.assertEquals("No invalid cycle", Collections.emptyList(), vc.invalidCycles);
+    Set<OperatorMeta> exp = Sets.newHashSet(dag.getMeta(opDelay2), dag.getMeta(opDelay), dag.getMeta(opC), dag.getMeta(opB), dag.getMeta(opD));
+    Assert.assertEquals("cycle", exp, vc.stronglyConnected.get(0));
+  }
+
+
+  public static class ValidationOperator extends BaseOperator
+  {
     public final transient DefaultOutputPort<Object> goodOutputPort = new DefaultOutputPort<Object>();
 
     public final transient DefaultOutputPort<Object> badOutputPort = new DefaultOutputPort<Object>();
   }
 
-  public static class CounterOperator extends BaseOperator {
+  public static class CounterOperator extends BaseOperator
+  {
     final public transient InputPort<Object> countInputPort = new DefaultInputPort<Object>() {
       @Override
       final public void process(Object payload) {
@@ -130,8 +164,8 @@ public class LogicalPlanTest {
   }
 
   @Test
-  public void testLogicalPlanSerialization() throws Exception {
-
+  public void testLogicalPlanSerialization() throws Exception
+  {
     LogicalPlan dag = new LogicalPlan();
     dag.setAttribute(OperatorContext.STORAGE_AGENT, new MemoryStorageAgent());
 
@@ -188,7 +222,8 @@ public class LogicalPlanTest {
     Assert.assertEquals("", 2, dag.getAllOperators().size());
   }
 
-  public static class ValidationTestOperator extends BaseOperator implements InputOperator {
+  public static class ValidationTestOperator extends BaseOperator implements InputOperator
+  {
     @NotNull
     @Pattern(regexp=".*malhar.*", message="Value has to contain 'malhar'!")
     private String stringField1;
@@ -271,8 +306,8 @@ public class LogicalPlanTest {
   }
 
   @Test
-  public void testOperatorValidation() {
-
+  public void testOperatorValidation()
+  {
     ValidationTestOperator bean = new ValidationTestOperator();
     bean.stringField1 = "malhar1";
     bean.intField1 = 1;
@@ -348,7 +383,8 @@ public class LogicalPlanTest {
   }
 
   @OperatorAnnotation(partitionable = false)
-  public static class TestOperatorAnnotationOperator extends BaseOperator {
+  public static class TestOperatorAnnotationOperator extends BaseOperator
+  {
 
     @InputPortFieldAnnotation( optional = true)
     final public transient DefaultInputPort<Object> input1 = new DefaultInputPort<Object>() {
@@ -358,11 +394,13 @@ public class LogicalPlanTest {
     };
   }
 
-  class NoInputPortOperator extends BaseOperator {
+  class NoInputPortOperator extends BaseOperator
+  {
   }
 
   @Test
-  public void testValidationForNonInputRootOperator() {
+  public void testValidationForNonInputRootOperator()
+  {
     LogicalPlan dag = new LogicalPlan();
     NoInputPortOperator x = dag.addOperator("x", new NoInputPortOperator());
     try {
@@ -374,8 +412,8 @@ public class LogicalPlanTest {
   }
 
   @OperatorAnnotation(partitionable = false)
-  public static class TestOperatorAnnotationOperator2 extends BaseOperator implements Partitioner<TestOperatorAnnotationOperator2> {
-
+  public static class TestOperatorAnnotationOperator2 extends BaseOperator implements Partitioner<TestOperatorAnnotationOperator2>
+  {
     @Override
     public Collection<Partition<TestOperatorAnnotationOperator2>> definePartitions(Collection<Partition<TestOperatorAnnotationOperator2>> partitions, PartitioningContext context)
     {
@@ -389,7 +427,8 @@ public class LogicalPlanTest {
   }
 
   @Test
-  public void testOperatorAnnotation() {
+  public void testOperatorAnnotation()
+  {
     LogicalPlan dag = new LogicalPlan();
     TestGeneratorInputOperator input = dag.addOperator("input1", TestGeneratorInputOperator.class);
     TestOperatorAnnotationOperator operator = dag.addOperator("operator1", TestOperatorAnnotationOperator.class);
@@ -430,8 +469,8 @@ public class LogicalPlanTest {
   }
 
   @Test
-  public void testPortConnectionValidation() {
-
+  public void testPortConnectionValidation()
+  {
     LogicalPlan dag = new LogicalPlan();
 
     TestNonOptionalOutportInputOperator input = dag.addOperator("input1", TestNonOptionalOutportInputOperator.class);
@@ -459,7 +498,8 @@ public class LogicalPlanTest {
   }
 
   @Test
-  public void testAtMostOnceProcessingModeValidation() {
+  public void testAtMostOnceProcessingModeValidation()
+  {
     LogicalPlan dag = new LogicalPlan();
 
     TestGeneratorInputOperator input1 = dag.addOperator("input1", TestGeneratorInputOperator.class);
@@ -489,8 +529,9 @@ public class LogicalPlanTest {
 
   }
 
-    @Test
-  public void testExactlyOnceProcessingModeValidation() {
+  @Test
+  public void testExactlyOnceProcessingModeValidation()
+  {
     LogicalPlan dag = new LogicalPlan();
 
     TestGeneratorInputOperator input1 = dag.addOperator("input1", TestGeneratorInputOperator.class);
@@ -527,7 +568,8 @@ public class LogicalPlanTest {
   }
 
   @Test
-  public void testLocalityValidation() {
+  public void testLocalityValidation()
+  {
     LogicalPlan dag = new LogicalPlan();
 
     TestGeneratorInputOperator input1 = dag.addOperator("input1", TestGeneratorInputOperator.class);
@@ -549,7 +591,8 @@ public class LogicalPlanTest {
     dag.validate();
   }
 
-  private class TestAnnotationsOperator extends BaseOperator implements InputOperator {
+  private class TestAnnotationsOperator extends BaseOperator implements InputOperator
+  {
     //final public transient DefaultOutputPort<Object> outport1 = new DefaultOutputPort<Object>();
 
     @OutputPortFieldAnnotation( optional=false)
@@ -562,7 +605,8 @@ public class LogicalPlanTest {
     }
   }
 
-  private class TestAnnotationsOperator2 extends BaseOperator implements InputOperator{
+  private class TestAnnotationsOperator2 extends BaseOperator implements InputOperator
+  {
     // multiple ports w/o annotation, one of them must be connected
     final public transient DefaultOutputPort<Object> outport1 = new DefaultOutputPort<Object>();
 
@@ -573,7 +617,8 @@ public class LogicalPlanTest {
     }
   }
 
-  private class TestAnnotationsOperator3 extends BaseOperator implements InputOperator{
+  private class TestAnnotationsOperator3 extends BaseOperator implements InputOperator
+  {
     // multiple ports w/o annotation, one of them must be connected
     @OutputPortFieldAnnotation( optional=true)
     final public transient DefaultOutputPort<Object> outport1 = new DefaultOutputPort<Object>();
@@ -587,7 +632,8 @@ public class LogicalPlanTest {
   }
 
   @Test
-  public void testOutputPortAnnotation() {
+  public void testOutputPortAnnotation()
+  {
     LogicalPlan dag = new LogicalPlan();
     TestAnnotationsOperator ta1 = dag.addOperator("testAnnotationsOperator", new TestAnnotationsOperator());
 
@@ -623,7 +669,8 @@ public class LogicalPlanTest {
    * Operator that can be used with default Java serialization instead of Kryo
    */
   @DefaultSerializer(JavaSerializer.class)
-  public static class JdkSerializableOperator extends BaseOperator implements Serializable {
+  public static class JdkSerializableOperator extends BaseOperator implements Serializable
+  {
     private static final long serialVersionUID = -4024202339520027097L;
 
     public abstract class SerializableInputPort<T> implements InputPort<T>, Sink<T>, java.io.Serializable {
@@ -673,7 +720,8 @@ public class LogicalPlanTest {
   }
 
   @Test
-  public void testJdkSerializableOperator() throws Exception {
+  public void testJdkSerializableOperator() throws Exception
+  {
     LogicalPlan dag = new LogicalPlan();
     dag.addOperator("o1", new JdkSerializableOperator());
 
@@ -785,7 +833,8 @@ public class LogicalPlanTest {
     }
   }
 
-  public static class TestPortCodecOperator extends BaseOperator {
+  public static class TestPortCodecOperator extends BaseOperator
+  {
     public transient final DefaultInputPort<Object> inport1 = new DefaultInputPort<Object>()
     {
       @Override



[07/50] incubator-apex-core git commit: Update JIRA and install links.

Posted by th...@apache.org.
Update JIRA and install links.


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

Branch: refs/heads/master
Commit: 0d5bfa5be27128de0be89335346c7c3cac40e31a
Parents: 0b0f914
Author: Thomas Weise <th...@datatorrent.com>
Authored: Mon Dec 14 11:34:34 2015 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Mon Dec 14 11:34:34 2015 -0800

----------------------------------------------------------------------
 README.md | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/0d5bfa5b/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index a4a2b46..0b9136f 100644
--- a/README.md
+++ b/README.md
@@ -29,20 +29,20 @@ at the top level. You can then use the command line interface (CLI) from the bui
 ```
 Type help to list available commands. 
 
-Pre-built distributions (see [README](https://www.datatorrent.com/docs/README.html)) are available from
+Pre-built distributions (see [README](https://docs.datatorrent.com/installation)) are available from
 https://www.datatorrent.com/download/
 
 ##Issue tracking
 
 (Note that we will be moving to the Apache JIRA system soon.)
 
-[Apex JIRA](https://malhar.atlassian.net/projects/APEX) issue tracking system is used for this project.
+[Apex JIRA](https://issues.apache.org/jira/browse/APEXCORE) issue tracking system is used for this project.
 You can submit new issues and track the progress of existing issues at https://malhar.atlassian.net/projects/APEX.
 
 When working with JIRA to submit pull requests, please use [smart commits](https://confluence.atlassian.com/display/AOD/Processing+JIRA+issues+with+commit+messages) feature by specifying APEX-XXXX in the commit messages.
 It helps us link commits with issues being tracked for easy reference.  And example commit might look like this:
 
-    git commit -am "APEX-1234 #comment Task completed ahead of schedule #resolve"
+    git commit -am "APEXCORE-1234 #comment Task completed ahead of schedule #resolve"
 
 ##License
 


[20/50] incubator-apex-core git commit: checkstyle and semver maven plugin upgrade

Posted by th...@apache.org.
checkstyle and semver maven plugin upgrade


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

Branch: refs/heads/master
Commit: 07bc9690e2517be4dd7ba04c3a30c55202f4f1f0
Parents: 3f76dcb
Author: MalharJenkins <je...@datatorrent.com>
Authored: Mon Dec 28 10:00:13 2015 -0800
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Wed Dec 30 21:01:04 2015 -0800

----------------------------------------------------------------------
 engine/pom.xml | 2 +-
 pom.xml        | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/07bc9690/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 6bf8283..5c59dce 100644
--- a/engine/pom.xml
+++ b/engine/pom.xml
@@ -145,7 +145,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>4400</maxAllowedViolations>
+          <maxAllowedViolations>4402</maxAllowedViolations>
         </configuration>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/07bc9690/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4cd1a5d..89c60c9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -306,7 +306,7 @@
             <dependency>
               <groupId>com.puppycrawl.tools</groupId>
               <artifactId>checkstyle</artifactId>
-              <version>6.13</version>
+              <version>6.14</version>
             </dependency>
           </dependencies>
           <executions>
@@ -336,7 +336,7 @@
         <plugin>
           <groupId>com.github.siom79.japicmp</groupId>
           <artifactId>japicmp-maven-plugin</artifactId>
-          <version>0.6.1</version>
+          <version>0.6.2</version>
           <configuration>
             <parameter>
               <onlyModified>true</onlyModified>


[41/50] incubator-apex-core git commit: Fix rawtype warnings.

Posted by th...@apache.org.
Fix rawtype warnings.


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

Branch: refs/heads/master
Commit: d0b4bdb24e77a08181e07f13d7c5794d39c6203c
Parents: c284c39
Author: Thomas Weise <th...@datatorrent.com>
Authored: Wed Jan 20 10:57:54 2016 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Wed Jan 20 10:59:36 2016 -0800

----------------------------------------------------------------------
 .../stram/plan/physical/PhysicalPlan.java       | 24 +++++++++++---------
 1 file changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d0b4bdb2/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
index 7858ea0..829a6fd 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
@@ -206,10 +206,11 @@ public class PhysicalPlan implements Serializable
       p.statsListeners = this.statsHandlers;
     }
 
+    /**
+     * Return all partitions and unifiers, except MxN unifiers
+     * @return
+     */
     private Collection<PTOperator> getAllOperators() {
-//      if (partitions.size() == 1) {
-//        return Collections.singletonList(partitions.get(0));
-//      }
       Collection<PTOperator> c = new ArrayList<PTOperator>(partitions.size() + 1);
       c.addAll(partitions);
       for (StreamMapping ug : outputStreams.values()) {
@@ -391,7 +392,7 @@ public class PhysicalPlan implements Serializable
         for (StreamMeta s : n.getOutputStreams().values()) {
           if (s.getPersistOperator() != null) {
             InputPortMeta persistInputPort = s.getPersistOperatorInputPort();
-            StreamCodecWrapperForPersistance persistCodec = (StreamCodecWrapperForPersistance) persistInputPort.getAttributes().get(PortContext.STREAM_CODEC);
+            StreamCodecWrapperForPersistance<?> persistCodec = (StreamCodecWrapperForPersistance<?>) persistInputPort.getAttributes().get(PortContext.STREAM_CODEC);
             if (persistCodec == null)
               continue;
             // Logging is enabled for the stream
@@ -403,10 +404,10 @@ public class PhysicalPlan implements Serializable
           // Check partitioning for persist operators per sink too
           for (Entry<InputPortMeta, InputPortMeta> entry : s.sinkSpecificPersistInputPortMap.entrySet()) {
             InputPortMeta persistInputPort = entry.getValue();
-            StreamCodec codec = persistInputPort.getAttributes().get(PortContext.STREAM_CODEC);
+            StreamCodec<?> codec = persistInputPort.getAttributes().get(PortContext.STREAM_CODEC);
             if (codec != null) {
               if (codec instanceof StreamCodecWrapperForPersistance) {
-                StreamCodecWrapperForPersistance persistCodec = (StreamCodecWrapperForPersistance) codec;
+                StreamCodecWrapperForPersistance<?> persistCodec = (StreamCodecWrapperForPersistance<?>) codec;
                 updatePersistOperatorWithSinkPartitions(persistInputPort, s.sinkSpecificPersistOperatorMap.get(entry.getKey()), persistCodec, entry.getKey());
               }
             }
@@ -418,7 +419,7 @@ public class PhysicalPlan implements Serializable
     }
   }
 
-  private void updatePersistOperatorWithSinkPartitions(InputPortMeta persistInputPort, OperatorMeta persistOperatorMeta, StreamCodecWrapperForPersistance persistCodec, InputPortMeta sinkPortMeta)
+  private void updatePersistOperatorWithSinkPartitions(InputPortMeta persistInputPort, OperatorMeta persistOperatorMeta, StreamCodecWrapperForPersistance<?> persistCodec, InputPortMeta sinkPortMeta)
   {
     Collection<PTOperator> ptOperators = getOperators(sinkPortMeta.getOperatorWrapper());
     Collection<PartitionKeys> partitionKeysList = new ArrayList<PartitionKeys>();
@@ -437,7 +438,7 @@ public class PhysicalPlan implements Serializable
       for (OperatorMeta n : dag.getAllOperators()) {
         for (StreamMeta s : n.getOutputStreams().values()) {
           if (s.getPersistOperator() != null) {
-            Map<InputPortMeta, StreamCodec<Object>> inputStreamCodecs = new HashMap<InputPortMeta, StreamCodec<Object>>();
+            Map<InputPortMeta, StreamCodec<?>> inputStreamCodecs = new HashMap<>();
             // Logging is enabled for the stream
             for (InputPortMeta portMeta : s.getSinksToPersist()) {
               InputPort<?> port = portMeta.getPortObject();
@@ -452,7 +453,7 @@ public class PhysicalPlan implements Serializable
                   }
                 }
                 if (!alreadyAdded) {
-                  inputStreamCodecs.put(portMeta, (StreamCodec<Object>) inputStreamCodec);
+                  inputStreamCodecs.put(portMeta, inputStreamCodec);
                 }
               }
             }
@@ -466,8 +467,9 @@ public class PhysicalPlan implements Serializable
               // Create Wrapper codec for Stream persistence using all unique
               // stream codecs
               // Logger should write merged or union of all input stream codecs
-              StreamCodec<Object> specifiedCodecForLogger = (s.getPersistOperatorInputPort().getValue(PortContext.STREAM_CODEC) != null) ? (StreamCodec<Object>) s.getPersistOperatorInputPort().getValue(PortContext.STREAM_CODEC) : (StreamCodec<Object>) s.getPersistOperatorInputPort().getPortObject().getStreamCodec();
-              StreamCodecWrapperForPersistance<Object> codec = new StreamCodecWrapperForPersistance<Object>(inputStreamCodecs, specifiedCodecForLogger);
+              StreamCodec<?> specifiedCodecForLogger = (s.getPersistOperatorInputPort().getValue(PortContext.STREAM_CODEC) != null) ? s.getPersistOperatorInputPort().getValue(PortContext.STREAM_CODEC) : s.getPersistOperatorInputPort().getPortObject().getStreamCodec();
+              @SuppressWarnings({ "unchecked", "rawtypes" })
+              StreamCodecWrapperForPersistance<Object> codec = new StreamCodecWrapperForPersistance(inputStreamCodecs, specifiedCodecForLogger);
               streamMetaToCodecMap.put(s, codec);
             }
           }


[50/50] incubator-apex-core git commit: Preparing for 3.4.0 development Closing PRs for switch to master branch closes #208, closes #207, closes #206, closes #194, closes #186, closes #173, closes #168, closes #163, closes #160, closes #153, closes #129,

Posted by th...@apache.org.
Preparing for 3.4.0 development
Closing PRs for switch to master branch
closes #208, closes #207, closes #206, closes #194, closes #186, closes #173, closes #168, closes #163, closes #160, closes #153, closes #129, closes #98, closes #97, closes #76


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

Branch: refs/heads/master
Commit: 9ffbc73e7fa877bddbb0b3849b87c9149a1b00a2
Parents: f32fa1b
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Jan 28 23:03:58 2016 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Jan 28 23:13:16 2016 -0800

----------------------------------------------------------------------
 apex-app-archetype/README.md  | 2 +-
 apex-app-archetype/pom.xml    | 2 +-
 apex-conf-archetype/README.md | 2 +-
 apex-conf-archetype/pom.xml   | 2 +-
 api/pom.xml                   | 2 +-
 bufferserver/pom.xml          | 2 +-
 common/pom.xml                | 2 +-
 engine/pom.xml                | 2 +-
 pom.xml                       | 2 +-
 9 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9ffbc73e/apex-app-archetype/README.md
----------------------------------------------------------------------
diff --git a/apex-app-archetype/README.md b/apex-app-archetype/README.md
index 9c386df..3bd39fe 100644
--- a/apex-app-archetype/README.md
+++ b/apex-app-archetype/README.md
@@ -6,7 +6,7 @@ How to Generate an Apex Application Project Template
 
 Run the following command
 
-    mvn archetype:generate -DarchetypeGroupId=org.apache.apex -DarchetypeArtifactId=apex-app-archetype -DarchetypeVersion=3.3.0-incubating-SNAPSHOT -DgroupId=com.example -Dpackage=com.example.myapexapp -DartifactId=myapexapp -Dversion=1.0-SNAPSHOT
+    mvn archetype:generate -DarchetypeGroupId=org.apache.apex -DarchetypeArtifactId=apex-app-archetype -DarchetypeVersion=3.4.0-incubating-SNAPSHOT -DgroupId=com.example -Dpackage=com.example.myapexapp -DartifactId=myapexapp -Dversion=1.0-SNAPSHOT
 
 Using your favorite IDE, open the project that has just been created by the above command.
 Write your application code and optionally operator code 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9ffbc73e/apex-app-archetype/pom.xml
----------------------------------------------------------------------
diff --git a/apex-app-archetype/pom.xml b/apex-app-archetype/pom.xml
index b70cfeb..8e4bf7c 100644
--- a/apex-app-archetype/pom.xml
+++ b/apex-app-archetype/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.3.0-incubating-SNAPSHOT</version>
+    <version>3.4.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>apex-app-archetype</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9ffbc73e/apex-conf-archetype/README.md
----------------------------------------------------------------------
diff --git a/apex-conf-archetype/README.md b/apex-conf-archetype/README.md
index e80d1a2..9cc2359 100644
--- a/apex-conf-archetype/README.md
+++ b/apex-conf-archetype/README.md
@@ -6,7 +6,7 @@ How to Generate a Apex App Configuration Project Template
 
 Run the following command
 
-    mvn archetype:generate -DarchetypeGroupId=org.apache.apex -DarchetypeArtifactId=apex-conf-archetype -DarchetypeVersion=3.3.0-incubating-SNAPSHOT -DgroupId=com.example -Dpackage=com.example.myapexapp -DartifactId=myapexconf -Dversion=1.0-SNAPSHOT
+    mvn archetype:generate -DarchetypeGroupId=org.apache.apex -DarchetypeArtifactId=apex-conf-archetype -DarchetypeVersion=3.4.0-incubating-SNAPSHOT -DgroupId=com.example -Dpackage=com.example.myapexapp -DartifactId=myapexconf -Dversion=1.0-SNAPSHOT
 
 Using your favorite IDE, open the project that has just been created by the above command.
 Write your application code and optionally operator code 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9ffbc73e/apex-conf-archetype/pom.xml
----------------------------------------------------------------------
diff --git a/apex-conf-archetype/pom.xml b/apex-conf-archetype/pom.xml
index 2e6f78b..eb86e08 100644
--- a/apex-conf-archetype/pom.xml
+++ b/apex-conf-archetype/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.3.0-incubating-SNAPSHOT</version>
+    <version>3.4.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>apex-conf-archetype</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9ffbc73e/api/pom.xml
----------------------------------------------------------------------
diff --git a/api/pom.xml b/api/pom.xml
index d5ea932..00496c7 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.3.0-incubating-SNAPSHOT</version>
+    <version>3.4.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>apex-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9ffbc73e/bufferserver/pom.xml
----------------------------------------------------------------------
diff --git a/bufferserver/pom.xml b/bufferserver/pom.xml
index aa79c0e..d6026dc 100644
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.3.0-incubating-SNAPSHOT</version>
+    <version>3.4.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>apex-bufferserver</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9ffbc73e/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index a749cea..b02ad6f 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.3.0-incubating-SNAPSHOT</version>
+    <version>3.4.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>apex-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9ffbc73e/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 2b70aa6..ab9ed37 100644
--- a/engine/pom.xml
+++ b/engine/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.3.0-incubating-SNAPSHOT</version>
+    <version>3.4.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>apex-engine</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9ffbc73e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 46c9b9c..c5432ae 100644
--- a/pom.xml
+++ b/pom.xml
@@ -30,7 +30,7 @@
 
   <groupId>org.apache.apex</groupId>
   <artifactId>apex</artifactId>
-  <version>3.3.0-incubating-SNAPSHOT</version>
+  <version>3.4.0-incubating-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <name>Apache Apex (incubating)</name>


[03/50] incubator-apex-core git commit: Merge branch 'APEX-273' of https://github.com/vrozov/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'APEX-273' of https://github.com/vrozov/incubator-apex-core into devel-3


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

Branch: refs/heads/master
Commit: c3369486607a74299c4340bcd84b89c6b147e354
Parents: 38a6f4e 799df6c
Author: Thomas Weise <th...@datatorrent.com>
Authored: Tue Dec 1 20:42:45 2015 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Tue Dec 1 20:42:45 2015 -0800

----------------------------------------------------------------------
 bufferserver/pom.xml                            |  3 +-
 .../bufferserver/auth/AuthManager.java          |  2 +-
 .../bufferserver/client/AuthClient.java         |  6 +-
 .../bufferserver/client/Controller.java         |  1 -
 .../bufferserver/client/Subscriber.java         | 15 +---
 .../bufferserver/internal/DataList.java         | 90 ++++++++++++--------
 .../bufferserver/internal/DataListener.java     | 10 +--
 .../bufferserver/internal/FastDataList.java     | 12 +--
 .../bufferserver/internal/LogicalNode.java      | 29 ++++---
 .../bufferserver/internal/PhysicalNode.java     |  3 +-
 .../bufferserver/packet/DataTuple.java          |  8 +-
 .../bufferserver/packet/EmptyTuple.java         | 10 +--
 .../packet/GenericRequestTuple.java             | 20 ++---
 .../bufferserver/packet/MessageType.java        | 17 +++-
 .../bufferserver/packet/PayloadTuple.java       |  6 +-
 .../packet/PublishRequestTuple.java             |  5 +-
 .../bufferserver/packet/RequestTuple.java       |  6 +-
 .../bufferserver/packet/ResetRequestTuple.java  |  5 +-
 .../bufferserver/packet/ResetWindowTuple.java   |  6 +-
 .../packet/SubscribeRequestTuple.java           | 59 +++++--------
 .../datatorrent/bufferserver/packet/Tuple.java  | 15 ++--
 .../bufferserver/packet/WindowIdTuple.java      |  8 +-
 .../bufferserver/policy/AbstractPolicy.java     | 12 +--
 .../bufferserver/policy/GiveAll.java            |  2 +-
 .../bufferserver/policy/LeastBusy.java          | 14 +--
 .../datatorrent/bufferserver/policy/Policy.java | 10 +--
 .../bufferserver/policy/RandomOne.java          | 12 +--
 .../bufferserver/policy/RoundRobin.java         | 18 ++--
 .../datatorrent/bufferserver/server/Server.java | 44 ++++++----
 .../bufferserver/storage/DiskStorage.java       | 73 ++++++----------
 .../bufferserver/util/SerializedData.java       |  4 +-
 .../datatorrent/bufferserver/util/System.java   | 12 ++-
 .../datatorrent/bufferserver/util/VarInt.java   |  9 +-
 .../bufferserver/client/SubscriberTest.java     | 39 ++++-----
 .../bufferserver/packet/NoMessageTupleTest.java | 23 +----
 .../packet/PublishRequestTupleTest.java         | 10 +--
 .../packet/ResetWindowTupleTest.java            | 10 +--
 .../packet/SubscribeRequestTupleTest.java       | 20 +++--
 .../bufferserver/server/ServerTest.java         | 67 ++++++++-------
 .../bufferserver/storage/DiskStorageTest.java   | 11 +--
 .../bufferserver/support/Subscriber.java        |  9 +-
 .../bufferserver/util/CodecTest.java            |  4 +-
 42 files changed, 346 insertions(+), 393 deletions(-)
----------------------------------------------------------------------



[17/50] incubator-apex-core git commit: fixed readme

Posted by th...@apache.org.
fixed readme


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

Branch: refs/heads/master
Commit: 587912450b49d59ce8850063b6b56a385d567854
Parents: ecb96ba
Author: Gaurav <ga...@datatorrent.com>
Authored: Wed Dec 30 11:15:24 2015 -0800
Committer: Gaurav <ga...@datatorrent.com>
Committed: Wed Dec 30 11:15:24 2015 -0800

----------------------------------------------------------------------
 README.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/58791245/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 0b9136f..65bd2f7 100644
--- a/README.md
+++ b/README.md
@@ -39,7 +39,7 @@ https://www.datatorrent.com/download/
 [Apex JIRA](https://issues.apache.org/jira/browse/APEXCORE) issue tracking system is used for this project.
 You can submit new issues and track the progress of existing issues at https://malhar.atlassian.net/projects/APEX.
 
-When working with JIRA to submit pull requests, please use [smart commits](https://confluence.atlassian.com/display/AOD/Processing+JIRA+issues+with+commit+messages) feature by specifying APEX-XXXX in the commit messages.
+When working with JIRA to submit pull requests, please use [smart commits](https://confluence.atlassian.com/display/AOD/Processing+JIRA+issues+with+commit+messages) feature by specifying APEXCORE-XXXX in the commit messages.
 It helps us link commits with issues being tracked for easy reference.  And example commit might look like this:
 
     git commit -am "APEXCORE-1234 #comment Task completed ahead of schedule #resolve"


[35/50] incubator-apex-core git commit: Merge branch 'APEXCORE-268' of https://github.com/chandnisingh/incubator-apex-core into APEXCORE-268

Posted by th...@apache.org.
Merge branch 'APEXCORE-268' of https://github.com/chandnisingh/incubator-apex-core into APEXCORE-268


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

Branch: refs/heads/master
Commit: aded30c45cc7bd2dba08243c5d0536fe9fe13bb1
Parents: b14583c 778436f
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Sun Jan 10 09:52:05 2016 -0800
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Sun Jan 10 09:52:05 2016 -0800

----------------------------------------------------------------------
 common/pom.xml                                  |  2 +-
 .../common/codec/JsonStreamCodec.java           | 13 +++--
 .../common/experimental/AppData.java            | 14 ++++--
 .../common/metric/MetricsAggregator.java        | 12 ++---
 .../common/metric/SingleMetricAggregator.java   |  1 +
 .../partitioner/StatelessPartitioner.java       | 51 +++++++++-----------
 .../auth/callback/DefaultCallbackHandler.java   | 10 ++--
 .../common/util/AsyncFSStorageAgent.java        | 20 ++++++--
 .../datatorrent/common/util/BaseOperator.java   |  2 -
 .../util/BasicContainerOptConfigurator.java     | 22 ++++-----
 .../datatorrent/common/util/FSStorageAgent.java | 42 ++++++++--------
 .../util/JacksonObjectMapperProvider.java       |  6 ++-
 .../common/util/NameableThreadFactory.java      |  4 +-
 .../datatorrent/common/util/PubSubMessage.java  |  3 +-
 .../common/util/PubSubMessageCodec.java         |  9 ++--
 .../common/util/PubSubWebSocketClient.java      | 50 ++++++++++---------
 .../common/util/SerializableObject.java         | 41 ++++------------
 .../common/codec/JsonStreamCodecTest.java       |  4 +-
 .../partitioner/StatelessPartitionerTest.java   | 14 ++++--
 .../common/util/AsyncFSStorageAgentTest.java    | 13 ++---
 .../common/util/FSStorageAgentTest.java         | 19 ++++----
 .../common/util/SerializableObjectTest.java     |  5 +-
 22 files changed, 180 insertions(+), 177 deletions(-)
----------------------------------------------------------------------



[10/50] incubator-apex-core git commit: APEX-103 - Add module and dag interface in API

Posted by th...@apache.org.
APEX-103 - Add module and dag interface in API


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

Branch: refs/heads/master
Commit: 2f1e1dfeb400bd7f01275a95522766abe909d936
Parents: 0d5bfa5
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Wed Sep 23 20:30:51 2015 -0700
Committer: Tushar R. Gosavi <tu...@apache.org>
Committed: Mon Dec 21 23:41:04 2015 +0530

----------------------------------------------------------------------
 api/src/main/java/com/datatorrent/api/DAG.java  | 24 +++++++++++++++++
 .../main/java/com/datatorrent/api/Module.java   | 28 ++++++++++++++++++++
 .../stram/plan/logical/LogicalPlan.java         | 24 +++++++++++++++++
 3 files changed, 76 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/2f1e1dfe/api/src/main/java/com/datatorrent/api/DAG.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/DAG.java b/api/src/main/java/com/datatorrent/api/DAG.java
index 6b1d1b2..abe2954 100644
--- a/api/src/main/java/com/datatorrent/api/DAG.java
+++ b/api/src/main/java/com/datatorrent/api/DAG.java
@@ -20,6 +20,8 @@ package com.datatorrent.api;
 
 import java.io.Serializable;
 
+import org.apache.hadoop.classification.InterfaceStability;
+
 import com.datatorrent.api.Context.DAGContext;
 
 /**
@@ -157,6 +159,16 @@ public interface DAG extends DAGContext, Serializable
     public OutputPortMeta getMeta(Operator.OutputPort<?> port);
   }
 
+  @InterfaceStability.Evolving
+  interface ModuleMeta extends Serializable, Context
+  {
+    String getName();
+
+    InputPortMeta getMeta(Operator.InputPort<?> port);
+
+    OutputPortMeta getMeta(Operator.OutputPort<?> port);
+  }
+
   /**
    * Add new instance of operator under given name to the DAG.
    * The operator class must have a default constructor.
@@ -179,6 +191,12 @@ public interface DAG extends DAGContext, Serializable
    */
   public abstract <T extends Operator> T addOperator(String name, T operator);
 
+  @InterfaceStability.Evolving
+  <T extends Module> T addModule(String name, Class<T> moduleClass);
+
+  @InterfaceStability.Evolving
+  <T extends Module> T addModule(String name, T module);
+
   /**
    * <p>addStream.</p>
    * @param id Identifier of the stream that will be used to identify stream in DAG
@@ -256,9 +274,15 @@ public interface DAG extends DAGContext, Serializable
    */
   public abstract OperatorMeta getOperatorMeta(String operatorId);
 
+  @InterfaceStability.Evolving
+  ModuleMeta getModuleMeta(String moduleId);
+
   /**
    * <p>getMeta.</p>
    */
   public abstract OperatorMeta getMeta(Operator operator);
 
+  @InterfaceStability.Evolving
+  ModuleMeta getMeta(Module module);
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/2f1e1dfe/api/src/main/java/com/datatorrent/api/Module.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Module.java b/api/src/main/java/com/datatorrent/api/Module.java
new file mode 100644
index 0000000..1220fc1
--- /dev/null
+++ b/api/src/main/java/com/datatorrent/api/Module.java
@@ -0,0 +1,28 @@
+/**
+ * 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 com.datatorrent.api;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+@InterfaceStability.Evolving
+public interface Module
+{
+  void populateDAG(DAG dag, Configuration conf);
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/2f1e1dfe/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index 6405644..cca45d8 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -1079,6 +1079,18 @@ public class LogicalPlan implements Serializable, DAG
     return operator;
   }
 
+  @Override
+  public <T extends Module> T addModule(String name, Class<T> moduleClass)
+  {
+    throw new UnsupportedOperationException("Modules are not supported");
+  }
+
+  @Override
+  public <T extends Module> T addModule(String name, T module)
+  {
+    throw new UnsupportedOperationException("Modules are not supported");
+  }
+
   public void removeOperator(Operator operator)
   {
     OperatorMeta om = getMeta(operator);
@@ -1231,6 +1243,12 @@ public class LogicalPlan implements Serializable, DAG
   }
 
   @Override
+  public ModuleMeta getModuleMeta(String moduleName)
+  {
+    throw new UnsupportedOperationException("Modules are not supported");
+  }
+
+  @Override
   public OperatorMeta getMeta(Operator operator)
   {
     // TODO: cache mapping
@@ -1242,6 +1260,12 @@ public class LogicalPlan implements Serializable, DAG
     throw new IllegalArgumentException("Operator not associated with the DAG: " + operator);
   }
 
+  @Override
+  public ModuleMeta getMeta(Module module)
+  {
+    throw new UnsupportedOperationException("Modules are not supported");
+  }
+
   public int getMaxContainerCount()
   {
     return this.getValue(CONTAINERS_MAX_COUNT);


[09/50] incubator-apex-core git commit: APEXCORE-283 #comment added storage agent interface and stram client changes to retrieve application attributes

Posted by th...@apache.org.
APEXCORE-283 #comment added storage agent interface and stram client
changes to retrieve application attributes


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

Branch: refs/heads/master
Commit: 02c43eea28d27fd71fe3f551dfaceb0c3c931db8
Parents: 0d5bfa5
Author: Ashish Tadose <as...@gmail..com>
Authored: Fri Dec 18 01:23:46 2015 +0530
Committer: Ashish Tadose <as...@gmail..com>
Committed: Fri Dec 18 02:25:12 2015 +0530

----------------------------------------------------------------------
 .../java/com/datatorrent/api/StorageAgent.java  | 18 ++++++++++
 .../java/com/datatorrent/stram/StramClient.java | 38 ++++++++++++++++----
 2 files changed, 50 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/02c43eea/api/src/main/java/com/datatorrent/api/StorageAgent.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/StorageAgent.java b/api/src/main/java/com/datatorrent/api/StorageAgent.java
index e155ed3..b5dcf39 100644
--- a/api/src/main/java/com/datatorrent/api/StorageAgent.java
+++ b/api/src/main/java/com/datatorrent/api/StorageAgent.java
@@ -20,6 +20,8 @@ package com.datatorrent.api;
 
 import java.io.IOException;
 
+import com.datatorrent.api.Attribute.AttributeMap;
+
 /**
  * Interface to define writing/reading checkpoint state of any operator.
  *
@@ -77,4 +79,20 @@ public interface StorageAgent
    */
   public long[] getWindowIds(int operatorId) throws IOException;
 
+  /**
+   * Interface to pass application attributes to storage agent
+   * 
+   *
+   */
+  public interface ApplicationAwareStorageAgent extends StorageAgent
+  {
+   
+    /**
+     * Passes attributes of application to storage agent
+     * 
+     * @param map attributes of application
+     */
+    public void setApplicationAttributes(AttributeMap map);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/02c43eea/engine/src/main/java/com/datatorrent/stram/StramClient.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/StramClient.java b/engine/src/main/java/com/datatorrent/stram/StramClient.java
index 9a570e0..046a56c 100644
--- a/engine/src/main/java/com/datatorrent/stram/StramClient.java
+++ b/engine/src/main/java/com/datatorrent/stram/StramClient.java
@@ -24,10 +24,12 @@ import java.io.OutputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.base.Objects;
-import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,14 +40,29 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -54,7 +71,11 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.log4j.DTLoggerFactory;
 
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+
 import com.datatorrent.api.Context.OperatorContext;
+import com.datatorrent.api.StorageAgent;
 import com.datatorrent.common.util.AsyncFSStorageAgent;
 import com.datatorrent.common.util.BasicContainerOptConfigurator;
 import com.datatorrent.stram.client.StramClientUtils;
@@ -455,6 +476,11 @@ public class StramClient
       }
 
       dag.getAttributes().put(LogicalPlan.APPLICATION_PATH, appPath.toString());
+      StorageAgent agent = dag.getAttributes().get(OperatorContext.STORAGE_AGENT);
+      if (agent != null && agent instanceof StorageAgent.ApplicationAwareStorageAgent) {
+        ((StorageAgent.ApplicationAwareStorageAgent)agent).setApplicationAttributes(dag.getAttributes());
+      }
+      
       if (dag.getAttributes().get(OperatorContext.STORAGE_AGENT) == null) { /* which would be the most likely case */
         Path checkpointPath = new Path(appPath, LogicalPlan.SUBDIR_CHECKPOINTS);
         // use conf client side to pickup any proxy settings from dt-site.xml


[29/50] incubator-apex-core git commit: APEXCORE-302 Update NOTICE copyright year

Posted by th...@apache.org.
APEXCORE-302 Update NOTICE copyright year


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

Branch: refs/heads/master
Commit: f6e2930d4368d5abf63fdd96f03ae6e477a9e865
Parents: 7629be8
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Fri Jan 8 10:45:55 2016 -0800
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Fri Jan 8 10:45:55 2016 -0800

----------------------------------------------------------------------
 NOTICE  | 2 +-
 pom.xml | 1 +
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f6e2930d/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index d0f6855..ca47411 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,5 @@
 Apache Apex (incubating)
-Copyright (c) 2015 The Apache Software Foundation
+Copyright (c) 2015-2016 The Apache Software Foundation
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f6e2930d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 89c60c9..1972d37 100644
--- a/pom.xml
+++ b/pom.xml
@@ -35,6 +35,7 @@
 
   <name>Apache Apex (incubating)</name>
   <url>http://apex.incubator.apache.org</url>
+  <inceptionYear>2015</inceptionYear>
 
   <organization>
     <name>Apache Software Foundation</name>


[23/50] incubator-apex-core git commit: Adjust violation count (due to checkstyle plugin update?)

Posted by th...@apache.org.
Adjust violation count (due to checkstyle plugin update?)


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

Branch: refs/heads/master
Commit: 2da1000656b23f0ada2eb7f5a64a3f7cb76c7255
Parents: 4b2c884
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Dec 31 15:14:28 2015 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Dec 31 15:14:28 2015 -0800

----------------------------------------------------------------------
 engine/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/2da10006/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 5c59dce..f345410 100644
--- a/engine/pom.xml
+++ b/engine/pom.xml
@@ -145,7 +145,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>4402</maxAllowedViolations>
+          <maxAllowedViolations>4403</maxAllowedViolations>
         </configuration>
       </plugin>
       <plugin>


[28/50] incubator-apex-core git commit: APEX-78 #comment Checkpoint notification to notify operators before checkpoint is performed

Posted by th...@apache.org.
APEX-78 #comment Checkpoint notification to notify operators before checkpoint is performed


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

Branch: refs/heads/master
Commit: d5e82468cfd27cdfd09135f8e6956147ac5d9dc3
Parents: 7629be8
Author: Pramod Immaneni <pr...@datatorrent.com>
Authored: Fri Dec 11 06:03:29 2015 -0800
Committer: Pramod Immaneni <pr...@datatorrent.com>
Committed: Thu Jan 7 22:18:24 2016 -0800

----------------------------------------------------------------------
 .../main/java/com/datatorrent/api/Operator.java | 26 ++++++
 .../java/com/datatorrent/stram/engine/Node.java |  4 +
 .../com/datatorrent/stram/CheckpointTest.java   | 85 +++++++++++++++++---
 3 files changed, 104 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d5e82468/api/src/main/java/com/datatorrent/api/Operator.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Operator.java b/api/src/main/java/com/datatorrent/api/Operator.java
index eb69266..785c60b 100644
--- a/api/src/main/java/com/datatorrent/api/Operator.java
+++ b/api/src/main/java/com/datatorrent/api/Operator.java
@@ -224,6 +224,7 @@ public interface Operator extends Component<OperatorContext>
    * Operators must implement this interface if they are interested in being notified as
    * soon as the operator state is checkpointed or committed.
    *
+   * @deprecated Use {@link CheckpointNotificationListener} instead
    * @since 0.3.2
    */
   public static interface CheckpointListener
@@ -270,4 +271,29 @@ public interface Operator extends Component<OperatorContext>
 
   }
 
+  /**
+   * Operators that need to be notified about checkpoint events should implement this interface.
+   *
+   * The notification callbacks in this interface are called outside window boundaries so the operators should not
+   * attempt to send any tuples in these callbacks.
+   *
+   */
+  interface CheckpointNotificationListener extends CheckpointListener
+  {
+    /**
+     * Notify the operator before a checkpoint is performed.
+     *
+     * Operators may need to perform certain tasks before a checkpoint such as calling flush on a stream to write out
+     * pending data. Having this notification helps operators perform such operations optimally by doing them once
+     * before checkpoint as opposed to doing them at the end of every window.
+     *
+     * The method will be called before the checkpoint is performed. It will be called after
+     * {@link Operator#endWindow()} call of the window preceding the checkpoint and before the checkpoint is
+     * actually performed.
+     *
+     * @param windowId The window id of the window preceding the checkpoint
+     */
+    void beforeCheckpoint(long windowId);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d5e82468/engine/src/main/java/com/datatorrent/stram/engine/Node.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/engine/Node.java b/engine/src/main/java/com/datatorrent/stram/engine/Node.java
index 5c6b86f..068a325 100644
--- a/engine/src/main/java/com/datatorrent/stram/engine/Node.java
+++ b/engine/src/main/java/com/datatorrent/stram/engine/Node.java
@@ -486,6 +486,10 @@ public abstract class Node<OPERATOR extends Operator> implements Component<Opera
   void checkpoint(long windowId)
   {
     if (!context.stateless) {
+      if (operator instanceof Operator.CheckpointNotificationListener) {
+        ((Operator.CheckpointNotificationListener)operator).beforeCheckpoint(windowId);
+      }
+
       StorageAgent ba = context.getValue(OperatorContext.STORAGE_AGENT);
       if (ba != null) {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d5e82468/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java b/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java
index 5d11b86..ee3cbc3 100644
--- a/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/CheckpointTest.java
@@ -18,26 +18,37 @@
  */
 package com.datatorrent.stram;
 
-import com.datatorrent.common.util.BaseOperator;
-
-import java.util.*;
-
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-import org.junit.*;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 
-import com.datatorrent.api.*;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import com.datatorrent.api.DAG;
 import com.datatorrent.api.DAG.Locality;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.Operator;
 import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
 import com.datatorrent.api.annotation.Stateless;
-
 import com.datatorrent.common.util.AsyncFSStorageAgent;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.common.util.FSStorageAgent;
 import com.datatorrent.stram.MockContainer.MockOperatorStats;
 import com.datatorrent.stram.StreamingContainerManager.UpdateCheckpointsContext;
 import com.datatorrent.stram.api.Checkpoint;
@@ -63,12 +74,14 @@ public class CheckpointTest
   @Rule
   public TestMeta testMeta = new TestMeta();
 
-  private static class MockInputOperator extends BaseOperator implements InputOperator
+  private static class MockInputOperator extends BaseOperator implements InputOperator, Operator.CheckpointNotificationListener
   {
     @OutputPortFieldAnnotation( optional = true)
     public final transient DefaultOutputPort<Object> outport = new DefaultOutputPort<Object>();
     private transient int windowCount;
 
+    private int checkpointState;
+
     @Override
     public void beginWindow(long windowId)
     {
@@ -81,6 +94,22 @@ public class CheckpointTest
     public void emitTuples()
     {
     }
+
+    @Override
+    public void beforeCheckpoint(long windowId)
+    {
+      ++checkpointState;
+    }
+
+    @Override
+    public void checkpointed(long windowId)
+    {
+    }
+
+    @Override
+    public void committed(long windowId)
+    {
+    }
   }
 
   private LogicalPlan dag;
@@ -454,5 +483,39 @@ public class CheckpointTest
 
   }
 
+  @Test
+  public void testBeforeCheckpointNotification() throws IOException, ClassNotFoundException
+  {
+    FSStorageAgent storageAgent = new FSStorageAgent(testMeta.getPath(), null);
+    dag.setAttribute(OperatorContext.STORAGE_AGENT, storageAgent);
+    dag.setAttribute(LogicalPlan.CHECKPOINT_WINDOW_COUNT, 1);
+    dag.setAttribute(LogicalPlan.STREAMING_WINDOW_SIZE_MILLIS, 50);
+
+    MockInputOperator o1 = dag.addOperator("o1", new MockInputOperator());
+
+    GenericTestOperator o2 = dag.addOperator("o2", GenericTestOperator.class);
+    dag.setAttribute(o2, OperatorContext.STATELESS, true);
+
+    dag.addStream("o1.outport", o1.outport, o2.inport1);
+
+    StramLocalCluster sc = new StramLocalCluster(dag);
+    sc.setHeartbeatMonitoringEnabled(false);
+    sc.run();
+
+    StreamingContainerManager dnm = sc.dnmgr;
+    PhysicalPlan plan = dnm.getPhysicalPlan();
+    List<PTOperator> o1ps = plan.getOperators(dag.getMeta(o1));
+    Assert.assertEquals("Number partitions", 1, o1ps.size());
+
+    PTOperator o1p1 = o1ps.get(0);
+    long[] ckWIds = storageAgent.getWindowIds(o1p1.getId());
+    Arrays.sort(ckWIds);
+    int expectedState = 0;
+    for (long windowId : ckWIds) {
+      Object ckState = storageAgent.load(o1p1.getId(), windowId);
+      Assert.assertEquals("Checkpointed state class", MockInputOperator.class, ckState.getClass());
+      Assert.assertEquals("Checkpoint state", expectedState++, ((MockInputOperator)ckState).checkpointState);
+    }
+  }
 
 }


[05/50] incubator-apex-core git commit: APEX-278 #resolve

Posted by th...@apache.org.
APEX-278 #resolve


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

Branch: refs/heads/master
Commit: 60aa9bf290fb4fe78f2bd2aa5fe0d53f9e6a9862
Parents: 994f868
Author: Chandni Singh <cs...@apache.org>
Authored: Wed Dec 2 12:24:24 2015 -0800
Committer: Chandni Singh <cs...@apache.org>
Committed: Wed Dec 2 12:24:24 2015 -0800

----------------------------------------------------------------------
 .../src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/60aa9bf2/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java b/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java
index b8bff8a..c7e8ccc 100644
--- a/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java
@@ -108,7 +108,6 @@ public class GenericNodeTest
     @Override
     public void save(final Object object, final int operatorId, final long windowId) throws IOException
     {
-      LOG.info("Saving");
       //Do nothing
     }
 


[25/50] incubator-apex-core git commit: Merge branch 'APEXCORE-288' of https://github.com/davidyan74/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'APEXCORE-288' of https://github.com/davidyan74/incubator-apex-core into devel-3


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

Branch: refs/heads/master
Commit: 3f3afeaa102dfe1603e0d788ed9feaa956a71915
Parents: fadb6f4 c578075
Author: siyuan <hs...@gmail.com>
Authored: Tue Jan 5 15:17:57 2016 -0800
Committer: siyuan <hs...@gmail.com>
Committed: Tue Jan 5 15:17:57 2016 -0800

----------------------------------------------------------------------
 .../src/main/resources/archetype-resources/pom.xml        |  1 +
 .../src/main/resources/archetype-resources/pom.xml        |  2 ++
 engine/src/main/java/com/datatorrent/stram/cli/DTCli.java |  5 +++++
 .../java/com/datatorrent/stram/client/AppPackage.java     | 10 +++++++++-
 .../java/com/datatorrent/stram/client/ConfigPackage.java  |  8 ++++++++
 .../java/com/datatorrent/stram/client/AppPackageTest.java |  1 +
 .../com/datatorrent/stram/client/ConfigPackageTest.java   |  1 +
 engine/src/test/resources/testAppPackage/mydtapp/pom.xml  |  1 +
 .../testConfigPackageSrc/META-INF/MANIFEST.MF             |  1 +
 9 files changed, 29 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/3f3afeaa/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java
----------------------------------------------------------------------


[15/50] incubator-apex-core git commit: Merge branch 'APEXCORE-283.checkpoint' of https://github.com/ashishtadose/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'APEXCORE-283.checkpoint' of https://github.com/ashishtadose/incubator-apex-core into devel-3


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

Branch: refs/heads/master
Commit: 56b55fe1da241f3131b3fe1396401b06414d4ffb
Parents: 47b3ce8 02c43ee
Author: Thomas Weise <th...@datatorrent.com>
Authored: Tue Dec 22 10:25:54 2015 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Tue Dec 22 10:25:54 2015 -0800

----------------------------------------------------------------------
 .../java/com/datatorrent/api/StorageAgent.java  | 18 ++++++++++
 .../java/com/datatorrent/stram/StramClient.java | 38 ++++++++++++++++----
 2 files changed, 50 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[06/50] incubator-apex-core git commit: Merge branch 'APEX-278' into devel-3

Posted by th...@apache.org.
Merge branch 'APEX-278' into devel-3


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

Branch: refs/heads/master
Commit: 0b0f914cdf4ba77e435164561c077a215cd1c4cf
Parents: 994f868 60aa9bf
Author: Timothy Farkas <ti...@ge.com>
Authored: Wed Dec 2 13:07:32 2015 -0800
Committer: Timothy Farkas <ti...@ge.com>
Committed: Wed Dec 2 13:07:32 2015 -0800

----------------------------------------------------------------------
 .../src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------



[13/50] incubator-apex-core git commit: APEXCORE-194 Added support for proxy ports Added test cases.

Posted by th...@apache.org.
APEXCORE-194 Added support for proxy ports
Added test cases.


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

Branch: refs/heads/master
Commit: c1314eafaac239b420d085a4584d5c5acaf3e69b
Parents: 14a09bb
Author: bhupeshchawda <bh...@gmail.com>
Authored: Tue Oct 6 12:34:24 2015 +0530
Committer: Tushar R. Gosavi <tu...@apache.org>
Committed: Tue Dec 22 02:04:18 2015 +0530

----------------------------------------------------------------------
 .../main/java/com/datatorrent/api/Module.java   | 120 ++++
 .../stram/plan/logical/LogicalPlan.java         |  51 +-
 .../plan/logical/LogicalPlanConfiguration.java  |   1 +
 .../plan/logical/module/ModuleAppTest.java      | 168 ++++++
 .../logical/module/TestModuleExpansion.java     | 552 +++++++++++++++++++
 5 files changed, 888 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c1314eaf/api/src/main/java/com/datatorrent/api/Module.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Module.java b/api/src/main/java/com/datatorrent/api/Module.java
index 1220fc1..67682e7 100644
--- a/api/src/main/java/com/datatorrent/api/Module.java
+++ b/api/src/main/java/com/datatorrent/api/Module.java
@@ -21,8 +21,128 @@ package com.datatorrent.api;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 
+import com.datatorrent.api.Context.PortContext;
+import com.datatorrent.api.Operator.InputPort;
+import com.datatorrent.api.Operator.OutputPort;
+import com.datatorrent.api.Operator.Unifier;
+
+/**
+ * A Module is a component which can be added to the DAG similar to the operator,
+ * using addModule API. The module should implement populateDAG method, which
+ * will be called by the platform, and DAG populated by the module will be
+ * replace in place of the module.
+ *
+ */
 @InterfaceStability.Evolving
 public interface Module
 {
   void populateDAG(DAG dag, Configuration conf);
+
+  /**
+   * These ports allow platform to short circuit module port to the operator port. i.e When a module is expanded, it can
+   * specify  which operator's port is used to replaced the module port in the final DAG.
+   *
+   * @param <T> data type accepted at the input port.
+   */
+  final class ProxyInputPort<T> implements InputPort<T>
+  {
+    InputPort<T> inputPort;
+
+    public void set(InputPort<T> port)
+    {
+      inputPort = port;
+    }
+
+    public InputPort<T> get()
+    {
+      return inputPort;
+    }
+
+    @Override
+    public void setup(PortContext context)
+    {
+      if (inputPort != null) {
+        inputPort.setup(context);
+      }
+    }
+
+    @Override
+    public void teardown()
+    {
+      if (inputPort != null) {
+        inputPort.teardown();
+      }
+    }
+
+    @Override
+    public Sink<T> getSink()
+    {
+      return inputPort == null ? null : inputPort.getSink();
+    }
+
+    @Override
+    public void setConnected(boolean connected)
+    {
+      if (inputPort != null) {
+        inputPort.setConnected(connected);
+      }
+    }
+
+    @Override
+    public StreamCodec<T> getStreamCodec()
+    {
+      return inputPort == null ? null : inputPort.getStreamCodec();
+    }
+  }
+
+  /**
+   * Similar to ProxyInputPort, but on output side.
+   *
+   * @param <T> datatype emitted on the port.
+   */
+  final class ProxyOutputPort<T> implements OutputPort<T>
+  {
+    OutputPort<T> outputPort;
+
+    public void set(OutputPort<T> port)
+    {
+      outputPort = port;
+    }
+
+    public OutputPort<T> get()
+    {
+      return outputPort;
+    }
+
+    @Override
+    public void setup(PortContext context)
+    {
+      if (outputPort != null) {
+        outputPort.setup(context);
+      }
+    }
+
+    @Override
+    public void teardown()
+    {
+      if (outputPort != null) {
+        outputPort.teardown();
+      }
+    }
+
+    @Override
+    public void setSink(Sink<Object> s)
+    {
+      if (outputPort != null) {
+        outputPort.setSink(s);
+      }
+    }
+
+    @Override
+    public Unifier<T> getUnifier()
+    {
+      return outputPort == null ? null : outputPort.getUnifier();
+    }
+  }
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c1314eaf/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index 5a3e167..21039cc 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -37,12 +37,14 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Maps;
+import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Sets;
 
 import com.datatorrent.api.*;
 import com.datatorrent.api.Attribute.AttributeMap;
 import com.datatorrent.api.Attribute.AttributeMap.DefaultAttributeMap;
+import com.datatorrent.api.Module.ProxyInputPort;
+import com.datatorrent.api.Module.ProxyOutputPort;
 import com.datatorrent.api.Operator.InputPort;
 import com.datatorrent.api.Operator.OutputPort;
 import com.datatorrent.api.Operator.Unifier;
@@ -152,6 +154,7 @@ public class LogicalPlan implements Serializable, DAG
   private final Attribute.AttributeMap attributes = new DefaultAttributeMap();
   private transient int nodeIndex = 0; // used for cycle validation
   private transient Stack<OperatorMeta> stack = new Stack<OperatorMeta>(); // used for cycle validation
+  private transient Map<String, ArrayListMultimap<OutputPort<?>, InputPort<?>>> streamLinks = new HashMap<>();
 
   @Override
   public Attribute.AttributeMap getAttributes()
@@ -1197,6 +1200,7 @@ public class LogicalPlan implements Serializable, DAG
         subModuleMeta.setParent(this);
         subModuleMeta.flattenModule(dag, conf);
       }
+      dag.applyStreamLinks();
       parentDAG.addDAGToCurrentDAG(this);
     }
 
@@ -1300,13 +1304,52 @@ public class LogicalPlan implements Serializable, DAG
   public <T> StreamMeta addStream(String id, Operator.OutputPort<? extends T> source, Operator.InputPort<? super T>... sinks)
   {
     StreamMeta s = addStream(id);
-    s.setSource(source);
-    for (Operator.InputPort<?> sink: sinks) {
-      s.addSink(sink);
+    id = s.id;
+    ArrayListMultimap<OutputPort<?>, InputPort<?>> streamMap = ArrayListMultimap.create();
+    if (!(source instanceof ProxyOutputPort)) {
+      s.setSource(source);
+    }
+    for (Operator.InputPort<?> sink : sinks) {
+      if (source instanceof ProxyOutputPort || sink instanceof ProxyInputPort) {
+        streamMap.put(source, sink);
+        streamLinks.put(id, streamMap);
+      } else {
+        if (s.getSource() == null) {
+          s.setSource(source);
+        }
+        s.addSink(sink);
+      }
     }
     return s;
   }
 
+  /**
+   * This will be called once the Logical Dag is expanded, and the proxy input and proxy output ports are populated with
+   * the actual ports that they refer to This method adds sources and sinks for the StreamMeta objects which were left
+   * empty in the addStream call.
+   */
+  public void applyStreamLinks()
+  {
+    for (String id : streamLinks.keySet()) {
+      StreamMeta s = getStream(id);
+      for (Map.Entry<Operator.OutputPort<?>, Operator.InputPort<?>> pair : streamLinks.get(id).entries()) {
+        if (s.getSource() == null) {
+          Operator.OutputPort<?> outputPort = pair.getKey();
+          while (outputPort instanceof ProxyOutputPort) {
+            outputPort = ((ProxyOutputPort<?>)outputPort).get();
+          }
+          s.setSource(outputPort);
+        }
+
+        Operator.InputPort<?> inputPort = pair.getValue();
+        while (inputPort instanceof ProxyInputPort) {
+          inputPort = ((ProxyInputPort<?>)inputPort).get();
+        }
+        s.addSink(inputPort);
+      }
+    }
+  }
+
   @SuppressWarnings({ "unchecked", "rawtypes" })
   private void addDAGToCurrentDAG(ModuleMeta moduleMeta)
   {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c1314eaf/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
index 6dc4c0c..483576a 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
@@ -2128,6 +2128,7 @@ public class LogicalPlanConfiguration {
     for (ModuleMeta moduleMeta : dag.getAllModules()) {
       moduleMeta.flattenModule(dag, conf);
     }
+    dag.applyStreamLinks();
   }
 
   public static Properties readProperties(String filePath) throws IOException

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c1314eaf/engine/src/test/java/com/datatorrent/stram/plan/logical/module/ModuleAppTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/module/ModuleAppTest.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/ModuleAppTest.java
new file mode 100644
index 0000000..97c015e
--- /dev/null
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/ModuleAppTest.java
@@ -0,0 +1,168 @@
+/**
+ * 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 com.datatorrent.stram.plan.logical.module;
+
+import java.util.Random;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.Module;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.stram.plan.logical.LogicalPlan;
+import com.datatorrent.stram.plan.logical.LogicalPlanConfiguration;
+
+/**
+ * Unit tests for testing Dag expansion with modules and proxy port substitution
+ */
+public class ModuleAppTest
+{
+
+  /*
+   * Input Operator - 1
+   */
+  static class DummyInputOperator extends BaseOperator implements InputOperator
+  {
+
+    Random r = new Random();
+    public transient DefaultOutputPort<Integer> output = new DefaultOutputPort<Integer>();
+
+    @Override
+    public void emitTuples()
+    {
+      output.emit(r.nextInt());
+    }
+  }
+
+  /*
+   * Input Operator - 1.1
+   */
+  static class DummyOperatorAfterInput extends BaseOperator
+  {
+
+    public transient DefaultInputPort<Integer> input = new DefaultInputPort<Integer>()
+    {
+      @Override
+      public void process(Integer tuple)
+      {
+        output.emit(tuple);
+      }
+    };
+    public transient DefaultOutputPort<Integer> output = new DefaultOutputPort<Integer>();
+  }
+
+  /*
+   * Operator - 2
+   */
+  static class DummyOperator extends BaseOperator
+  {
+    int prop;
+
+    public transient DefaultInputPort<Integer> input = new DefaultInputPort<Integer>()
+    {
+      @Override
+      public void process(Integer tuple)
+      {
+        LOG.debug(tuple.intValue() + " processed");
+        output.emit(tuple);
+      }
+    };
+    public transient DefaultOutputPort<Integer> output = new DefaultOutputPort<Integer>();
+  }
+
+  /*
+   * Output Operator - 3
+   */
+  static class DummyOutputOperator extends BaseOperator
+  {
+    int prop;
+
+    public transient DefaultInputPort<Integer> input = new DefaultInputPort<Integer>()
+    {
+      @Override
+      public void process(Integer tuple)
+      {
+        LOG.debug(tuple.intValue() + " processed");
+      }
+    };
+  }
+
+  /*
+   * Module Definition
+   */
+  static class TestModule implements Module
+  {
+
+    public transient ProxyInputPort<Integer> moduleInput = new Module.ProxyInputPort<Integer>();
+    public transient ProxyOutputPort<Integer> moduleOutput = new Module.ProxyOutputPort<Integer>();
+
+    @Override
+    public void populateDAG(DAG dag, Configuration conf)
+    {
+      LOG.debug("Module - PopulateDAG");
+      DummyOperator dummyOperator = dag.addOperator("DummyOperator", new DummyOperator());
+      moduleInput.set(dummyOperator.input);
+      moduleOutput.set(dummyOperator.output);
+    }
+  }
+
+  static class Application implements StreamingApplication
+  {
+    @Override
+    public void populateDAG(DAG dag, Configuration conf)
+    {
+      LOG.debug("Application - PopulateDAG");
+      DummyInputOperator dummyInputOperator = dag.addOperator("DummyInputOperator", new DummyInputOperator());
+      DummyOperatorAfterInput dummyOperatorAfterInput = dag.addOperator("DummyOperatorAfterInput",
+          new DummyOperatorAfterInput());
+      Module m1 = dag.addModule("TestModule1", new TestModule());
+      Module m2 = dag.addModule("TestModule2", new TestModule());
+      DummyOutputOperator dummyOutputOperator = dag.addOperator("DummyOutputOperator", new DummyOutputOperator());
+      dag.addStream("Operator To Operator", dummyInputOperator.output, dummyOperatorAfterInput.input);
+      dag.addStream("Operator To Module", dummyOperatorAfterInput.output, ((TestModule)m1).moduleInput);
+      dag.addStream("Module To Module", ((TestModule)m1).moduleOutput, ((TestModule)m2).moduleInput);
+      dag.addStream("Module To Operator", ((TestModule)m2).moduleOutput, dummyOutputOperator.input);
+    }
+  }
+
+  @Test
+  public void validateTestApplication()
+  {
+    Configuration conf = new Configuration(false);
+    LogicalPlanConfiguration lpc = new LogicalPlanConfiguration(conf);
+    LogicalPlan dag = new LogicalPlan();
+    lpc.prepareDAG(dag, new Application(), "TestApp");
+
+    Assert.assertEquals(2, dag.getAllModules().size(), 2);
+    Assert.assertEquals(5, dag.getAllOperators().size());
+    Assert.assertEquals(4, dag.getAllStreams().size());
+    dag.validate();
+  }
+
+  private static Logger LOG = LoggerFactory.getLogger(ModuleAppTest.class);
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c1314eaf/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java
new file mode 100644
index 0000000..5bfd8f1
--- /dev/null
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java
@@ -0,0 +1,552 @@
+/**
+ * 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 com.datatorrent.stram.plan.logical.module;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.Module;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.InputPortFieldAnnotation;
+import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.stram.plan.logical.LogicalPlan;
+import com.datatorrent.stram.plan.logical.LogicalPlanConfiguration;
+
+public class TestModuleExpansion
+{
+  static class DummyInputOperator extends BaseOperator implements InputOperator
+  {
+    private int inputOperatorProp = 0;
+
+    Random r = new Random();
+    public transient DefaultOutputPort<Integer> out = new DefaultOutputPort<Integer>();
+
+    @Override
+    public void emitTuples()
+    {
+      out.emit(r.nextInt());
+    }
+
+    public int getInputOperatorProp()
+    {
+      return inputOperatorProp;
+    }
+
+    public void setInputOperatorProp(int inputOperatorProp)
+    {
+      this.inputOperatorProp = inputOperatorProp;
+    }
+  }
+
+  static class DummyOperator extends BaseOperator
+  {
+    private int operatorProp = 0;
+
+    @OutputPortFieldAnnotation(optional = true)
+    public final transient DefaultOutputPort<Integer> out1 = new DefaultOutputPort<>();
+
+    @OutputPortFieldAnnotation(optional = true)
+    public final transient DefaultOutputPort<Integer> out2 = new DefaultOutputPort<>();
+
+    @InputPortFieldAnnotation(optional = true)
+    public final transient DefaultInputPort<Integer> in = new DefaultInputPort<Integer>()
+    {
+      @Override
+      public void process(Integer tuple)
+      {
+        out1.emit(tuple);
+        out2.emit(tuple);
+      }
+    };
+
+    public int getOperatorProp()
+    {
+      return operatorProp;
+    }
+
+    public void setOperatorProp(int operatorProp)
+    {
+      this.operatorProp = operatorProp;
+    }
+  }
+
+  static class Level1Module implements Module
+  {
+    private int level1ModuleProp = 0;
+
+    @InputPortFieldAnnotation(optional = true)
+    public final transient ProxyInputPort<Integer> mIn = new ProxyInputPort<>();
+    @OutputPortFieldAnnotation(optional = true)
+    public final transient ProxyOutputPort<Integer> mOut = new ProxyOutputPort<>();
+
+    @Override
+    public void populateDAG(DAG dag, Configuration conf)
+    {
+      DummyOperator o1 = dag.addOperator("O1", new DummyOperator());
+      o1.setOperatorProp(level1ModuleProp);
+      mIn.set(o1.in);
+      mOut.set(o1.out1);
+    }
+
+    public int getLevel1ModuleProp()
+    {
+      return level1ModuleProp;
+    }
+
+    public void setLevel1ModuleProp(int level1ModuleProp)
+    {
+      this.level1ModuleProp = level1ModuleProp;
+    }
+  }
+
+  static class Level2ModuleA implements Module
+  {
+    private int level2ModuleAProp1 = 0;
+    private int level2ModuleAProp2 = 0;
+    private int level2ModuleAProp3 = 0;
+
+    @InputPortFieldAnnotation(optional = true)
+    public final transient ProxyInputPort<Integer> mIn = new ProxyInputPort<>();
+
+    @OutputPortFieldAnnotation(optional = true)
+    public final transient ProxyOutputPort<Integer> mOut1 = new ProxyOutputPort<>();
+
+    @OutputPortFieldAnnotation(optional = true)
+    public final transient ProxyOutputPort<Integer> mOut2 = new ProxyOutputPort<>();
+
+    @Override
+    public void populateDAG(DAG dag, Configuration conf)
+    {
+      Level1Module m1 = dag.addModule("M1", new Level1Module());
+      m1.setLevel1ModuleProp(level2ModuleAProp1);
+
+      Level1Module m2 = dag.addModule("M2", new Level1Module());
+      m2.setLevel1ModuleProp(level2ModuleAProp2);
+
+      DummyOperator o1 = dag.addOperator("O1", new DummyOperator());
+      o1.setOperatorProp(level2ModuleAProp3);
+
+      dag.addStream("M1_M2&O1", m1.mOut, m2.mIn, o1.in);
+
+      mIn.set(m1.mIn);
+      mOut1.set(m2.mOut);
+      mOut2.set(o1.out1);
+    }
+
+    public int getLevel2ModuleAProp1()
+    {
+      return level2ModuleAProp1;
+    }
+
+    public void setLevel2ModuleAProp1(int level2ModuleAProp1)
+    {
+      this.level2ModuleAProp1 = level2ModuleAProp1;
+    }
+
+    public int getLevel2ModuleAProp2()
+    {
+      return level2ModuleAProp2;
+    }
+
+    public void setLevel2ModuleAProp2(int level2ModuleAProp2)
+    {
+      this.level2ModuleAProp2 = level2ModuleAProp2;
+    }
+
+    public int getLevel2ModuleAProp3()
+    {
+      return level2ModuleAProp3;
+    }
+
+    public void setLevel2ModuleAProp3(int level2ModuleAProp3)
+    {
+      this.level2ModuleAProp3 = level2ModuleAProp3;
+    }
+  }
+
+  static class Level2ModuleB implements Module
+  {
+    private int level2ModuleBProp1 = 0;
+    private int level2ModuleBProp2 = 0;
+    private int level2ModuleBProp3 = 0;
+
+    @InputPortFieldAnnotation(optional = true)
+    public final transient ProxyInputPort<Integer> mIn = new ProxyInputPort<>();
+
+    @OutputPortFieldAnnotation(optional = true)
+    public final transient ProxyOutputPort<Integer> mOut1 = new ProxyOutputPort<>();
+
+    @OutputPortFieldAnnotation(optional = true)
+    public final transient ProxyOutputPort<Integer> mOut2 = new ProxyOutputPort<>();
+
+    @Override
+    public void populateDAG(DAG dag, Configuration conf)
+    {
+      DummyOperator o1 = dag.addOperator("O1", new DummyOperator());
+      o1.setOperatorProp(level2ModuleBProp1);
+
+      Level1Module m1 = dag.addModule("M1", new Level1Module());
+      m1.setLevel1ModuleProp(level2ModuleBProp2);
+
+      DummyOperator o2 = dag.addOperator("O2", new DummyOperator());
+      o2.setOperatorProp(level2ModuleBProp3);
+
+      dag.addStream("O1_M1", o1.out1, m1.mIn);
+      dag.addStream("O1_O2", o1.out2, o2.in);
+
+      mIn.set(o1.in);
+      mOut1.set(m1.mOut);
+      mOut2.set(o2.out1);
+    }
+
+    public int getLevel2ModuleBProp1()
+    {
+      return level2ModuleBProp1;
+    }
+
+    public void setLevel2ModuleBProp1(int level2ModuleBProp1)
+    {
+      this.level2ModuleBProp1 = level2ModuleBProp1;
+    }
+
+    public int getLevel2ModuleBProp2()
+    {
+      return level2ModuleBProp2;
+    }
+
+    public void setLevel2ModuleBProp2(int level2ModuleBProp2)
+    {
+      this.level2ModuleBProp2 = level2ModuleBProp2;
+    }
+
+    public int getLevel2ModuleBProp3()
+    {
+      return level2ModuleBProp3;
+    }
+
+    public void setLevel2ModuleBProp3(int level2ModuleBProp3)
+    {
+      this.level2ModuleBProp3 = level2ModuleBProp3;
+    }
+  }
+
+  static class Level3Module implements Module
+  {
+
+    public final transient ProxyInputPort<Integer> mIn = new ProxyInputPort<>();
+    public final transient ProxyOutputPort<Integer> mOut1 = new ProxyOutputPort<>();
+    public final transient ProxyOutputPort<Integer> mOut2 = new ProxyOutputPort<>();
+
+    @Override
+    public void populateDAG(DAG dag, Configuration conf)
+    {
+      DummyOperator op = dag.addOperator("O1", new DummyOperator());
+      Level2ModuleB m1 = dag.addModule("M1", new Level2ModuleB());
+      Level1Module m2 = dag.addModule("M2", new Level1Module());
+
+      dag.addStream("s1", op.out1, m1.mIn);
+      dag.addStream("s2", op.out2, m2.mIn);
+
+      mIn.set(op.in);
+      mOut1.set(m1.mOut1);
+      mOut2.set(m2.mOut);
+    }
+  }
+
+  static class NestedModuleApp implements StreamingApplication
+  {
+    @Override
+    public void populateDAG(DAG dag, Configuration conf)
+    {
+      DummyInputOperator o1 = dag.addOperator("O1", new DummyInputOperator());
+      o1.setInputOperatorProp(1);
+
+      DummyOperator o2 = dag.addOperator("O2", new DummyOperator());
+      o2.setOperatorProp(2);
+
+      Level2ModuleA ma = dag.addModule("Ma", new Level2ModuleA());
+      ma.setLevel2ModuleAProp1(11);
+      ma.setLevel2ModuleAProp2(12);
+      ma.setLevel2ModuleAProp3(13);
+
+      Level2ModuleB mb = dag.addModule("Mb", new Level2ModuleB());
+      mb.setLevel2ModuleBProp1(21);
+      mb.setLevel2ModuleBProp2(22);
+      mb.setLevel2ModuleBProp3(23);
+
+      Level2ModuleA mc = dag.addModule("Mc", new Level2ModuleA());
+      mc.setLevel2ModuleAProp1(31);
+      mc.setLevel2ModuleAProp2(32);
+      mc.setLevel2ModuleAProp3(33);
+
+      Level2ModuleB md = dag.addModule("Md", new Level2ModuleB());
+      md.setLevel2ModuleBProp1(41);
+      md.setLevel2ModuleBProp2(42);
+      md.setLevel2ModuleBProp3(43);
+
+      Level3Module me = dag.addModule("Me", new Level3Module());
+
+      dag.addStream("O1_O2", o1.out, o2.in, me.mIn);
+      dag.addStream("O2_Ma", o2.out1, ma.mIn);
+      dag.addStream("Ma_Mb", ma.mOut1, mb.mIn);
+      dag.addStream("Ma_Md", ma.mOut2, md.mIn);
+      dag.addStream("Mb_Mc", mb.mOut2, mc.mIn);
+    }
+  }
+
+  @Test
+  public void testModuleExtreme()
+  {
+    StreamingApplication app = new NestedModuleApp();
+    Configuration conf = new Configuration(false);
+    LogicalPlanConfiguration lpc = new LogicalPlanConfiguration(conf);
+    LogicalPlan dag = new LogicalPlan();
+    lpc.prepareDAG(dag, app, "ModuleApp");
+
+    dag.validate();
+    validateTopLevelOperators(dag);
+    validateTopLevelStreams(dag);
+    validatePublicMethods(dag);
+  }
+
+  private void validateTopLevelStreams(LogicalPlan dag)
+  {
+    List<String> streamNames = new ArrayList<>();
+    for (LogicalPlan.StreamMeta streamMeta : dag.getAllStreams()) {
+      streamNames.add(streamMeta.getName());
+    }
+
+    Assert.assertTrue(streamNames.contains(componentName("Mb", "O1_M1")));
+    Assert.assertTrue(streamNames.contains("O2_Ma"));
+    Assert.assertTrue(streamNames.contains("Mb_Mc"));
+    Assert.assertTrue(streamNames.contains(componentName("Mb", "O1_O2")));
+    Assert.assertTrue(streamNames.contains(componentName("Ma", "M1_M2&O1")));
+    Assert.assertTrue(streamNames.contains(componentName("Md", "O1_M1")));
+    Assert.assertTrue(streamNames.contains(componentName("Ma_Md")));
+    Assert.assertTrue(streamNames.contains(componentName("Mc", "M1_M2&O1")));
+    Assert.assertTrue(streamNames.contains(componentName("Md", "O1_O2")));
+    Assert.assertTrue(streamNames.contains("Ma_Mb"));
+    Assert.assertTrue(streamNames.contains("O1_O2"));
+
+    validateSeperateStream(dag, componentName("Mb", "O1_M1"), componentName("Mb", "O1"),
+        componentName("Mb", "M1", "O1"));
+    validateSeperateStream(dag, "O2_Ma", "O2", componentName("Ma", "M1", "O1"));
+    validateSeperateStream(dag, "Mb_Mc", componentName("Mb", "O2"), componentName("Mc", "M1", "O1"));
+    validateSeperateStream(dag, componentName("Mb", "O1_O2"), componentName("Mb", "O1"), componentName("Mb", "O2"));
+    validateSeperateStream(dag, componentName("Ma", "M1_M2&O1"), componentName("Ma", "M1", "O1"),
+        componentName("Ma", "O1"), componentName("Ma", "M2", "O1"));
+    validateSeperateStream(dag, componentName("Md", "O1_M1"), componentName("Md", "O1"),
+        componentName("Md", "M1", "O1"));
+    validateSeperateStream(dag, "Ma_Md", componentName("Ma", "O1"), componentName("Md", "O1"));
+    validateSeperateStream(dag, componentName("Mc", "M1_M2&O1"), componentName("Mc", "M1", "O1"),
+        componentName("Mc", "O1"), componentName("Mc", "M2", "O1"));
+    validateSeperateStream(dag, componentName("Md", "O1_O2"), componentName("Md", "O1"), componentName("Md", "O2"));
+    validateSeperateStream(dag, "Ma_Mb", componentName("Ma", "M2", "O1"), componentName("Mb", "O1"));
+    validateSeperateStream(dag, "O1_O2", "O1", "O2", componentName("Me", "O1"));
+  }
+
+  private void validateSeperateStream(LogicalPlan dag, String streamName, String inputOperatorName,
+      String... outputOperatorNames)
+  {
+    LogicalPlan.StreamMeta streamMeta = dag.getStream(streamName);
+    String sourceName = streamMeta.getSource().getOperatorMeta().getName();
+
+    List<String> sinksName = new ArrayList<>();
+    for (LogicalPlan.InputPortMeta inputPortMeta : streamMeta.getSinks()) {
+      sinksName.add(inputPortMeta.getOperatorWrapper().getName());
+    }
+
+    Assert.assertTrue(inputOperatorName.equals(sourceName));
+    Assert.assertEquals(outputOperatorNames.length, sinksName.size());
+
+    for (String outputOperatorName : outputOperatorNames) {
+      Assert.assertTrue(sinksName.contains(outputOperatorName));
+    }
+  }
+
+  private void validateTopLevelOperators(LogicalPlan dag)
+  {
+    List<String> operatorNames = new ArrayList<>();
+    for (LogicalPlan.OperatorMeta operatorMeta : dag.getAllOperators()) {
+      operatorNames.add(operatorMeta.getName());
+    }
+    Assert.assertTrue(operatorNames.contains("O1"));
+    Assert.assertTrue(operatorNames.contains("O2"));
+    Assert.assertTrue(operatorNames.contains(componentName("Ma", "M1", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Ma", "M2", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Ma", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Mb", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Mb", "M1", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Mb", "O2")));
+    Assert.assertTrue(operatorNames.contains(componentName("Mc", "M1", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Mc", "M2", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Mc", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Md", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Md", "M1", "O1")));
+    Assert.assertTrue(operatorNames.contains(componentName("Md", "O2")));
+
+    validateOperatorPropertyValue(dag, "O1", 1);
+    validateOperatorPropertyValue(dag, "O2", 2);
+    validateOperatorPropertyValue(dag, componentName("Ma", "M1", "O1"), 11);
+    validateOperatorPropertyValue(dag, componentName("Ma", "M2", "O1"), 12);
+    validateOperatorPropertyValue(dag, componentName("Ma", "O1"), 13);
+    validateOperatorPropertyValue(dag, componentName("Mb", "O1"), 21);
+    validateOperatorPropertyValue(dag, componentName("Mb", "M1", "O1"), 22);
+    validateOperatorPropertyValue(dag, componentName("Mb", "O2"), 23);
+    validateOperatorPropertyValue(dag, componentName("Mc", "M1", "O1"), 31);
+    validateOperatorPropertyValue(dag, componentName("Mc", "M2", "O1"), 32);
+    validateOperatorPropertyValue(dag, componentName("Mc", "O1"), 33);
+    validateOperatorPropertyValue(dag, componentName("Md", "O1"), 41);
+    validateOperatorPropertyValue(dag, componentName("Md", "M1", "O1"), 42);
+    validateOperatorPropertyValue(dag, componentName("Md", "O2"), 43);
+
+    validateOperatorParent(dag, "O1", null);
+    validateOperatorParent(dag, "O2", null);
+    validateOperatorParent(dag, componentName("Ma", "M1", "O1"), componentName("Ma", "M1"));
+    validateOperatorParent(dag, componentName("Ma", "M2", "O1"), componentName("Ma", "M2"));
+    validateOperatorParent(dag, componentName("Ma", "O1"), "Ma");
+    validateOperatorParent(dag, componentName("Mb", "O1"), "Mb");
+    validateOperatorParent(dag, componentName("Mb", "M1", "O1"), componentName("Mb", "M1"));
+    validateOperatorParent(dag, componentName("Mb", "O2"), "Mb");
+    validateOperatorParent(dag, componentName("Mc", "M1", "O1"), componentName("Mc", "M1"));
+    validateOperatorParent(dag, componentName("Mc", "M2", "O1"), componentName("Mc", "M2"));
+    validateOperatorParent(dag, componentName("Mc", "O1"), "Mc");
+    validateOperatorParent(dag, componentName("Md", "O1"), "Md");
+    validateOperatorParent(dag, componentName("Md", "M1", "O1"), componentName("Md", "M1"));
+    validateOperatorParent(dag, componentName("Md", "O2"), "Md");
+  }
+
+  private void validateOperatorParent(LogicalPlan dag, String operatorName, String parentModuleName)
+  {
+    LogicalPlan.OperatorMeta operatorMeta = dag.getOperatorMeta(operatorName);
+    if (parentModuleName == null) {
+      Assert.assertNull(operatorMeta.getModuleName());
+    } else {
+      Assert.assertTrue(parentModuleName.equals(operatorMeta.getModuleName()));
+    }
+  }
+
+  private void validateOperatorPropertyValue(LogicalPlan dag, String operatorName, int expectedValue)
+  {
+    LogicalPlan.OperatorMeta oMeta = dag.getOperatorMeta(operatorName);
+    if (operatorName.equals("O1")) {
+      DummyInputOperator operator = (DummyInputOperator)oMeta.getOperator();
+      Assert.assertEquals(expectedValue, operator.getInputOperatorProp());
+    } else {
+      DummyOperator operator = (DummyOperator)oMeta.getOperator();
+      Assert.assertEquals(expectedValue, operator.getOperatorProp());
+    }
+  }
+
+  private void validatePublicMethods(LogicalPlan dag)
+  {
+    // Logical dag contains 4 modules added on top level.
+    List<String> moduleNames = new ArrayList<>();
+    for (LogicalPlan.ModuleMeta moduleMeta : dag.getAllModules()) {
+      moduleNames.add(moduleMeta.getName());
+    }
+    Assert.assertTrue(moduleNames.contains("Ma"));
+    Assert.assertTrue(moduleNames.contains("Mb"));
+    Assert.assertTrue(moduleNames.contains("Mc"));
+    Assert.assertTrue(moduleNames.contains("Md"));
+    Assert.assertTrue(moduleNames.contains("Me"));
+    Assert.assertEquals("Number of modules are 5", 5, dag.getAllModules().size());
+
+    // correct module meta is returned by getMeta call.
+    LogicalPlan.ModuleMeta m = dag.getModuleMeta("Ma");
+    Assert.assertEquals("Name of module is Ma", m.getName(), "Ma");
+
+  }
+
+  private static String componentName(String... names)
+  {
+    if (names.length == 0) {
+      return "";
+    }
+    StringBuilder sb = new StringBuilder(names[0]);
+    for (int i = 1; i < names.length; i++) {
+      sb.append(LogicalPlan.MODULE_NAMESPACE_SEPARATOR);
+      sb.append(names[i]);
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Generate a conflict, Add a top level operator with name "m1_O1",
+   * and add a module "m1" which will populate operator "O1", causing name conflict with
+   * top level operator.
+   */
+  @Test(expected = java.lang.IllegalArgumentException.class)
+  public void conflictingNamesWithExpandedModule()
+  {
+    Configuration conf = new Configuration(false);
+    LogicalPlanConfiguration lpc = new LogicalPlanConfiguration(conf);
+    LogicalPlan dag = new LogicalPlan();
+    DummyInputOperator in = dag.addOperator(componentName("m1", "O1"), new DummyInputOperator());
+    Level2ModuleA module = dag.addModule("m1", new Level2ModuleA());
+    dag.addStream("s1", in.out, module.mIn);
+    lpc.prepareDAG(dag, null, "ModuleApp");
+    dag.validate();
+  }
+
+  /**
+   * Module and Operator with same name is not allowed in a DAG, to prevent properties
+   * conflict.
+   */
+  @Test(expected = java.lang.IllegalArgumentException.class)
+  public void conflictingNamesWithOperator1()
+  {
+    Configuration conf = new Configuration(false);
+    LogicalPlanConfiguration lpc = new LogicalPlanConfiguration(conf);
+    LogicalPlan dag = new LogicalPlan();
+    DummyInputOperator in = dag.addOperator("M1", new DummyInputOperator());
+    Level2ModuleA module = dag.addModule("M1", new Level2ModuleA());
+    dag.addStream("s1", in.out, module.mIn);
+    lpc.prepareDAG(dag, null, "ModuleApp");
+    dag.validate();
+  }
+
+  /**
+   * Module and Operator with same name is not allowed in a DAG, to prevent properties
+   * conflict.
+   */
+  @Test(expected = java.lang.IllegalArgumentException.class)
+  public void conflictingNamesWithOperator2()
+  {
+    Configuration conf = new Configuration(false);
+    LogicalPlanConfiguration lpc = new LogicalPlanConfiguration(conf);
+    LogicalPlan dag = new LogicalPlan();
+    Level2ModuleA module = dag.addModule("M1", new Level2ModuleA());
+    DummyInputOperator in = dag.addOperator("M1", new DummyInputOperator());
+    dag.addStream("s1", in.out, module.mIn);
+    lpc.prepareDAG(dag, null, "ModuleApp");
+    dag.validate();
+  }
+}


[31/50] incubator-apex-core git commit: APEXCORE-268 #resolve #comment removed style violations from common

Posted by th...@apache.org.
APEXCORE-268 #resolve #comment removed style violations from common


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

Branch: refs/heads/master
Commit: 778436f76a21f655d844c18b7a2956115c3853a9
Parents: 7629be8
Author: MalharJenkins <je...@datatorrent.com>
Authored: Fri Jan 8 13:49:38 2016 -0800
Committer: Chandni Singh <cs...@apache.org>
Committed: Fri Jan 8 13:49:59 2016 -0800

----------------------------------------------------------------------
 common/pom.xml                                  |  2 +-
 .../common/codec/JsonStreamCodec.java           | 13 +++--
 .../common/experimental/AppData.java            | 14 ++++--
 .../common/metric/MetricsAggregator.java        | 12 ++---
 .../common/metric/SingleMetricAggregator.java   |  1 +
 .../partitioner/StatelessPartitioner.java       | 51 +++++++++-----------
 .../auth/callback/DefaultCallbackHandler.java   | 10 ++--
 .../common/util/AsyncFSStorageAgent.java        | 20 ++++++--
 .../datatorrent/common/util/BaseOperator.java   |  2 -
 .../util/BasicContainerOptConfigurator.java     | 22 ++++-----
 .../datatorrent/common/util/FSStorageAgent.java | 42 ++++++++--------
 .../util/JacksonObjectMapperProvider.java       |  6 ++-
 .../common/util/NameableThreadFactory.java      |  4 +-
 .../datatorrent/common/util/PubSubMessage.java  |  3 +-
 .../common/util/PubSubMessageCodec.java         |  9 ++--
 .../common/util/PubSubWebSocketClient.java      | 50 ++++++++++---------
 .../common/util/SerializableObject.java         | 41 ++++------------
 .../common/codec/JsonStreamCodecTest.java       |  4 +-
 .../partitioner/StatelessPartitionerTest.java   | 14 ++++--
 .../common/util/AsyncFSStorageAgentTest.java    | 13 ++---
 .../common/util/FSStorageAgentTest.java         | 19 ++++----
 .../common/util/SerializableObjectTest.java     |  5 +-
 22 files changed, 180 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 94f317f..5c2c98f 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -56,7 +56,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>114</maxAllowedViolations>
+          <logViolationsToConsole>true</logViolationsToConsole>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/codec/JsonStreamCodec.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/codec/JsonStreamCodec.java b/common/src/main/java/com/datatorrent/common/codec/JsonStreamCodec.java
index 1c3240b..a17023f 100644
--- a/common/src/main/java/com/datatorrent/common/codec/JsonStreamCodec.java
+++ b/common/src/main/java/com/datatorrent/common/codec/JsonStreamCodec.java
@@ -18,7 +18,9 @@
  */
 package com.datatorrent.common.codec;
 
-import java.io.*;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.util.Map;
 
 import org.codehaus.jackson.JsonGenerator;
@@ -65,8 +67,7 @@ public class JsonStreamCodec<T> implements StreamCodec<T>
             }
 
           });
-        }
-        catch (Exception ex) {
+        } catch (Exception ex) {
           logger.error("Caught exception when instantiating codec for class {}", entry.getKey().getName(), ex);
         }
       }
@@ -80,8 +81,7 @@ public class JsonStreamCodec<T> implements StreamCodec<T>
     ByteArrayInputStream bis = new ByteArrayInputStream(data.buffer, data.offset, data.length);
     try {
       return mapper.readValue(bis, Object.class);
-    }
-    catch (Exception ioe) {
+    } catch (Exception ioe) {
       throw new RuntimeException(ioe);
     }
   }
@@ -95,8 +95,7 @@ public class JsonStreamCodec<T> implements StreamCodec<T>
       mapper.writeValue(bos, o);
       byte[] bytes = bos.toByteArray();
       return new Slice(bytes, 0, bytes.length);
-    }
-    catch (IOException ex) {
+    } catch (IOException ex) {
       throw new RuntimeException(ex);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/experimental/AppData.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/experimental/AppData.java b/common/src/main/java/com/datatorrent/common/experimental/AppData.java
index 7c2a56a..bbf9753 100644
--- a/common/src/main/java/com/datatorrent/common/experimental/AppData.java
+++ b/common/src/main/java/com/datatorrent/common/experimental/AppData.java
@@ -18,17 +18,19 @@
  */
 package com.datatorrent.common.experimental;
 
-import com.datatorrent.api.Context.OperatorContext;
-import com.datatorrent.api.DefaultOutputPort;
-import com.datatorrent.api.Operator;
 import java.lang.annotation.Documented;
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Inherited;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
+
 import org.apache.hadoop.classification.InterfaceStability;
 
+import com.datatorrent.api.Context.OperatorContext;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.Operator;
+
 /**
  * Interface for App Data support. Experimental only. This interface will likely change in the near future.
  *
@@ -96,6 +98,7 @@ public interface AppData
      * @return The connection url used by the AppData Query or Result operator.
      */
     public String getAppDataURL();
+
     /**
      * Returns the topic that the appdata Query or Result operator sends data to.
      * @return The topic that the appdata Query or Result operator sends data to.
@@ -110,7 +113,10 @@ public interface AppData
   @Target(ElementType.TYPE)
   @Retention(RetentionPolicy.RUNTIME)
   @Inherited
-  public @interface AppendQueryIdToTopic{boolean value() default false;}
+  public @interface AppendQueryIdToTopic
+  {
+    boolean value() default false;
+  }
 
   /**
    * Marker annotation for specifying appdata query ports.

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/metric/MetricsAggregator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/metric/MetricsAggregator.java b/common/src/main/java/com/datatorrent/common/metric/MetricsAggregator.java
index 4192354..e34f74a 100644
--- a/common/src/main/java/com/datatorrent/common/metric/MetricsAggregator.java
+++ b/common/src/main/java/com/datatorrent/common/metric/MetricsAggregator.java
@@ -98,8 +98,7 @@ public class MetricsAggregator implements AutoMetric.Aggregator, Serializable
     String aggregatorDesc;
     if (aggregatorName == null) {
       aggregatorDesc = aggregator.getClass().getName();
-    }
-    else {
+    } else {
       aggregatorDesc = aggregatorName.value();
     }
     return aggregatorDesc + aggregatorMetricSeparator + metric;
@@ -129,13 +128,13 @@ public class MetricsAggregator implements AutoMetric.Aggregator, Serializable
    *                           be used for the result of aggregators[i].
    */
   public void addAggregators(@NotNull String metric, @NotNull SingleMetricAggregator[] aggregators,
-                             @NotNull String[] logicalMetricNames)
+      @NotNull String[] logicalMetricNames)
   {
     Preconditions.checkNotNull(metric, "metric");
     Preconditions.checkNotNull(aggregators, "aggregators");
     Preconditions.checkNotNull(logicalMetricNames, "logicalMetricNames");
-    Preconditions.checkArgument(aggregators.length == logicalMetricNames.length, "different length aggregators and" +
-      " logical names");
+    Preconditions.checkArgument(aggregators.length == logicalMetricNames.length,
+        "different length aggregators and logical names");
     addAggregatorsHelper(metric, aggregators, logicalMetricNames);
   }
 
@@ -149,8 +148,7 @@ public class MetricsAggregator implements AutoMetric.Aggregator, Serializable
     for (int i = 0; i < aggregators.length; i++) {
 
       String resultName = (logicalMetricNames == null || logicalMetricNames[i] == null) ?
-        (aggregators.length == 1 ? metric : deriveLogicalMetricName(metric, aggregators[i]))
-        : logicalMetricNames[i];
+          (aggregators.length == 1 ? metric : deriveLogicalMetricName(metric, aggregators[i])) : logicalMetricNames[i];
 
       laggregators.add(new LogicalMetricMeta(aggregators[i], resultName));
     }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/metric/SingleMetricAggregator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/metric/SingleMetricAggregator.java b/common/src/main/java/com/datatorrent/common/metric/SingleMetricAggregator.java
index 9b568d2..dbc9b8a 100644
--- a/common/src/main/java/com/datatorrent/common/metric/SingleMetricAggregator.java
+++ b/common/src/main/java/com/datatorrent/common/metric/SingleMetricAggregator.java
@@ -19,6 +19,7 @@
 package com.datatorrent.common.metric;
 
 import java.util.Collection;
+
 /**
  * <p>SingleMetricAggregator interface.</p>
  *

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/partitioner/StatelessPartitioner.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/partitioner/StatelessPartitioner.java b/common/src/main/java/com/datatorrent/common/partitioner/StatelessPartitioner.java
index c36dd8a..165d8cf 100644
--- a/common/src/main/java/com/datatorrent/common/partitioner/StatelessPartitioner.java
+++ b/common/src/main/java/com/datatorrent/common/partitioner/StatelessPartitioner.java
@@ -19,16 +19,22 @@
 package com.datatorrent.common.partitioner;
 
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import javax.validation.constraints.Min;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
 import com.datatorrent.api.DefaultPartition;
 import com.datatorrent.api.Operator;
 import com.datatorrent.api.Operator.InputPort;
@@ -100,7 +106,7 @@ public class StatelessPartitioner<T extends Operator> implements Partitioner<T>,
     logger.debug("define partitions, partitionCount current {} requested {}", partitions.size(), newPartitionCount);
 
     //Get a partition
-    DefaultPartition<T> partition = (DefaultPartition<T>) partitions.iterator().next();
+    DefaultPartition<T> partition = (DefaultPartition<T>)partitions.iterator().next();
     Collection<Partition<T>> newPartitions;
 
     if (partitions.iterator().next().getStats() == null) {
@@ -117,16 +123,13 @@ public class StatelessPartitioner<T extends Operator> implements Partitioner<T>,
       if (inputPortList != null && !inputPortList.isEmpty()) {
         DefaultPartition.assignPartitionKeys(newPartitions, inputPortList.iterator().next());
       }
-    }
-    else {
+    } else {
       // define partitions is being called again
       if (context.getParallelPartitionCount() != 0) {
         newPartitions = repartitionParallel(partitions, context);
-      }
-      else if (partition.getPartitionKeys().isEmpty()) {
+      } else if (partition.getPartitionKeys().isEmpty()) {
         newPartitions = repartitionInputOperator(partitions);
-      }
-      else {
+      } else {
         newPartitions = repartition(partitions);
       }
     }
@@ -166,8 +169,7 @@ public class StatelessPartitioner<T extends Operator> implements Partitioner<T>,
           Partition<T> siblingPartition = lowLoadPartitions.remove(partitionKey & reducedMask);
           if (siblingPartition == null) {
             lowLoadPartitions.put(partitionKey & reducedMask, p);
-          }
-          else {
+          } else {
             // both of the partitions are low load, combine
             PartitionKeys newPks = new PartitionKeys(reducedMask, Sets.newHashSet(partitionKey & reducedMask));
             // put new value so the map gets marked as modified
@@ -178,8 +180,7 @@ public class StatelessPartitioner<T extends Operator> implements Partitioner<T>,
             //LOG.debug("partition keys after merge {}", siblingPartition.getPartitionKeys());
           }
         }
-      }
-      else if (load > 0) {
+      } else if (load > 0) {
         // split bottlenecks
         Map<InputPort<?>, PartitionKeys> keys = p.getPartitionKeys();
         Map.Entry<InputPort<?>, PartitionKeys> e = keys.entrySet().iterator().next();
@@ -193,8 +194,7 @@ public class StatelessPartitioner<T extends Operator> implements Partitioner<T>,
           int key = e.getValue().partitions.iterator().next();
           int key2 = (newMask ^ e.getValue().mask) | key;
           newKeys = Sets.newHashSet(key, key2);
-        }
-        else {
+        } else {
           // assign keys to separate partitions
           newMask = e.getValue().mask;
           newKeys = e.getValue().partitions;
@@ -205,8 +205,7 @@ public class StatelessPartitioner<T extends Operator> implements Partitioner<T>,
           newPartition.getPartitionKeys().put(e.getKey(), new PartitionKeys(newMask, Sets.newHashSet(key)));
           newPartitions.add(newPartition);
         }
-      }
-      else {
+      } else {
         // leave unchanged
         newPartitions.add(p);
       }
@@ -232,16 +231,13 @@ public class StatelessPartitioner<T extends Operator> implements Partitioner<T>,
       if (load < 0) {
         if (!lowLoadPartitions.isEmpty()) {
           newPartitions.add(lowLoadPartitions.remove(0));
-        }
-        else {
+        } else {
           lowLoadPartitions.add(p);
         }
-      }
-      else if (load > 0) {
+      } else if (load > 0) {
         newPartitions.add(new DefaultPartition<T>(p.getPartitionedInstance()));
         newPartitions.add(new DefaultPartition<T>(p.getPartitionedInstance()));
-      }
-      else {
+      } else {
         newPartitions.add(p);
       }
     }
@@ -259,7 +255,7 @@ public class StatelessPartitioner<T extends Operator> implements Partitioner<T>,
    * @return new adjusted partitions
    */
   public static <T extends Operator> Collection<Partition<T>> repartitionParallel(Collection<Partition<T>> partitions,
-                                                                                  PartitioningContext context)
+      PartitioningContext context)
   {
     List<Partition<T>> newPartitions = Lists.newArrayList();
     newPartitions.addAll(partitions);
@@ -273,8 +269,7 @@ public class StatelessPartitioner<T extends Operator> implements Partitioner<T>,
         partitionIterator.next();
         partitionIterator.remove();
       }
-    }
-    else {
+    } else {
       //Add more partitions
       T anOperator = newPartitions.iterator().next().getPartitionedInstance();
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/security/auth/callback/DefaultCallbackHandler.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/security/auth/callback/DefaultCallbackHandler.java b/common/src/main/java/com/datatorrent/common/security/auth/callback/DefaultCallbackHandler.java
index 5792ad5..779a156 100644
--- a/common/src/main/java/com/datatorrent/common/security/auth/callback/DefaultCallbackHandler.java
+++ b/common/src/main/java/com/datatorrent/common/security/auth/callback/DefaultCallbackHandler.java
@@ -20,14 +20,18 @@ package com.datatorrent.common.security.auth.callback;
 
 import java.io.IOException;
 
-import javax.security.auth.callback.*;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.TextOutputCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
 import javax.security.sasl.RealmCallback;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.datatorrent.api.Component;
-
 import com.datatorrent.common.security.SecurityContext;
 
 /**
@@ -78,7 +82,7 @@ public class DefaultCallbackHandler implements CallbackHandler, Component<Securi
       PasswordCallback passcb = (PasswordCallback)callback;
       passcb.setPassword(context.getValue(SecurityContext.PASSWORD));
     } else if (callback instanceof RealmCallback) {
-      RealmCallback realmcb = (RealmCallback) callback;
+      RealmCallback realmcb = (RealmCallback)callback;
       realmcb.setText(context.getValue(SecurityContext.REALM));
     } else if (callback instanceof TextOutputCallback) {
       TextOutputCallback textcb = (TextOutputCallback)callback;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/AsyncFSStorageAgent.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/AsyncFSStorageAgent.java b/common/src/main/java/com/datatorrent/common/util/AsyncFSStorageAgent.java
index 83bbdca..788a68c 100644
--- a/common/src/main/java/com/datatorrent/common/util/AsyncFSStorageAgent.java
+++ b/common/src/main/java/com/datatorrent/common/util/AsyncFSStorageAgent.java
@@ -18,16 +18,26 @@
  */
 package com.datatorrent.common.util;
 
-import java.io.*;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectStreamException;
 import java.nio.file.Files;
 import java.util.EnumSet;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.IOUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+
 import com.datatorrent.netlet.util.DTThrowable;
 
 /**
@@ -74,7 +84,7 @@ public class AsyncFSStorageAgent extends FSStorageAgent
   @Override
   public void save(final Object object, final int operatorId, final long windowId) throws IOException
   {
-    if(syncCheckpoint){
+    if (syncCheckpoint) {
       super.save(object, operatorId, windowId);
       return;
     }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/BaseOperator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/BaseOperator.java b/common/src/main/java/com/datatorrent/common/util/BaseOperator.java
index 5b00e44..4601f3a 100644
--- a/common/src/main/java/com/datatorrent/common/util/BaseOperator.java
+++ b/common/src/main/java/com/datatorrent/common/util/BaseOperator.java
@@ -21,8 +21,6 @@ package com.datatorrent.common.util;
 import com.datatorrent.api.Context.OperatorContext;
 import com.datatorrent.api.Operator;
 
-import com.datatorrent.common.util.SerializableObject;
-
 /**
  * Base class for operator implementations that provides empty implementations
  * for all interface methods.

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/BasicContainerOptConfigurator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/BasicContainerOptConfigurator.java b/common/src/main/java/com/datatorrent/common/util/BasicContainerOptConfigurator.java
index 328ec7b..86ce23d 100644
--- a/common/src/main/java/com/datatorrent/common/util/BasicContainerOptConfigurator.java
+++ b/common/src/main/java/com/datatorrent/common/util/BasicContainerOptConfigurator.java
@@ -63,11 +63,10 @@ public class BasicContainerOptConfigurator implements Context.ContainerOptConfig
     for (DAG.OperatorMeta operatorMeta : operatorMetaList) {
       Map<String, Object> operatorMap = parseJvmOpts(operatorMeta.getValue(Context.OperatorContext.JVM_OPTIONS), operatorMeta.getValue(Context.OperatorContext.MEMORY_MB));
       LOG.info("property map for operator {}", operatorMap);
-      Set<String> operatorPropertySet = (Set<String>) operatorMap.get(GENERIC);
+      Set<String> operatorPropertySet = (Set<String>)operatorMap.get(GENERIC);
       if (genericProperties == null) {
         genericProperties = operatorPropertySet;
-      }
-      else {
+      } else {
         if (operatorPropertySet != null && !genericProperties.equals(operatorPropertySet)) {
           throw new AssertionError("Properties don't match: " + genericProperties + " " + operatorPropertySet);
         }
@@ -77,15 +76,15 @@ public class BasicContainerOptConfigurator implements Context.ContainerOptConfig
     for (Map<String, Object> map : jvmOptsList) {
       String value;
       if (map.containsKey(XMX)) {
-        value = (String) map.get(XMX);
+        value = (String)map.get(XMX);
         xmx += getOptValue(value);
       }
       if (map.containsKey(XMS)) {
-        value = (String) map.get(XMS);
+        value = (String)map.get(XMS);
         xms += getOptValue(value);
       }
       if (map.containsKey(XSS)) {
-        value = (String) map.get(XSS);
+        value = (String)map.get(XSS);
         xss += getOptValue(value);
       }
     }
@@ -110,14 +109,11 @@ public class BasicContainerOptConfigurator implements Context.ContainerOptConfig
     long result;
     if (value.endsWith("g") || value.endsWith("G")) {
       result = Long.valueOf(value.substring(0, value.length() - 1)) * GB_TO_B;
-    }
-    else if (value.endsWith("m") || value.endsWith("M")) {
+    } else if (value.endsWith("m") || value.endsWith("M")) {
       result = Long.valueOf(value.substring(0, value.length() - 1)) * MB_TO_B;
-    }
-    else if (value.endsWith("k") || value.endsWith("K")) {
+    } else if (value.endsWith("k") || value.endsWith("K")) {
       result = Long.valueOf(value.substring(0, value.length() - 1)) * KB_TO_B;
-    }
-    else {
+    } else {
       result = Long.valueOf(value);
     }
     return result;
@@ -151,7 +147,7 @@ public class BasicContainerOptConfigurator implements Context.ContainerOptConfig
       int memoryOverhead = memory / 4;
       int heapSize = memory - memoryOverhead;
       if (memoryOverhead > 1024) {
-         heapSize = memory - 1024;
+        heapSize = memory - 1024;
       }
       map.put(XMX, heapSize + "m");
     }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/FSStorageAgent.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/FSStorageAgent.java b/common/src/main/java/com/datatorrent/common/util/FSStorageAgent.java
index c2f68a0..fd4c450 100644
--- a/common/src/main/java/com/datatorrent/common/util/FSStorageAgent.java
+++ b/common/src/main/java/com/datatorrent/common/util/FSStorageAgent.java
@@ -18,16 +18,28 @@
  */
 package com.datatorrent.common.util;
 
-import java.io.*;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectStreamException;
+import java.io.OutputStream;
+import java.io.Serializable;
 import java.net.URI;
 import java.util.EnumSet;
 import java.util.List;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
 import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
@@ -35,7 +47,6 @@ import com.google.common.collect.Lists;
 
 import com.datatorrent.api.StorageAgent;
 import com.datatorrent.api.annotation.Stateless;
-
 import com.datatorrent.netlet.util.DTThrowable;
 
 /**
@@ -71,12 +82,10 @@ public class FSStorageAgent implements StorageAgent, Serializable
 
       if (pathUri.getScheme() != null) {
         fileContext = FileContext.getFileContext(pathUri, conf == null ? new Configuration() : conf);
-      }
-      else {
+      } else {
         fileContext = FileContext.getFileContext(conf == null ? new Configuration() : conf);
       }
-    }
-    catch (IOException ex) {
+    } catch (IOException ex) {
       throw new RuntimeException(ex);
     }
   }
@@ -95,27 +104,23 @@ public class FSStorageAgent implements StorageAgent, Serializable
         Options.CreateOpts.CreateParent.createParent());
       store(stream, object);
       stateSaved = true;
-    }
-    catch (Throwable t) {
+    } catch (Throwable t) {
       logger.debug("while saving {} {}", operatorId, window, t);
       stateSaved = false;
       DTThrowable.rethrow(t);
-    }
-    finally {
+    } finally {
       try {
         if (stream != null) {
           stream.close();
         }
-      }
-      catch (IOException ie) {
+      } catch (IOException ie) {
         stateSaved = false;
         throw new RuntimeException(ie);
-      }
-      finally {
+      } finally {
         if (stateSaved) {
           logger.debug("Saving {}: {}", operatorId, window);
           fileContext.rename(lPath, new Path(path + Path.SEPARATOR + operatorIdStr + Path.SEPARATOR + window),
-            Options.Rename.OVERWRITE);
+              Options.Rename.OVERWRITE);
         }
       }
     }
@@ -130,8 +135,7 @@ public class FSStorageAgent implements StorageAgent, Serializable
     FSDataInputStream stream = fileContext.open(lPath);
     try {
       return retrieve(stream);
-    }
-    finally {
+    } finally {
       stream.close();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/JacksonObjectMapperProvider.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/JacksonObjectMapperProvider.java b/common/src/main/java/com/datatorrent/common/util/JacksonObjectMapperProvider.java
index 2f1735f..7723fed 100644
--- a/common/src/main/java/com/datatorrent/common/util/JacksonObjectMapperProvider.java
+++ b/common/src/main/java/com/datatorrent/common/util/JacksonObjectMapperProvider.java
@@ -19,13 +19,17 @@
 package com.datatorrent.common.util;
 
 import java.io.IOException;
+
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.ext.ContextResolver;
 import javax.ws.rs.ext.Provider;
+
 import org.codehaus.jackson.JsonGenerator;
 import org.codehaus.jackson.Version;
-import org.codehaus.jackson.map.*;
+import org.codehaus.jackson.map.JsonSerializer;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
 import org.codehaus.jackson.map.module.SimpleModule;
 import org.codehaus.jackson.map.ser.std.RawSerializer;
 import org.codehaus.jettison.json.JSONArray;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/NameableThreadFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/NameableThreadFactory.java b/common/src/main/java/com/datatorrent/common/util/NameableThreadFactory.java
index f3aa170..f0c61b5 100644
--- a/common/src/main/java/com/datatorrent/common/util/NameableThreadFactory.java
+++ b/common/src/main/java/com/datatorrent/common/util/NameableThreadFactory.java
@@ -54,9 +54,7 @@ public class NameableThreadFactory implements ThreadFactory
   @Override
   public Thread newThread(Runnable r)
   {
-    Thread t = new Thread(group, r,
-                          namePrefix + threadNumber.getAndIncrement(),
-                          0);
+    Thread t = new Thread(group, r, namePrefix + threadNumber.getAndIncrement(), 0);
     if (t.isDaemon() != this.isDaemon) {
       t.setDaemon(isDaemon);
     }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/PubSubMessage.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/PubSubMessage.java b/common/src/main/java/com/datatorrent/common/util/PubSubMessage.java
index 1704edb..f263d0c 100644
--- a/common/src/main/java/com/datatorrent/common/util/PubSubMessage.java
+++ b/common/src/main/java/com/datatorrent/common/util/PubSubMessage.java
@@ -53,7 +53,8 @@ public class PubSubMessage<T>
       return identifier;
     }
 
-    public static PubSubMessageType getPubSubMessageType(String identifier) {
+    public static PubSubMessageType getPubSubMessageType(String identifier)
+    {
       PubSubMessageType pubSubMessageType = null;
       for (PubSubMessageType value : PubSubMessageType.values()) {
         if (value.getIdentifier().equals(identifier)) {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/PubSubMessageCodec.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/PubSubMessageCodec.java b/common/src/main/java/com/datatorrent/common/util/PubSubMessageCodec.java
index aa84213..926fa8f 100644
--- a/common/src/main/java/com/datatorrent/common/util/PubSubMessageCodec.java
+++ b/common/src/main/java/com/datatorrent/common/util/PubSubMessageCodec.java
@@ -36,11 +36,13 @@ public class PubSubMessageCodec<T>
 
   private final ObjectMapper mapper;
 
-  public PubSubMessageCodec(ObjectMapper mapper) {
+  public PubSubMessageCodec(ObjectMapper mapper)
+  {
     this.mapper = mapper;
   }
 
-  public String formatMessage(PubSubMessage<T> pubSubMessage) throws IOException {
+  public String formatMessage(PubSubMessage<T> pubSubMessage) throws IOException
+  {
     HashMap<String, Object> map = new HashMap<String, Object>();
     map.put(PubSubMessage.TYPE_KEY, pubSubMessage.getType().getIdentifier());
     map.put(PubSubMessage.TOPIC_KEY, pubSubMessage.getTopic());
@@ -59,7 +61,8 @@ public class PubSubMessageCodec<T>
    * @throws IOException
    */
   @SuppressWarnings({"unchecked"})
-  public PubSubMessage<T> parseMessage(String message) throws IOException {
+  public PubSubMessage<T> parseMessage(String message) throws IOException
+  {
     HashMap<String, Object> map = mapper.readValue(message, HashMap.class);
     PubSubMessage<T> pubSubMessage = new PubSubMessage<T>();
     pubSubMessage.setType(PubSubMessageType.getPubSubMessageType((String)map.get(PubSubMessage.TYPE_KEY)));

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/PubSubWebSocketClient.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/PubSubWebSocketClient.java b/common/src/main/java/com/datatorrent/common/util/PubSubWebSocketClient.java
index 58072ee..c3f5961 100644
--- a/common/src/main/java/com/datatorrent/common/util/PubSubWebSocketClient.java
+++ b/common/src/main/java/com/datatorrent/common/util/PubSubWebSocketClient.java
@@ -21,13 +21,12 @@ package com.datatorrent.common.util;
 import java.io.IOException;
 import java.net.URI;
 import java.util.List;
-import java.util.concurrent.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
 
-import com.ning.http.client.*;
-import com.ning.http.client.AsyncHttpClient.BoundRequestBuilder;
-import com.ning.http.client.websocket.*;
-
 import org.codehaus.jackson.JsonParseException;
 import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -36,11 +35,19 @@ import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.datatorrent.common.util.PubSubMessage.PubSubMessageType;
+import com.ning.http.client.AsyncCompletionHandler;
+import com.ning.http.client.AsyncHttpClient;
+import com.ning.http.client.AsyncHttpClient.BoundRequestBuilder;
+import com.ning.http.client.AsyncHttpClientConfigBean;
+import com.ning.http.client.Cookie;
+import com.ning.http.client.Response;
+import com.ning.http.client.websocket.WebSocket;
+import com.ning.http.client.websocket.WebSocketTextListener;
+import com.ning.http.client.websocket.WebSocketUpgradeHandler;
 
 import com.datatorrent.api.Component;
 import com.datatorrent.api.Context;
-
+import com.datatorrent.common.util.PubSubMessage.PubSubMessageType;
 import com.datatorrent.netlet.util.DTThrowable;
 
 /**
@@ -71,14 +78,11 @@ public abstract class PubSubWebSocketClient implements Component<Context>
       try {
         pubSubMessage = codec.parseMessage(message);
         PubSubWebSocketClient.this.onMessage(pubSubMessage.getType().getIdentifier(), pubSubMessage.getTopic(), pubSubMessage.getData());
-      }
-      catch (JsonParseException jpe) {
+      } catch (JsonParseException jpe) {
         logger.warn("Ignoring unparseable JSON message: {}", message, jpe);
-      }
-      catch (JsonMappingException jme) {
+      } catch (JsonMappingException jme) {
         logger.warn("Ignoring JSON mapping in message: {}", message, jme);
-      }
-      catch (IOException ex) {
+      } catch (IOException ex) {
         onError(ex);
       }
     }
@@ -175,8 +179,7 @@ public abstract class PubSubWebSocketClient implements Component<Context>
       try {
         json.put("userName", userName);
         json.put("password", password);
-      }
-      catch (JSONException ex) {
+      } catch (JSONException ex) {
         throw new RuntimeException(ex);
       }
       Response response = client.preparePost(loginUrl).setHeader("Content-Type", "application/json").setBody(json.toString()).execute().get();
@@ -201,8 +204,7 @@ public abstract class PubSubWebSocketClient implements Component<Context>
       try {
         json.put("userName", userName);
         json.put("password", password);
-      }
-      catch (JSONException ex) {
+      } catch (JSONException ex) {
         throw new RuntimeException(ex);
       }
       client.preparePost(loginUrl).setHeader("Content-Type", "application/json").setBody(json.toString()).execute(new AsyncCompletionHandler<Response>()
@@ -223,9 +225,8 @@ public abstract class PubSubWebSocketClient implements Component<Context>
         }
 
       });
-    }
-    else {
-      client.prepareGet(uri.toString()).execute(new WebSocketUpgradeHandler.Builder().addWebSocketListener(new PubSubWebSocket()
+    } else {
+      final PubSubWebSocket webSocket = new PubSubWebSocket()
       {
         @Override
         public void onOpen(WebSocket ws)
@@ -233,8 +234,9 @@ public abstract class PubSubWebSocketClient implements Component<Context>
           connection = ws;
           super.onOpen(ws);
         }
-
-      }).build());
+      };
+      client.prepareGet(uri.toString()).execute(
+          new WebSocketUpgradeHandler.Builder().addWebSocketListener(webSocket).build());
     }
   }
 
@@ -289,11 +291,11 @@ public abstract class PubSubWebSocketClient implements Component<Context>
     Throwable t = throwable.get();
     if (t instanceof IOException) {
       throw (IOException)t;
-    }
-    else {
+    } else {
       DTThrowable.rethrow(t);
     }
   }
+
   /**
    * <p>publish.</p>
    *

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/main/java/com/datatorrent/common/util/SerializableObject.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/SerializableObject.java b/common/src/main/java/com/datatorrent/common/util/SerializableObject.java
index 203399f..caed968 100644
--- a/common/src/main/java/com/datatorrent/common/util/SerializableObject.java
+++ b/common/src/main/java/com/datatorrent/common/util/SerializableObject.java
@@ -22,8 +22,8 @@ import java.io.ObjectStreamException;
 import java.io.Serializable;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Modifier;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,37 +58,21 @@ public class SerializableObject implements Serializable
       Constructor<? extends SerializableObject> constructor = this.getClass().getConstructor(this.getClass());
       try {
         constructor.setAccessible(true);
-      }
-      catch (SecurityException ex) {
+      } catch (SecurityException ex) {
         logger.warn("Accessing copy constructor {} failed.", constructor, ex);
       }
       try {
         return constructor.newInstance(this);
-      }
-      catch (InstantiationException ex) {
-        throw new RuntimeException("Instantiation using copy constructor failed!", ex);
-      }
-      catch (IllegalAccessException ex) {
+      } catch (ReflectiveOperationException | IllegalArgumentException ex) {
         throw new RuntimeException("Instantiation using copy constructor failed!", ex);
       }
-      catch (IllegalArgumentException ex) {
-        throw new RuntimeException("Instantiation using copy constructor failed!", ex);
-      }
-      catch (InvocationTargetException ex) {
-        throw new RuntimeException("Instantiation using copy constructor failed!", ex);
-      }
-    }
-    catch (NoSuchMethodException snme) {
+    } catch (NoSuchMethodException snme) {
       logger.debug("No copy constructor detected for class {}, trying default constructor.", this.getClass().getSimpleName());
       try {
         SerializableObject newInstance = this.getClass().newInstance();
         transferStateTo(newInstance);
         return newInstance;
-      }
-      catch (IllegalAccessException ex) {
-        throw new RuntimeException("Deserialization using default constructor failed!", ex);
-      }
-      catch (InstantiationException ex) {
+      } catch (ReflectiveOperationException ex) {
         throw new RuntimeException("Deserialization using default constructor failed!", ex);
       }
     }
@@ -108,27 +92,20 @@ public class SerializableObject implements Serializable
         if (!(Modifier.isFinal(modifiers) && Modifier.isTransient(modifiers) || Modifier.isStatic(modifiers))) {
           try {
             field.setAccessible(true);
-          }
-          catch (SecurityException ex) {
+          } catch (SecurityException ex) {
             logger.warn("Cannot set field {} accessible.", field, ex);
           }
           try {
             field.set(dest, field.get(this));
-          }
-          catch (IllegalArgumentException ex) {
+          } catch (IllegalArgumentException ex) {
             throw new RuntimeException("Getter/Setter argument failed using reflection on " + field, ex);
-          }
-          catch (IllegalAccessException ex) {
+          } catch (IllegalAccessException ex) {
             throw new RuntimeException("Getter/Setter access failed using reflection on " + field, ex);
           }
           if (!field.getType().isPrimitive()) {
             try {
               field.set(this, null);
-            }
-            catch (IllegalArgumentException ex) {
-              logger.warn("Failed to set field {} to null; generally it's harmless, but with reference counted data structure this may be an issue.", field, ex);
-            }
-            catch (IllegalAccessException ex) {
+            } catch (IllegalArgumentException | IllegalAccessException ex) {
               logger.warn("Failed to set field {} to null; generally it's harmless, but with reference counted data structure this may be an issue.", field, ex);
             }
           }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/test/java/com/datatorrent/common/codec/JsonStreamCodecTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/datatorrent/common/codec/JsonStreamCodecTest.java b/common/src/test/java/com/datatorrent/common/codec/JsonStreamCodecTest.java
index d99a186..b31009e 100644
--- a/common/src/test/java/com/datatorrent/common/codec/JsonStreamCodecTest.java
+++ b/common/src/test/java/com/datatorrent/common/codec/JsonStreamCodecTest.java
@@ -18,13 +18,15 @@
  */
 package com.datatorrent.common.codec;
 
-import com.datatorrent.api.StringCodec;
 import java.util.HashMap;
 import java.util.Map;
+
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.datatorrent.api.StringCodec;
+
 public class JsonStreamCodecTest
 {
   static class PojoClass

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/test/java/com/datatorrent/common/partitioner/StatelessPartitionerTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/datatorrent/common/partitioner/StatelessPartitionerTest.java b/common/src/test/java/com/datatorrent/common/partitioner/StatelessPartitionerTest.java
index 25e5fcc..687957c 100644
--- a/common/src/test/java/com/datatorrent/common/partitioner/StatelessPartitionerTest.java
+++ b/common/src/test/java/com/datatorrent/common/partitioner/StatelessPartitionerTest.java
@@ -25,9 +25,13 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
+
 import com.datatorrent.api.Context.OperatorContext;
-import com.datatorrent.api.*;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.DefaultPartition;
+import com.datatorrent.api.Operator;
 import com.datatorrent.api.Operator.InputPort;
+import com.datatorrent.api.Partitioner;
 import com.datatorrent.api.Partitioner.Partition;
 import com.datatorrent.api.StringCodec.Object2String;
 
@@ -97,7 +101,7 @@ public class StatelessPartitionerTest
     Collection<Partition<DummyOperator>> newPartitions = statelessPartitioner.definePartitions(partitions, new PartitioningContextImpl(null, 0));
     Assert.assertEquals("Incorrect number of partitions", 1, newPartitions.size());
 
-    for(Partition<DummyOperator> partition: newPartitions) {
+    for (Partition<DummyOperator> partition : newPartitions) {
       Assert.assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
     }
   }
@@ -115,7 +119,7 @@ public class StatelessPartitionerTest
     Collection<Partition<DummyOperator>> newPartitions = statelessPartitioner.definePartitions(partitions, new PartitioningContextImpl(null, 0));
     Assert.assertEquals("Incorrect number of partitions", 5, newPartitions.size());
 
-    for(Partition<DummyOperator> partition: newPartitions) {
+    for (Partition<DummyOperator> partition : newPartitions) {
       Assert.assertEquals("Incorrect cloned value", 5, partition.getPartitionedInstance().getValue());
     }
   }
@@ -138,7 +142,7 @@ public class StatelessPartitionerTest
     partitions.add(new DefaultPartition<DummyOperator>(dummyOperator));
 
     Collection<Partition<DummyOperator>> newPartitions = statelessPartitioner.definePartitions(partitions,
-      new PartitioningContextImpl(null, 5));
+        new PartitioningContextImpl(null, 5));
     Assert.assertEquals("after partition", 5, newPartitions.size());
   }
 
@@ -155,7 +159,7 @@ public class StatelessPartitionerTest
     }
 
     Collection<Partition<DummyOperator>> newPartitions = statelessPartitioner.definePartitions(partitions,
-      new PartitioningContextImpl(null, 1));
+        new PartitioningContextImpl(null, 1));
     Assert.assertEquals("after partition", 1, newPartitions.size());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/test/java/com/datatorrent/common/util/AsyncFSStorageAgentTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/datatorrent/common/util/AsyncFSStorageAgentTest.java b/common/src/test/java/com/datatorrent/common/util/AsyncFSStorageAgentTest.java
index e2522cb..e644846 100644
--- a/common/src/test/java/com/datatorrent/common/util/AsyncFSStorageAgentTest.java
+++ b/common/src/test/java/com/datatorrent/common/util/AsyncFSStorageAgentTest.java
@@ -22,15 +22,16 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Map;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestWatcher;
 import org.junit.runner.Description;
 
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+
 import com.google.common.collect.Maps;
 
 import com.datatorrent.api.Attribute;
@@ -85,7 +86,7 @@ public class AsyncFSStorageAgentTest
     testMeta.storageAgent.save(data, 1, 1);
     testMeta.storageAgent.copyToHDFS(1, 1);
     @SuppressWarnings("unchecked")
-    Map<Integer, String> decoded = (Map<Integer, String>) testMeta.storageAgent.load(1, 1);
+    Map<Integer, String> decoded = (Map<Integer, String>)testMeta.storageAgent.load(1, 1);
     Assert.assertEquals("dataOf1", data, decoded);
   }
 
@@ -107,10 +108,10 @@ public class AsyncFSStorageAgentTest
     testMeta.storageAgent.save(dataOf2, 2, 1);
     testMeta.storageAgent.copyToHDFS(2, 1);
     @SuppressWarnings("unchecked")
-    Map<Integer, String> decoded1 = (Map<Integer, String>) testMeta.storageAgent.load(1, 1);
+    Map<Integer, String> decoded1 = (Map<Integer, String>)testMeta.storageAgent.load(1, 1);
 
     @SuppressWarnings("unchecked")
-    Map<Integer, String> decoded2 = (Map<Integer, String>) testMeta.storageAgent.load(2, 1);
+    Map<Integer, String> decoded2 = (Map<Integer, String>)testMeta.storageAgent.load(2, 1);
     Assert.assertEquals("data of 1", dataOf1, decoded1);
     Assert.assertEquals("data of 2", dataOf2, decoded2);
   }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/test/java/com/datatorrent/common/util/FSStorageAgentTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/datatorrent/common/util/FSStorageAgentTest.java b/common/src/test/java/com/datatorrent/common/util/FSStorageAgentTest.java
index e9fc1ea..0d6e38b 100644
--- a/common/src/test/java/com/datatorrent/common/util/FSStorageAgentTest.java
+++ b/common/src/test/java/com/datatorrent/common/util/FSStorageAgentTest.java
@@ -22,15 +22,16 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Map;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestWatcher;
 import org.junit.runner.Description;
 
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+
 import com.google.common.collect.Maps;
 
 import com.datatorrent.api.Attribute;
@@ -50,8 +51,7 @@ public class FSStorageAgentTest
       applicationPath = "target/" + description.getClassName() + "/" + description.getMethodName();
       try {
         FileUtils.forceMkdir(new File("target/" + description.getClassName()));
-      }
-      catch (IOException e) {
+      } catch (IOException e) {
         throw new RuntimeException(e);
       }
       storageAgent = new FSStorageAgent(applicationPath, null);
@@ -65,8 +65,7 @@ public class FSStorageAgentTest
     {
       try {
         FileUtils.deleteDirectory(new File("target/" + description.getClassName()));
-      }
-      catch (IOException e) {
+      } catch (IOException e) {
         throw new RuntimeException(e);
       }
     }
@@ -84,7 +83,7 @@ public class FSStorageAgentTest
     data.put(3, "three");
     testMeta.storageAgent.save(data, 1, 1);
     @SuppressWarnings("unchecked")
-    Map<Integer, String> decoded = (Map<Integer, String>) testMeta.storageAgent.load(1, 1);
+    Map<Integer, String> decoded = (Map<Integer, String>)testMeta.storageAgent.load(1, 1);
     Assert.assertEquals("dataOf1", data, decoded);
   }
 
@@ -104,10 +103,10 @@ public class FSStorageAgentTest
     testMeta.storageAgent.save(dataOf1, 1, 1);
     testMeta.storageAgent.save(dataOf2, 2, 1);
     @SuppressWarnings("unchecked")
-    Map<Integer, String> decoded1 = (Map<Integer, String>) testMeta.storageAgent.load(1, 1);
+    Map<Integer, String> decoded1 = (Map<Integer, String>)testMeta.storageAgent.load(1, 1);
 
     @SuppressWarnings("unchecked")
-    Map<Integer, String> decoded2 = (Map<Integer, String>) testMeta.storageAgent.load(2, 1);
+    Map<Integer, String> decoded2 = (Map<Integer, String>)testMeta.storageAgent.load(2, 1);
     Assert.assertEquals("data of 1", dataOf1, decoded1);
     Assert.assertEquals("data of 2", dataOf2, decoded2);
   }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/778436f7/common/src/test/java/com/datatorrent/common/util/SerializableObjectTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/com/datatorrent/common/util/SerializableObjectTest.java b/common/src/test/java/com/datatorrent/common/util/SerializableObjectTest.java
index c44b953..97debe3 100644
--- a/common/src/test/java/com/datatorrent/common/util/SerializableObjectTest.java
+++ b/common/src/test/java/com/datatorrent/common/util/SerializableObjectTest.java
@@ -24,13 +24,14 @@ import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 
 import org.junit.Test;
-import static org.junit.Assert.*;
 
 import com.datatorrent.api.DefaultInputPort;
 import com.datatorrent.api.DefaultOutputPort;
 import com.datatorrent.api.Operator.InputPort;
 import com.datatorrent.api.Operator.OutputPort;
 
+import static org.junit.Assert.assertEquals;
+
 /**
  *
  */
@@ -124,4 +125,4 @@ public class SerializableObjectTest
     assertEquals("Serialized Deserialized Objects", pre, post);
   }
 
-}
\ No newline at end of file
+}


[26/50] incubator-apex-core git commit: APEXCORE-300 Fix checkstyle regular expression APEXCORE-301 Added "io" to imports in checkstyle

Posted by th...@apache.org.
APEXCORE-300 Fix checkstyle regular expression
APEXCORE-301 Added "io" to imports in checkstyle


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

Branch: refs/heads/master
Commit: ffa844732e5cd31710ebdf9d827aef2ace530244
Parents: 3f3afea
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Thu Jan 7 18:43:00 2016 -0800
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Thu Jan 7 18:43:00 2016 -0800

----------------------------------------------------------------------
 apex_checks.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ffa84473/apex_checks.xml
----------------------------------------------------------------------
diff --git a/apex_checks.xml b/apex_checks.xml
index 136a0d0..dd4d035 100644
--- a/apex_checks.xml
+++ b/apex_checks.xml
@@ -35,7 +35,7 @@
   </module>
 
   <module name="RegexpMultiline">
-    <property name="format" value="System\.(out)|(err)\.print(ln)?\("/>
+    <property name="format" value="System\.(out|err)\.print(ln)?\("/>
   </module>
 
   <module name="TreeWalker">
@@ -158,7 +158,7 @@
     </module>
 
     <module name="ImportOrder">
-      <property name="groups" value="/^javax?\./,org,/org\.apache/,com,/com\.datatorrent/,*"/>
+      <property name="groups" value="/^javax?\./,org,/org\.apache/,io,com,/com\.datatorrent/,*"/>
       <property name="ordered" value="true"/>
       <property name="option" value="bottom"/>
       <property name="sortStaticImportsAlphabetically" value="true"/>


[30/50] incubator-apex-core git commit: Merge branch 'APEX-78' of https://github.com/PramodSSImmaneni/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'APEX-78' of https://github.com/PramodSSImmaneni/incubator-apex-core into devel-3


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

Branch: refs/heads/master
Commit: b14583c4d86a2e6f6efec7b484f3804e9fc73f98
Parents: f6e2930 d5e8246
Author: Thomas Weise <th...@datatorrent.com>
Authored: Fri Jan 8 12:43:09 2016 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Fri Jan 8 12:43:09 2016 -0800

----------------------------------------------------------------------
 .../main/java/com/datatorrent/api/Operator.java | 26 ++++++
 .../java/com/datatorrent/stram/engine/Node.java |  4 +
 .../com/datatorrent/stram/CheckpointTest.java   | 85 +++++++++++++++++---
 3 files changed, 104 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[22/50] incubator-apex-core git commit: Merge branch 'APEXCORE-272' of https://github.com/tushargosavi/incubator-apex-core into APEXCORE-272

Posted by th...@apache.org.
Merge branch 'APEXCORE-272' of https://github.com/tushargosavi/incubator-apex-core into APEXCORE-272


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

Branch: refs/heads/master
Commit: 4b2c88444b7d5029bd64ff7f7485af8aeeb97de3
Parents: 07bc969 be075b8
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Dec 31 14:30:17 2015 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Dec 31 14:30:17 2015 -0800

----------------------------------------------------------------------
 .../stram/plan/logical/LogicalPlan.java         |  47 ++++++-
 .../logical/module/TestModuleExpansion.java     | 128 ++++++++++++++++++-
 2 files changed, 169 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[27/50] incubator-apex-core git commit: Merge branch 'APEXCORE-300' into devel-3

Posted by th...@apache.org.
Merge branch 'APEXCORE-300' into devel-3


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

Branch: refs/heads/master
Commit: 7629be8f42b58485c97332087a68e13e3f23cdf8
Parents: 3f3afea ffa8447
Author: Chandni Singh <cs...@apache.org>
Authored: Thu Jan 7 19:35:24 2016 -0800
Committer: Chandni Singh <cs...@apache.org>
Committed: Thu Jan 7 19:35:24 2016 -0800

----------------------------------------------------------------------
 apex_checks.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[32/50] incubator-apex-core git commit: APEXCORE-267 #resolve #comment removed violations from API

Posted by th...@apache.org.
APEXCORE-267 #resolve #comment removed violations from API


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

Branch: refs/heads/master
Commit: 9da2e852f559d4c916d0d25a2c168810d2951eb0
Parents: 7629be8
Author: MalharJenkins <je...@datatorrent.com>
Authored: Fri Jan 8 14:12:52 2016 -0800
Committer: Chandni Singh <cs...@apache.org>
Committed: Fri Jan 8 14:13:21 2016 -0800

----------------------------------------------------------------------
 api/pom.xml                                     |  2 +-
 .../java/com/datatorrent/api/Attribute.java     | 20 +++++++-------------
 .../java/com/datatorrent/api/AutoMetric.java    |  6 +++++-
 .../java/com/datatorrent/api/Component.java     |  2 +-
 .../main/java/com/datatorrent/api/Context.java  |  9 +++++++--
 api/src/main/java/com/datatorrent/api/DAG.java  |  3 ++-
 .../com/datatorrent/api/DefaultInputPort.java   |  3 +--
 .../com/datatorrent/api/DefaultOutputPort.java  |  4 ++--
 .../com/datatorrent/api/DefaultPartition.java   | 19 +++++++++----------
 .../java/com/datatorrent/api/LocalMode.java     |  8 ++++----
 .../java/com/datatorrent/api/Partitioner.java   |  4 ++--
 .../main/java/com/datatorrent/api/Stats.java    |  1 +
 .../java/com/datatorrent/api/StatsListener.java | 13 ++++++++++++-
 .../datatorrent/api/StreamingApplication.java   |  2 +-
 .../java/com/datatorrent/api/StringCodec.java   | 14 ++++++--------
 .../api/annotation/ApplicationAnnotation.java   |  3 ++-
 .../annotation/InputPortFieldAnnotation.java    |  6 +++++-
 .../com/datatorrent/api/annotation/Name.java    |  6 +++++-
 .../annotation/OutputPortFieldAnnotation.java   |  5 ++---
 .../datatorrent/api/annotation/RecordField.java |  6 +++++-
 .../datatorrent/api/annotation/Stateless.java   |  6 +++++-
 .../com/datatorrent/api/AttributeMapTest.java   |  8 ++++----
 .../com/datatorrent/api/Object2StringTest.java  | 13 +++++++------
 23 files changed, 96 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/pom.xml
----------------------------------------------------------------------
diff --git a/api/pom.xml b/api/pom.xml
index f6e1ada..4c8f00b 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -71,7 +71,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>61</maxAllowedViolations>
+          <logViolationsToConsole>true</logViolationsToConsole>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/Attribute.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Attribute.java b/api/src/main/java/com/datatorrent/api/Attribute.java
index 459d719..fe6907a 100644
--- a/api/src/main/java/com/datatorrent/api/Attribute.java
+++ b/api/src/main/java/com/datatorrent/api/Attribute.java
@@ -189,7 +189,7 @@ public class Attribute<T> implements Serializable
       public DefaultAttributeMap clone() throws CloneNotSupportedException
       {
         DefaultAttributeMap clone = (DefaultAttributeMap)super.clone();
-        clone.map = (HashMap< Attribute< ?>, Object>)map.clone();
+        clone.map = (HashMap<Attribute<?>, Object>)map.clone();
         return clone;
       }
 
@@ -255,8 +255,7 @@ public class Attribute<T> implements Serializable
               result.put(attribute, context.getValue(attribute));
             }
           }
-        }
-        catch (Exception ex) {
+        } catch (Exception ex) {
           DTThrowable.rethrow(ex);
         }
         return result;
@@ -298,17 +297,13 @@ public class Attribute<T> implements Serializable
                   Class<?> klass = attribute.defaultValue.getClass();
                   if (klass == String.class) {
                     codec = new String2String();
-                  }
-                  else if (klass == Integer.class) {
+                  } else if (klass == Integer.class) {
                     codec = new Integer2String();
-                  }
-                  else if (klass == Long.class) {
+                  } else if (klass == Long.class) {
                     codec = new Long2String();
-                  }
-                  else if (klass == Boolean.class) {
+                  } else if (klass == Boolean.class) {
                     codec = new Boolean2String();
-                  }
-                  else if (Enum.class.isAssignableFrom(klass)) {
+                  } else if (Enum.class.isAssignableFrom(klass)) {
                     codec = new Enum2String(klass);
                   }
                 }
@@ -322,8 +317,7 @@ public class Attribute<T> implements Serializable
               set.add(attribute);
             }
           }
-        }
-        catch (Exception ex) {
+        } catch (Exception ex) {
           DTThrowable.rethrow(ex);
         }
         map.put(clazz, set);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/AutoMetric.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/AutoMetric.java b/api/src/main/java/com/datatorrent/api/AutoMetric.java
index 74d062a..b487e04 100644
--- a/api/src/main/java/com/datatorrent/api/AutoMetric.java
+++ b/api/src/main/java/com/datatorrent/api/AutoMetric.java
@@ -19,7 +19,11 @@
 package com.datatorrent.api;
 
 import java.io.IOException;
-import java.lang.annotation.*;
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 import java.util.Collection;
 import java.util.Map;
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/Component.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Component.java b/api/src/main/java/com/datatorrent/api/Component.java
index f15d6de..05c1a30 100644
--- a/api/src/main/java/com/datatorrent/api/Component.java
+++ b/api/src/main/java/com/datatorrent/api/Component.java
@@ -65,7 +65,7 @@ public interface Component<CONTEXT extends Context>
    * @param <COMPLEMENT>
    * @since 0.3.2
    */
-  public static abstract class ComponentComplementPair<COMPONENT extends Component<?>, COMPLEMENT>
+  public abstract static class ComponentComplementPair<COMPONENT extends Component<?>, COMPLEMENT>
   {
     public final COMPONENT component;
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/Context.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Context.java b/api/src/main/java/com/datatorrent/api/Context.java
index 6092dce..ceed8a2 100644
--- a/api/src/main/java/com/datatorrent/api/Context.java
+++ b/api/src/main/java/com/datatorrent/api/Context.java
@@ -25,7 +25,12 @@ import java.util.Map;
 
 import com.datatorrent.api.Attribute.AttributeMap;
 import com.datatorrent.api.Operator.ProcessingMode;
-import com.datatorrent.api.StringCodec.*;
+import com.datatorrent.api.StringCodec.Class2String;
+import com.datatorrent.api.StringCodec.Collection2String;
+import com.datatorrent.api.StringCodec.Integer2String;
+import com.datatorrent.api.StringCodec.Map2String;
+import com.datatorrent.api.StringCodec.Object2String;
+import com.datatorrent.api.StringCodec.String2String;
 import com.datatorrent.api.annotation.Stateless;
 
 /**
@@ -292,7 +297,7 @@ public interface Context
      * by this construct is conveyed to tracker application and influences the aggregations done on it by the tracker.
      */
     Attribute<AutoMetric.DimensionsScheme> METRICS_DIMENSIONS_SCHEME = new Attribute<AutoMetric.DimensionsScheme>(new
-      Object2String<AutoMetric.DimensionsScheme>());
+        Object2String<AutoMetric.DimensionsScheme>());
 
     /**
      * Return the operator runtime id.

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/DAG.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/DAG.java b/api/src/main/java/com/datatorrent/api/DAG.java
index 7c793f9..72096e1 100644
--- a/api/src/main/java/com/datatorrent/api/DAG.java
+++ b/api/src/main/java/com/datatorrent/api/DAG.java
@@ -54,7 +54,8 @@ public interface DAG extends DAGContext, Serializable
    * significant performance gains. Optimizations are subject to resource
    * availability.
    */
-  enum Locality {
+  enum Locality
+  {
     /**
      * Adjacent operators should be deployed into the same executing thread,
      * effectively serializing the computation. This setting is beneficial

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/DefaultInputPort.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/DefaultInputPort.java b/api/src/main/java/com/datatorrent/api/DefaultInputPort.java
index 2ed871d..046a35d 100644
--- a/api/src/main/java/com/datatorrent/api/DefaultInputPort.java
+++ b/api/src/main/java/com/datatorrent/api/DefaultInputPort.java
@@ -85,8 +85,7 @@ public abstract class DefaultInputPort<T> implements InputPort<T>, Sink<T>
   {
     try {
       return count;
-    }
-    finally {
+    } finally {
       if (reset) {
         count = 0;
       }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/DefaultOutputPort.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/DefaultOutputPort.java b/api/src/main/java/com/datatorrent/api/DefaultOutputPort.java
index d145657..078a372 100644
--- a/api/src/main/java/com/datatorrent/api/DefaultOutputPort.java
+++ b/api/src/main/java/com/datatorrent/api/DefaultOutputPort.java
@@ -57,9 +57,9 @@ public class DefaultOutputPort<T> implements Operator.OutputPort<T>
    * Called by execution engine to inject sink at deployment time.
    */
   @Override
-  final public void setSink(Sink<Object> s)
+  public final void setSink(Sink<Object> s)
   {
-    this.sink = s == null? Sink.BLACKHOLE: s;
+    this.sink = s == null ? Sink.BLACKHOLE : s;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/DefaultPartition.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/DefaultPartition.java b/api/src/main/java/com/datatorrent/api/DefaultPartition.java
index 222f44e..16b435f 100644
--- a/api/src/main/java/com/datatorrent/api/DefaultPartition.java
+++ b/api/src/main/java/com/datatorrent/api/DefaultPartition.java
@@ -18,18 +18,19 @@
  */
 package com.datatorrent.api;
 
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import com.google.common.collect.Sets;
+
 import com.datatorrent.api.Attribute.AttributeMap.DefaultAttributeMap;
 import com.datatorrent.api.Operator.InputPort;
 import com.datatorrent.api.Partitioner.Partition;
 import com.datatorrent.api.Partitioner.PartitionKeys;
 import com.datatorrent.api.Partitioner.PartitioningContext;
 import com.datatorrent.api.StatsListener.BatchedOperatorStats;
-import com.google.common.collect.Sets;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
 
 /**
  * <p>
@@ -204,8 +205,7 @@ public class DefaultPartition<T> implements Partitioner.Partition<T>
       Partition<?> p;
       if (iterator.hasNext()) {
         p = iterator.next();
-      }
-      else {
+      } else {
         iterator = partitions.iterator();
         p = iterator.next();
       }
@@ -213,8 +213,7 @@ public class DefaultPartition<T> implements Partitioner.Partition<T>
       PartitionKeys pks = p.getPartitionKeys().get(inputPort);
       if (pks == null) {
         p.getPartitionKeys().put(inputPort, new PartitionKeys(partitionMask, Sets.newHashSet(i)));
-      }
-      else {
+      } else {
         pks.partitions.add(i);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/LocalMode.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/LocalMode.java b/api/src/main/java/com/datatorrent/api/LocalMode.java
index 0115f2a..51d3da3 100644
--- a/api/src/main/java/com/datatorrent/api/LocalMode.java
+++ b/api/src/main/java/com/datatorrent/api/LocalMode.java
@@ -37,7 +37,7 @@ public abstract class LocalMode
    *
    * @return
    */
-  abstract public DAG getDAG();
+  public abstract DAG getDAG();
 
   /**
    * <p>
@@ -46,7 +46,7 @@ public abstract class LocalMode
    * @return
    * @throws java.lang.Exception
    */
-  abstract public DAG cloneDAG() throws Exception;
+  public abstract DAG cloneDAG() throws Exception;
 
   /**
    * Build the logical plan through the given streaming application instance and/or from configuration.
@@ -64,7 +64,7 @@ public abstract class LocalMode
    * @throws Exception
    * @since 0.3.5
    */
-  abstract public DAG prepareDAG(StreamingApplication app, Configuration conf) throws Exception;
+  public abstract DAG prepareDAG(StreamingApplication app, Configuration conf) throws Exception;
 
   /**
    * <p>
@@ -72,7 +72,7 @@ public abstract class LocalMode
    *
    * @return
    */
-  abstract public Controller getController();
+  public abstract Controller getController();
 
   public interface Controller
   {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/Partitioner.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Partitioner.java b/api/src/main/java/com/datatorrent/api/Partitioner.java
index 1627c51..d743821 100644
--- a/api/src/main/java/com/datatorrent/api/Partitioner.java
+++ b/api/src/main/java/com/datatorrent/api/Partitioner.java
@@ -67,8 +67,8 @@ public interface Partitioner<T>
   public class PartitionKeys implements java.io.Serializable
   {
     private static final long serialVersionUID = 201312271835L;
-    final public int mask;
-    final public Set<Integer> partitions;
+    public final int mask;
+    public final Set<Integer> partitions;
 
     public PartitionKeys(int mask, Set<Integer> partitions)
     {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/Stats.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Stats.java b/api/src/main/java/com/datatorrent/api/Stats.java
index 3ba1402..0e4377d 100644
--- a/api/src/main/java/com/datatorrent/api/Stats.java
+++ b/api/src/main/java/com/datatorrent/api/Stats.java
@@ -47,6 +47,7 @@ public interface Stats extends Serializable
       return "CheckpointStats{" + "checkpointStartTime=" + checkpointStartTime + ", checkpointTime=" + checkpointTime + '}';
     }
   }
+
   public static class OperatorStats implements Stats
   {
     public long windowId;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/StatsListener.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/StatsListener.java b/api/src/main/java/com/datatorrent/api/StatsListener.java
index 38b93cd..624a095 100644
--- a/api/src/main/java/com/datatorrent/api/StatsListener.java
+++ b/api/src/main/java/com/datatorrent/api/StatsListener.java
@@ -20,7 +20,10 @@ package com.datatorrent.api;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.lang.annotation.*;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 import java.util.List;
 
 import com.datatorrent.api.Stats.OperatorStats;
@@ -78,6 +81,7 @@ public interface StatsListener
      * The Object to identify the response
      */
     public Object getResponseId();
+
     /*
      * The data payload that needs to be sent back
      */
@@ -95,12 +99,19 @@ public interface StatsListener
       * @return
       */
     List<OperatorStats> getLastWindowedStats();
+
     int getOperatorId();
+
     long getCurrentWindowId();
+
     long getTuplesProcessedPSMA();
+
     long getTuplesEmittedPSMA();
+
     double getCpuPercentageMA();
+
     long getLatencyMA();
+
     List<OperatorResponse> getOperatorResponse();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/StreamingApplication.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/StreamingApplication.java b/api/src/main/java/com/datatorrent/api/StreamingApplication.java
index f4ecd4a..cf1d6ec 100644
--- a/api/src/main/java/com/datatorrent/api/StreamingApplication.java
+++ b/api/src/main/java/com/datatorrent/api/StreamingApplication.java
@@ -55,7 +55,7 @@ public interface StreamingApplication
   {
     LOCAL,
     CLUSTER
-  };
+  }
 
   /**
    * <p>populateDAG.</p>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/StringCodec.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/StringCodec.java b/api/src/main/java/com/datatorrent/api/StringCodec.java
index b19d15e..72d5f34 100644
--- a/api/src/main/java/com/datatorrent/api/StringCodec.java
+++ b/api/src/main/java/com/datatorrent/api/StringCodec.java
@@ -22,12 +22,13 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.commons.beanutils.BeanUtils;
 import org.apache.commons.lang.StringUtils;
 
 import com.datatorrent.netlet.util.DTThrowable;
-import java.util.*;
 
 /**
  * This interface is essentially serializer/deserializer interface which works with String as
@@ -166,7 +167,7 @@ public interface StringCodec<T>
 
       try {
         @SuppressWarnings("unchecked")
-        Class<? extends T> clazz = (Class<? extends T>) Thread.currentThread().getContextClassLoader().loadClass(parts[0]);
+        Class<? extends T> clazz = (Class<? extends T>)Thread.currentThread().getContextClassLoader().loadClass(parts[0]);
         if (parts.length == 1) {
           return clazz.newInstance();
         }
@@ -174,8 +175,7 @@ public interface StringCodec<T>
         //String[] properties = parts[1].split(separator, 2);
         if (parts.length == 2) {
           return clazz.getConstructor(String.class).newInstance(parts[1]);
-        }
-        else {
+        } else {
           T object = clazz.getConstructor(String.class).newInstance(parts[1]);
           HashMap<String, String> hashMap = new HashMap<String, String>();
           for (int i = 2; i < parts.length; i++) {
@@ -185,8 +185,7 @@ public interface StringCodec<T>
           BeanUtils.populate(object, hashMap);
           return object;
         }
-      }
-      catch (Throwable cause) {
+      } catch (Throwable cause) {
         DTThrowable.rethrow(cause);
       }
 
@@ -354,8 +353,7 @@ public interface StringCodec<T>
         @SuppressWarnings({"rawtypes", "unchecked"})
         Class<? extends T> clazz = (Class)Thread.currentThread().getContextClassLoader().loadClass(string);
         return clazz;
-      }
-      catch (Throwable cause) {
+      } catch (Throwable cause) {
         DTThrowable.rethrow(cause);
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/annotation/ApplicationAnnotation.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/annotation/ApplicationAnnotation.java b/api/src/main/java/com/datatorrent/api/annotation/ApplicationAnnotation.java
index b568ade..f81c15e 100644
--- a/api/src/main/java/com/datatorrent/api/annotation/ApplicationAnnotation.java
+++ b/api/src/main/java/com/datatorrent/api/annotation/ApplicationAnnotation.java
@@ -33,7 +33,8 @@ import java.lang.annotation.Target;
 @Documented
 @Target(ElementType.TYPE)
 @Retention(RetentionPolicy.RUNTIME)
-public @interface ApplicationAnnotation {
+public @interface ApplicationAnnotation
+{
   
   /**
    * <p>Compile time application alias.</p> 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/annotation/InputPortFieldAnnotation.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/annotation/InputPortFieldAnnotation.java b/api/src/main/java/com/datatorrent/api/annotation/InputPortFieldAnnotation.java
index c247c82..76fe497 100644
--- a/api/src/main/java/com/datatorrent/api/annotation/InputPortFieldAnnotation.java
+++ b/api/src/main/java/com/datatorrent/api/annotation/InputPortFieldAnnotation.java
@@ -18,7 +18,11 @@
  */
 package com.datatorrent.api.annotation;
 
-import java.lang.annotation.*;
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
 /**
  * Annotation for input ports on streaming operators.<p>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/annotation/Name.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/annotation/Name.java b/api/src/main/java/com/datatorrent/api/annotation/Name.java
index 2d03a92..57ead03 100644
--- a/api/src/main/java/com/datatorrent/api/annotation/Name.java
+++ b/api/src/main/java/com/datatorrent/api/annotation/Name.java
@@ -18,7 +18,11 @@
  */
 package com.datatorrent.api.annotation;
 
-import java.lang.annotation.*;
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
 /**
  * <p>Name class.</p>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/annotation/OutputPortFieldAnnotation.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/annotation/OutputPortFieldAnnotation.java b/api/src/main/java/com/datatorrent/api/annotation/OutputPortFieldAnnotation.java
index 46aee64..62b43c6 100644
--- a/api/src/main/java/com/datatorrent/api/annotation/OutputPortFieldAnnotation.java
+++ b/api/src/main/java/com/datatorrent/api/annotation/OutputPortFieldAnnotation.java
@@ -24,8 +24,6 @@ import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
-import com.datatorrent.api.Context;
-
 /**
  *
  * Annotation for output ports on streaming operators.<p>
@@ -35,7 +33,8 @@ import com.datatorrent.api.Context;
 @Documented
 @Target(ElementType.FIELD)
 @Retention(RetentionPolicy.RUNTIME)
-public @interface OutputPortFieldAnnotation {
+public @interface OutputPortFieldAnnotation
+{
   /**
    * <p>optional.</p>
    */

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/annotation/RecordField.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/annotation/RecordField.java b/api/src/main/java/com/datatorrent/api/annotation/RecordField.java
index 8a95d26..1675c54 100644
--- a/api/src/main/java/com/datatorrent/api/annotation/RecordField.java
+++ b/api/src/main/java/com/datatorrent/api/annotation/RecordField.java
@@ -18,7 +18,11 @@
  */
 package com.datatorrent.api.annotation;
 
-import java.lang.annotation.*;
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
 /**
  * <p>RecordField class.</p>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/main/java/com/datatorrent/api/annotation/Stateless.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/annotation/Stateless.java b/api/src/main/java/com/datatorrent/api/annotation/Stateless.java
index 2d79bc8..150c35c 100644
--- a/api/src/main/java/com/datatorrent/api/annotation/Stateless.java
+++ b/api/src/main/java/com/datatorrent/api/annotation/Stateless.java
@@ -18,7 +18,11 @@
  */
 package com.datatorrent.api.annotation;
 
-import java.lang.annotation.*;
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
 /**
  * When a class or interface is annotated with this annotation, the object which are

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/test/java/com/datatorrent/api/AttributeMapTest.java
----------------------------------------------------------------------
diff --git a/api/src/test/java/com/datatorrent/api/AttributeMapTest.java b/api/src/test/java/com/datatorrent/api/AttributeMapTest.java
index 97703f7..fcb1809 100644
--- a/api/src/test/java/com/datatorrent/api/AttributeMapTest.java
+++ b/api/src/test/java/com/datatorrent/api/AttributeMapTest.java
@@ -23,9 +23,9 @@ import java.util.Set;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
 
-import com.datatorrent.api.Attribute;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
 
 /**
  *
@@ -47,7 +47,7 @@ public class AttributeMapTest
   {
     hello,
     howdy
-  };
+  }
 
   interface iface
   {
@@ -63,4 +63,4 @@ public class AttributeMapTest
   }
 
   private static final Logger logger = LoggerFactory.getLogger(AttributeMapTest.class);
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9da2e852/api/src/test/java/com/datatorrent/api/Object2StringTest.java
----------------------------------------------------------------------
diff --git a/api/src/test/java/com/datatorrent/api/Object2StringTest.java b/api/src/test/java/com/datatorrent/api/Object2StringTest.java
index 0307588..e42a462 100644
--- a/api/src/test/java/com/datatorrent/api/Object2StringTest.java
+++ b/api/src/test/java/com/datatorrent/api/Object2StringTest.java
@@ -22,7 +22,9 @@ import org.hamcrest.BaseMatcher;
 import org.hamcrest.Description;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
 
 /**
  * This tests the Object2String codec
@@ -100,7 +102,7 @@ public class Object2StringTest
         return false;
       }
 
-      TestBean testBean = (TestBean) o;
+      TestBean testBean = (TestBean)o;
 
       if (intVal != testBean.intVal) {
         return false;
@@ -120,7 +122,7 @@ public class Object2StringTest
     {
       int result = intVal;
       result = 31 * result + (stringVal != null ? stringVal.hashCode() : 0);
-      result = 31 * result + (int) (longVal ^ (longVal >>> 32));
+      result = 31 * result + (int)(longVal ^ (longVal >>> 32));
       return result;
     }
   }
@@ -222,8 +224,7 @@ public class Object2StringTest
     try {
       bean2String.fromString(bean);
       assertFalse(true);
-    }
-    catch (RuntimeException e) {
+    } catch (RuntimeException e) {
       if (e.getCause() instanceof ClassNotFoundException) {
         String expRegex = "java.lang.ClassNotFoundException: com.datatorrent.api.Object2StringTest\\$TestBean1";
         assertThat("exception message", e.getMessage(), RegexMatcher.matches(expRegex));
@@ -257,7 +258,7 @@ public class Object2StringTest
     @Override
     public boolean matches(Object o)
     {
-      return ((String) o).matches(regex);
+      return ((String)o).matches(regex);
 
     }
 


[49/50] incubator-apex-core git commit: Correct JIRA instructions in README.md

Posted by th...@apache.org.
Correct JIRA instructions in README.md


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

Branch: refs/heads/master
Commit: f32fa1b862ada333c31746ea3cb5510b198591d3
Parents: d58de0c
Author: Thomas Weise <th...@datatorrent.com>
Authored: Mon Jan 25 17:46:55 2016 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Mon Jan 25 17:46:55 2016 -0800

----------------------------------------------------------------------
 README.md | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f32fa1b8/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 65bd2f7..4b22f71 100644
--- a/README.md
+++ b/README.md
@@ -34,15 +34,15 @@ https://www.datatorrent.com/download/
 
 ##Issue tracking
 
-(Note that we will be moving to the Apache JIRA system soon.)
+[JIRA](https://issues.apache.org/jira/browse/APEXCORE) issue tracking system is used for this project.
+You can submit new issues and track the progress of existing issues at https://issues.apache.org/jira/browse/APEXCORE
 
-[Apex JIRA](https://issues.apache.org/jira/browse/APEXCORE) issue tracking system is used for this project.
-You can submit new issues and track the progress of existing issues at https://malhar.atlassian.net/projects/APEX.
+Please include the JIRA ticket number into the commit messages. It will automatically add the commit message to the JIRA ticket(s) and help link the commit with the issue(s) being tracked for easy reference.
+An example commit might look like this:
 
-When working with JIRA to submit pull requests, please use [smart commits](https://confluence.atlassian.com/display/AOD/Processing+JIRA+issues+with+commit+messages) feature by specifying APEXCORE-XXXX in the commit messages.
-It helps us link commits with issues being tracked for easy reference.  And example commit might look like this:
+    git commit -am "APEXCORE-1234 Task completed ahead of schedule"
 
-    git commit -am "APEXCORE-1234 #comment Task completed ahead of schedule #resolve"
+JIRA tickets should be resolved and fix version field set by the committer merging the pull request.
 
 ##License
 


[02/50] incubator-apex-core git commit: All annotations should be on a separate line. Upgraded checkstyle to 6.13

Posted by th...@apache.org.
All annotations should be on a separate line.
Upgraded checkstyle to 6.13


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

Branch: refs/heads/master
Commit: 564888e5930e0cddf0b5037696fe2ba0cdd0e515
Parents: 38a6f4e
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Tue Dec 1 17:04:04 2015 -0800
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Tue Dec 1 17:04:04 2015 -0800

----------------------------------------------------------------------
 apex_checks.xml | 2 +-
 pom.xml         | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/564888e5/apex_checks.xml
----------------------------------------------------------------------
diff --git a/apex_checks.xml b/apex_checks.xml
index 7474e67..60d99fe 100644
--- a/apex_checks.xml
+++ b/apex_checks.xml
@@ -152,7 +152,7 @@
 
     <module name="AnnotationLocation">
       <property name="allowSamelineMultipleAnnotations" value="false"/>
-      <property name="allowSamelineSingleParameterlessAnnotation" value="true"/>
+      <property name="allowSamelineSingleParameterlessAnnotation" value="false"/>
       <property name="allowSamelineParameterizedAnnotation" value="false"/>
     </module>
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/564888e5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a875581..4cd1a5d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -306,7 +306,7 @@
             <dependency>
               <groupId>com.puppycrawl.tools</groupId>
               <artifactId>checkstyle</artifactId>
-              <version>6.12.1</version>
+              <version>6.13</version>
             </dependency>
           </dependencies>
           <executions>


[18/50] incubator-apex-core git commit: APEXCORE-276 made METRICS_TRANSPORT pluggable

Posted by th...@apache.org.
APEXCORE-276 made METRICS_TRANSPORT pluggable


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

Branch: refs/heads/master
Commit: 845796abc5f5d1f92f43b40e105f1b18cb2d8632
Parents: 56b55fe
Author: David Yan <da...@datatorrent.com>
Authored: Tue Dec 29 13:46:58 2015 -0800
Committer: David Yan <da...@datatorrent.com>
Committed: Wed Dec 30 16:39:40 2015 -0800

----------------------------------------------------------------------
 .../java/com/datatorrent/api/AutoMetric.java    | 19 ++++++
 .../main/java/com/datatorrent/api/Context.java  |  6 +-
 .../metric/AutoMetricBuiltInTransport.java      | 67 +++++++++++++++++++
 .../stram/PubSubWebSocketMetricTransport.java   | 68 ++++++++++++++++++++
 .../stram/StreamingAppMasterService.java        |  2 +-
 .../stram/WebsocketAppDataPusher.java           | 65 -------------------
 .../datatorrent/stram/api/AppDataPusher.java    | 34 ----------
 .../stram/appdata/AppDataPushAgent.java         | 34 +++++-----
 .../stram/StreamingContainerManagerTest.java    | 62 ++++++++++++++----
 9 files changed, 225 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/845796ab/api/src/main/java/com/datatorrent/api/AutoMetric.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/AutoMetric.java b/api/src/main/java/com/datatorrent/api/AutoMetric.java
index 1c1fb25..74d062a 100644
--- a/api/src/main/java/com/datatorrent/api/AutoMetric.java
+++ b/api/src/main/java/com/datatorrent/api/AutoMetric.java
@@ -18,6 +18,7 @@
  */
 package com.datatorrent.api;
 
+import java.io.IOException;
 import java.lang.annotation.*;
 import java.util.Collection;
 import java.util.Map;
@@ -106,4 +107,22 @@ public @interface AutoMetric
     String[] getDimensionAggregationsFor(String logicalMetricName);
   }
 
+  /**
+   * Interface of transport for STRAM to push metrics data
+   */
+  interface Transport
+  {
+    /**
+     * Pushes the metrics data (in JSON) to the transport.
+     *
+     * @param jsonData The metric data in JSON to be pushed to this transport
+     */
+    void push(String jsonData) throws IOException;
+
+    /**
+     * Returns the number of milliseconds for resending the metric schema. The schema will need to be resent for
+     * unreliable transport. Return 0 if the schema does not need to be resent.
+     */
+    long getSchemaResendInterval();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/845796ab/api/src/main/java/com/datatorrent/api/Context.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Context.java b/api/src/main/java/com/datatorrent/api/Context.java
index 2054920..6092dce 100644
--- a/api/src/main/java/com/datatorrent/api/Context.java
+++ b/api/src/main/java/com/datatorrent/api/Context.java
@@ -334,10 +334,10 @@ public interface Context
      */
     Attribute<String> APPLICATION_DATA_LINK = new Attribute<String>(new String2String());
     /**
-     * Transport to push the stats and the metrics, "builtin:{topic}" if STRAM should push the data directly
-     * using websocket with the given topic
+     * Transport to push the stats and the metrics.
+     * If using the built-in transport, please use an AutoMetricBuiltInTransport object
      */
-    Attribute<String> METRICS_TRANSPORT = new Attribute<String>(new String2String());
+    Attribute<AutoMetric.Transport> METRICS_TRANSPORT = new Attribute<>(new Object2String<AutoMetric.Transport>());
     /**
      * Application instance identifier. An application with the same name can run in multiple instances, each with a
      * unique identifier. The identifier is set by the client that submits the application and can be used in operators

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/845796ab/common/src/main/java/com/datatorrent/common/metric/AutoMetricBuiltInTransport.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/metric/AutoMetricBuiltInTransport.java b/common/src/main/java/com/datatorrent/common/metric/AutoMetricBuiltInTransport.java
new file mode 100644
index 0000000..ee9cbdd
--- /dev/null
+++ b/common/src/main/java/com/datatorrent/common/metric/AutoMetricBuiltInTransport.java
@@ -0,0 +1,67 @@
+/**
+ * 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 com.datatorrent.common.metric;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import com.datatorrent.api.AutoMetric;
+
+/**
+ * AutoMetricBuiltinTransport. This will be replaced by the internal websocket pubsub transport
+ * provided here: {@link com.datatorrent.stram.PubSubWebSocketMetricTransport}.
+ */
+public class AutoMetricBuiltInTransport implements AutoMetric.Transport, Serializable
+{
+  private final String topic;
+  private final long schemaResendInterval;
+  private static final long DEFAULT_SCHEMA_RESEND_INTERVAL = 10000;
+
+  public AutoMetricBuiltInTransport(String topic)
+  {
+    this.topic = topic;
+    this.schemaResendInterval = DEFAULT_SCHEMA_RESEND_INTERVAL;
+  }
+
+  public AutoMetricBuiltInTransport(String topic, long schemaResendInterval)
+  {
+    this.topic = topic;
+    this.schemaResendInterval = schemaResendInterval;
+  }
+
+  @Override
+  public void push(String jsonData) throws IOException
+  {
+    throw new UnsupportedOperationException("This class is a placeholder and is supposed to replaced by internal " +
+        "implementation.");
+  }
+
+  @Override
+  public long getSchemaResendInterval()
+  {
+    return schemaResendInterval;
+  }
+
+  public String getTopic()
+  {
+    return topic;
+  }
+
+  private static final long serialVersionUID = 201512301009L;
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/845796ab/engine/src/main/java/com/datatorrent/stram/PubSubWebSocketMetricTransport.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/PubSubWebSocketMetricTransport.java b/engine/src/main/java/com/datatorrent/stram/PubSubWebSocketMetricTransport.java
new file mode 100644
index 0000000..85b8006
--- /dev/null
+++ b/engine/src/main/java/com/datatorrent/stram/PubSubWebSocketMetricTransport.java
@@ -0,0 +1,68 @@
+/**
+ * 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 com.datatorrent.stram;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datatorrent.api.AutoMetric;
+import com.datatorrent.common.util.PubSubWebSocketClient;
+
+/**
+ * <p>PubSubWebSocketMetricTransport class.</p>
+ *
+ * @since 3.0.0
+ */
+public class PubSubWebSocketMetricTransport implements AutoMetric.Transport, Serializable
+{
+  private final String topic;
+  private final long schemaResendInterval;
+  protected PubSubWebSocketClient client;
+
+  public PubSubWebSocketMetricTransport(PubSubWebSocketClient wsClient, String topic, long schemaResendInterval)
+  {
+    client = wsClient;
+    this.topic = topic;
+    this.schemaResendInterval = schemaResendInterval;
+  }
+
+  @Override
+  public void push(String msg) throws IOException
+  {
+    try {
+      client.publish(topic, new JSONObject(msg));
+    } catch (JSONException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  @Override
+  public long getSchemaResendInterval()
+  {
+    return schemaResendInterval;
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(PubSubWebSocketMetricTransport.class);
+  private static final long serialVersionUID = 201512301008L;
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/845796ab/engine/src/main/java/com/datatorrent/stram/StreamingAppMasterService.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/StreamingAppMasterService.java b/engine/src/main/java/com/datatorrent/stram/StreamingAppMasterService.java
index 5d84e10..db8c255 100644
--- a/engine/src/main/java/com/datatorrent/stram/StreamingAppMasterService.java
+++ b/engine/src/main/java/com/datatorrent/stram/StreamingAppMasterService.java
@@ -536,7 +536,7 @@ public class StreamingAppMasterService extends CompositeService
     this.heartbeatListener = new StreamingContainerParent(this.getClass().getName(), dnmgr, delegationTokenManager, rpcListenerCount);
     addService(heartbeatListener);
 
-    String appDataPushTransport = dag.getValue(LogicalPlan.METRICS_TRANSPORT);
+    AutoMetric.Transport appDataPushTransport = dag.getValue(LogicalPlan.METRICS_TRANSPORT);
     if (appDataPushTransport != null) {
       this.appDataPushAgent = new AppDataPushAgent(dnmgr, appContext);
       addService(this.appDataPushAgent);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/845796ab/engine/src/main/java/com/datatorrent/stram/WebsocketAppDataPusher.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/WebsocketAppDataPusher.java b/engine/src/main/java/com/datatorrent/stram/WebsocketAppDataPusher.java
deleted file mode 100644
index f052ded..0000000
--- a/engine/src/main/java/com/datatorrent/stram/WebsocketAppDataPusher.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * 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 com.datatorrent.stram;
-
-import com.datatorrent.common.util.PubSubWebSocketClient;
-import com.datatorrent.stram.api.AppDataPusher;
-import java.io.IOException;
-import org.codehaus.jettison.json.JSONObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * <p>WebsocketAppDataPusher class.</p>
- *
- * @since 3.0.0
- */
-public class WebsocketAppDataPusher implements AppDataPusher
-{
-  private final String topic;
-  private long resendSchemaInterval = 10000; // 10 seconds
-  protected PubSubWebSocketClient client;
-
-
-  public WebsocketAppDataPusher(PubSubWebSocketClient wsClient, String topic)
-  {
-    client = wsClient;
-    this.topic = topic;
-  }
-
-  public void setResendSchemaInterval(long resendSchemaInterval)
-  {
-    this.resendSchemaInterval = resendSchemaInterval;
-  }
-
-  @Override
-  public void push(JSONObject msg) throws IOException
-  {
-    client.publish(topic, msg);
-  }
-
-  @Override
-  public long getResendSchemaInterval()
-  {
-    return resendSchemaInterval;
-  }
-
-  private static final Logger LOG = LoggerFactory.getLogger(WebsocketAppDataPusher.class);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/845796ab/engine/src/main/java/com/datatorrent/stram/api/AppDataPusher.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/api/AppDataPusher.java b/engine/src/main/java/com/datatorrent/stram/api/AppDataPusher.java
deleted file mode 100644
index 25cc2cf..0000000
--- a/engine/src/main/java/com/datatorrent/stram/api/AppDataPusher.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * 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 com.datatorrent.stram.api;
-
-import java.io.IOException;
-import org.codehaus.jettison.json.JSONObject;
-
-/**
- * <p>AppDataPusher interface.</p>
- *
- * @since 3.0.0
- */
-public interface AppDataPusher
-{
-  public void push(JSONObject appData) throws IOException;
-
-  public long getResendSchemaInterval();
-}

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/845796ab/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java b/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
index 9389e3c..7fd4c3c 100644
--- a/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
+++ b/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
@@ -41,11 +41,12 @@ import com.google.common.collect.Maps;
 import com.datatorrent.api.AutoMetric;
 import com.datatorrent.api.Context.DAGContext;
 
+import com.datatorrent.api.StringCodec;
+import com.datatorrent.common.metric.AutoMetricBuiltInTransport;
 import com.datatorrent.common.util.Pair;
+import com.datatorrent.stram.PubSubWebSocketMetricTransport;
 import com.datatorrent.stram.StramAppContext;
 import com.datatorrent.stram.StreamingContainerManager;
-import com.datatorrent.stram.WebsocketAppDataPusher;
-import com.datatorrent.stram.api.AppDataPusher;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
 import com.datatorrent.stram.webapp.LogicalOperatorInfo;
 
@@ -64,7 +65,7 @@ public class AppDataPushAgent extends AbstractService
   private final StreamingContainerManager dnmgr;
   private final StramAppContext appContext;
   private final AppDataPushThread appDataPushThread = new AppDataPushThread();
-  private AppDataPusher appDataPusher;
+  private AutoMetric.Transport metricsTransport;
   private final Map<Class<?>, List<Field>> cacheFields = new HashMap<Class<?>, List<Field>>();
   private final Map<Class<?>, Map<String, Method>> cacheGetMethods = new HashMap<Class<?>, Map<String, Method>>();
 
@@ -81,7 +82,7 @@ public class AppDataPushAgent extends AbstractService
   @Override
   protected void serviceStop() throws Exception
   {
-    if (appDataPusher != null) {
+    if (metricsTransport != null) {
       appDataPushThread.interrupt();
       try {
         appDataPushThread.join();
@@ -95,7 +96,7 @@ public class AppDataPushAgent extends AbstractService
   @Override
   protected void serviceStart() throws Exception
   {
-    if (appDataPusher != null) {
+    if (metricsTransport != null) {
       appDataPushThread.start();
     }
     super.serviceStart();
@@ -110,15 +111,12 @@ public class AppDataPushAgent extends AbstractService
 
   public void init()
   {
-    String appDataPushTransport = dnmgr.getLogicalPlan().getValue(DAGContext.METRICS_TRANSPORT);
-    if (appDataPushTransport.startsWith(APP_DATA_PUSH_TRANSPORT_BUILTIN_VALUE + ":")) {
-      String topic = appDataPushTransport.substring(APP_DATA_PUSH_TRANSPORT_BUILTIN_VALUE.length() + 1);
-      appDataPusher = new WebsocketAppDataPusher(dnmgr.getWsClient(), topic);
-      LOG.info("App Data Push Transport set up for {}", appDataPushTransport);
-    } else {
-      // TBD add kakfa
-      LOG.error("App Data Push Transport not recognized: {}", appDataPushTransport);
+    metricsTransport = dnmgr.getLogicalPlan().getValue(DAGContext.METRICS_TRANSPORT);
+    if (metricsTransport instanceof AutoMetricBuiltInTransport) {
+      AutoMetricBuiltInTransport transport = (AutoMetricBuiltInTransport)metricsTransport;
+      metricsTransport = new PubSubWebSocketMetricTransport(dnmgr.getWsClient(), transport.getTopic(), transport.getSchemaResendInterval());
     }
+    LOG.info("Metrics Transport set up for {}", metricsTransport);
   }
 
   private JSONObject getPushData()
@@ -132,7 +130,6 @@ public class AppDataPushAgent extends AbstractService
       json.put("appUser", appContext.getUser());
       List<LogicalOperatorInfo> logicalOperatorInfoList = dnmgr.getLogicalOperatorInfoList();
       JSONObject logicalOperators = new JSONObject();
-      long resendSchemaInterval = appDataPusher.getResendSchemaInterval();
       for (LogicalOperatorInfo logicalOperator : logicalOperatorInfoList) {
         JSONObject logicalOperatorJson = extractFields(logicalOperator);
         JSONArray metricsList = new JSONArray();
@@ -144,8 +141,9 @@ public class AppDataPushAgent extends AbstractService
             // metric name, aggregated value
             Map<String, Object> aggregates = metrics.second;
             long now = System.currentTimeMillis();
-            if (!operatorsSchemaLastSentTime.containsKey(logicalOperator.name)
-                    || operatorsSchemaLastSentTime.get(logicalOperator.name) < now - resendSchemaInterval) {
+            if (!operatorsSchemaLastSentTime.containsKey(logicalOperator.name) ||
+                (metricsTransport.getSchemaResendInterval() > 0 &&
+                    operatorsSchemaLastSentTime.get(logicalOperator.name) < now - metricsTransport.getSchemaResendInterval())) {
               try {
                 pushMetricsSchema(dnmgr.getLogicalPlan().getOperatorMeta(logicalOperator.name), aggregates);
                 operatorsSchemaLastSentTime.put(logicalOperator.name, now);
@@ -287,12 +285,12 @@ public class AppDataPushAgent extends AbstractService
       schema = getMetricsSchemaData(operatorMeta, aggregates);
       operatorSchemas.put(operatorMeta.getName(), schema);
     }
-    appDataPusher.push(schema);
+    metricsTransport.push(schema.toString());
   }
 
   public void pushData() throws IOException
   {
-    appDataPusher.push(getPushData());
+    metricsTransport.push(getPushData().toString());
   }
 
   public class AppDataPushThread extends Thread

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/845796ab/engine/src/test/java/com/datatorrent/stram/StreamingContainerManagerTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/StreamingContainerManagerTest.java b/engine/src/test/java/com/datatorrent/stram/StreamingContainerManagerTest.java
index f1d6ec4..8fc957b 100644
--- a/engine/src/test/java/com/datatorrent/stram/StreamingContainerManagerTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/StreamingContainerManagerTest.java
@@ -19,6 +19,7 @@
 package com.datatorrent.stram;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.net.InetSocketAddress;
 import java.util.*;
 import java.util.concurrent.Future;
@@ -33,6 +34,8 @@ import org.junit.Test;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+
+import com.datatorrent.api.AutoMetric;
 import com.datatorrent.api.Context.OperatorContext;
 import com.datatorrent.api.Context.PortContext;
 import com.datatorrent.api.DAG.Locality;
@@ -40,6 +43,7 @@ import com.datatorrent.api.Stats.OperatorStats;
 import com.datatorrent.api.Stats.OperatorStats.PortStats;
 import com.datatorrent.api.StatsListener;
 import com.datatorrent.api.annotation.Stateless;
+import com.datatorrent.common.metric.AutoMetricBuiltInTransport;
 import com.datatorrent.common.partitioner.StatelessPartitioner;
 import com.datatorrent.common.util.AsyncFSStorageAgent;
 import com.datatorrent.common.util.FSStorageAgent;
@@ -75,7 +79,6 @@ import com.datatorrent.stram.support.StramTestSupport.TestMeta;
 import com.datatorrent.stram.tuple.Tuple;
 
 import org.apache.commons.lang.StringUtils;
-import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.eclipse.jetty.websocket.WebSocket;
 
@@ -904,7 +907,7 @@ public class StreamingContainerManagerTest
   public void testAppDataPush() throws Exception
   {
     final String topic = "xyz";
-    final List<JSONObject> messages = new ArrayList<JSONObject>();
+    final List<String> messages = new ArrayList<>();
     EmbeddedWebSocketServer server = new EmbeddedWebSocketServer(0);
     server.setWebSocket(new WebSocket.OnTextMessage()
     {
@@ -912,11 +915,7 @@ public class StreamingContainerManagerTest
       @Override
       public void onMessage(String data)
       {
-        try {
-          messages.add(new JSONObject(data));
-        } catch (JSONException ex) {
-          throw new RuntimeException(ex);
-        }
+        messages.add(data);
       }
 
       @Override
@@ -935,10 +934,9 @@ public class StreamingContainerManagerTest
       TestGeneratorInputOperator o1 = dag.addOperator("o1", TestGeneratorInputOperator.class);
       GenericTestOperator o2 = dag.addOperator("o2", GenericTestOperator.class);
       dag.addStream("o1.outport", o1.outport, o2.inport1);
-      dag.setAttribute(LogicalPlan.METRICS_TRANSPORT, "builtin:" + topic);
+      dag.setAttribute(LogicalPlan.METRICS_TRANSPORT, new AutoMetricBuiltInTransport(topic));
       dag.setAttribute(LogicalPlan.GATEWAY_CONNECT_ADDRESS, "localhost:" + port);
       StramLocalCluster lc = new StramLocalCluster(dag);
-      //lc.runAsync();
       StreamingContainerManager dnmgr = lc.dnmgr;
       StramAppContext appContext = new StramTestSupport.TestAppContext();
 
@@ -948,8 +946,7 @@ public class StreamingContainerManagerTest
       Thread.sleep(1000);
       Assert.assertTrue(messages.size() > 0);
       pushAgent.close();
-      JSONObject message = messages.get(0);
-      System.out.println("Got this message: " + message.toString(2));
+      JSONObject message = new JSONObject(messages.get(0));
       Assert.assertEquals(topic, message.getString("topic"));
       Assert.assertEquals("publish", message.getString("type"));
       JSONObject data = message.getJSONObject("data");
@@ -970,4 +967,47 @@ public class StreamingContainerManagerTest
       server.stop();
     }
   }
+
+  public static class TestMetricTransport implements AutoMetric.Transport, Serializable
+  {
+    private String prefix;
+    private static List<String> messages = new ArrayList<>();
+
+    public TestMetricTransport(String prefix)
+    {
+      this.prefix = prefix;
+    }
+
+    @Override
+    public void push(String jsonData) throws IOException
+    {
+      messages.add(prefix + ":" + jsonData);
+    }
+
+    @Override
+    public long getSchemaResendInterval()
+    {
+      return 0;
+    }
+  }
+
+  @Test
+  public void testCustomMetricsTransport() throws Exception
+  {
+    TestGeneratorInputOperator o1 = dag.addOperator("o1", TestGeneratorInputOperator.class);
+    GenericTestOperator o2 = dag.addOperator("o2", GenericTestOperator.class);
+    dag.addStream("o1.outport", o1.outport, o2.inport1);
+    dag.setAttribute(LogicalPlan.METRICS_TRANSPORT, new TestMetricTransport("xyz"));
+    StramLocalCluster lc = new StramLocalCluster(dag);
+    StreamingContainerManager dnmgr = lc.dnmgr;
+    StramAppContext appContext = new StramTestSupport.TestAppContext();
+
+    AppDataPushAgent pushAgent = new AppDataPushAgent(dnmgr, appContext);
+    pushAgent.init();
+    pushAgent.pushData();
+    Assert.assertTrue(TestMetricTransport.messages.size() > 0);
+    pushAgent.close();
+    String msg = TestMetricTransport.messages.get(0);
+    Assert.assertTrue(msg.startsWith("xyz:"));
+  }
 }


[47/50] incubator-apex-core git commit: APEXCORE-60 Isolate test checkpoint directories.

Posted by th...@apache.org.
APEXCORE-60 Isolate test checkpoint directories.


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

Branch: refs/heads/master
Commit: 50b0855d840eb99d03cf8877e4b208e9a0ec85f1
Parents: 5371bc7
Author: Thomas Weise <th...@datatorrent.com>
Authored: Sat Jan 23 01:00:29 2016 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sat Jan 23 01:00:29 2016 -0800

----------------------------------------------------------------------
 .../stram/plan/logical/DelayOperatorTest.java            | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/50b0855d/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
index cb4222a..0bcc791 100644
--- a/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
@@ -72,6 +72,9 @@ import static org.junit.Assert.fail;
  */
 public class DelayOperatorTest
 {
+  @Rule
+  public TestMeta testMeta = new TestMeta();
+
   private static Lock sequential = new ReentrantLock();
 
   @Before
@@ -324,7 +327,7 @@ public class DelayOperatorTest
   @Test
   public void testFibonacciRecovery1() throws Exception
   {
-    LogicalPlan dag = new LogicalPlan();
+    LogicalPlan dag = StramTestSupport.createDAG(testMeta);
 
     TestGeneratorInputOperator dummyInput = dag.addOperator("DUMMY", TestGeneratorInputOperator.class);
     FailableFibonacciOperator fib = dag.addOperator("FIB", FailableFibonacciOperator.class);
@@ -355,11 +358,10 @@ public class DelayOperatorTest
         Arrays.copyOfRange(new TreeSet<>(FibonacciOperator.results).toArray(), 0, 20));
   }
 
-  @Ignore // Out of sequence BEGIN_WINDOW tuple on Travis. Will tackle in the next version
   @Test
   public void testFibonacciRecovery2() throws Exception
   {
-    LogicalPlan dag = new LogicalPlan();
+    LogicalPlan dag = StramTestSupport.createDAG(testMeta);
 
     TestGeneratorInputOperator dummyInput = dag.addOperator("DUMMY", TestGeneratorInputOperator.class);
     FibonacciOperator fib = dag.addOperator("FIB", FibonacciOperator.class);
@@ -391,9 +393,6 @@ public class DelayOperatorTest
         Arrays.copyOfRange(new TreeSet<>(FibonacciOperator.results).toArray(), 0, 20));
   }
 
-  @Rule
-  public TestMeta testMeta = new TestMeta();
-
   @Test
   public void testCheckpointUpdate()
   {


[19/50] incubator-apex-core git commit: Merge branch 'APEXCORE-276' of https://github.com/davidyan74/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'APEXCORE-276' of https://github.com/davidyan74/incubator-apex-core into devel-3


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

Branch: refs/heads/master
Commit: 3f76dcbfa04d74ca34da0d360b533f8f3d4536f3
Parents: 5879124 845796a
Author: Thomas Weise <th...@datatorrent.com>
Authored: Wed Dec 30 16:51:07 2015 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Wed Dec 30 16:51:07 2015 -0800

----------------------------------------------------------------------
 .../java/com/datatorrent/api/AutoMetric.java    | 19 ++++++
 .../main/java/com/datatorrent/api/Context.java  |  6 +-
 .../metric/AutoMetricBuiltInTransport.java      | 67 +++++++++++++++++++
 .../stram/PubSubWebSocketMetricTransport.java   | 68 ++++++++++++++++++++
 .../stram/StreamingAppMasterService.java        |  2 +-
 .../stram/WebsocketAppDataPusher.java           | 65 -------------------
 .../datatorrent/stram/api/AppDataPusher.java    | 34 ----------
 .../stram/appdata/AppDataPushAgent.java         | 34 +++++-----
 .../stram/StreamingContainerManagerTest.java    | 62 ++++++++++++++----
 9 files changed, 225 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/3f76dcbf/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
----------------------------------------------------------------------
diff --cc engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
index b47dc41,7fd4c3c..9297f89
--- a/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
+++ b/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
@@@ -41,13 -41,13 +41,14 @@@ import com.google.common.collect.Maps
  import com.datatorrent.api.AutoMetric;
  import com.datatorrent.api.Context.DAGContext;
  
+ import com.datatorrent.api.StringCodec;
+ import com.datatorrent.common.metric.AutoMetricBuiltInTransport;
  import com.datatorrent.common.util.Pair;
+ import com.datatorrent.stram.PubSubWebSocketMetricTransport;
  import com.datatorrent.stram.StramAppContext;
  import com.datatorrent.stram.StreamingContainerManager;
- import com.datatorrent.stram.WebsocketAppDataPusher;
- import com.datatorrent.stram.api.AppDataPusher;
  import com.datatorrent.stram.plan.logical.LogicalPlan;
 +import com.datatorrent.stram.plan.logical.MetricAggregatorMeta;
  import com.datatorrent.stram.webapp.LogicalOperatorInfo;
  
  /**


[11/50] incubator-apex-core git commit: APEXCORE-105 Introduce module meta Inject properties through xml file on modules.

Posted by th...@apache.org.
APEXCORE-105 Introduce module meta
Inject properties through xml file on modules.


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

Branch: refs/heads/master
Commit: b0360d45a361e462124db9c3000977987ca830e6
Parents: 2f1e1df
Author: Tushar R. Gosavi <tu...@apache.org>
Authored: Tue Oct 6 13:48:53 2015 +0530
Committer: Tushar R. Gosavi <tu...@apache.org>
Committed: Tue Dec 22 01:16:51 2015 +0530

----------------------------------------------------------------------
 api/src/main/java/com/datatorrent/api/DAG.java  |   2 +
 .../stram/codec/LogicalPlanSerializer.java      |   2 +-
 .../stram/plan/logical/LogicalPlan.java         | 120 ++++++++++-
 .../plan/logical/LogicalPlanConfiguration.java  |  99 +++++++--
 .../stram/webapp/StramWebServices.java          |   2 +-
 .../logical/module/TestModuleProperties.java    |  58 +++++
 .../stram/plan/logical/module/TestModules.java  | 216 +++++++++++++++++++
 7 files changed, 472 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b0360d45/api/src/main/java/com/datatorrent/api/DAG.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/DAG.java b/api/src/main/java/com/datatorrent/api/DAG.java
index abe2954..1dce402 100644
--- a/api/src/main/java/com/datatorrent/api/DAG.java
+++ b/api/src/main/java/com/datatorrent/api/DAG.java
@@ -164,6 +164,8 @@ public interface DAG extends DAGContext, Serializable
   {
     String getName();
 
+    Module getModule();
+
     InputPortMeta getMeta(Operator.InputPort<?> port);
 
     OutputPortMeta getMeta(Operator.OutputPort<?> port);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b0360d45/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java b/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java
index 9e5ac04..90dd2b5 100644
--- a/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java
+++ b/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java
@@ -212,7 +212,7 @@ public class LogicalPlanSerializer extends JsonSerializer<LogicalPlan>
       String operatorKey = LogicalPlanConfiguration.OPERATOR_PREFIX + operatorMeta.getName();
       Operator operator = operatorMeta.getOperator();
       props.setProperty(operatorKey + "." + LogicalPlanConfiguration.OPERATOR_CLASSNAME, operator.getClass().getName());
-      BeanMap operatorProperties = LogicalPlanConfiguration.getOperatorProperties(operator);
+      BeanMap operatorProperties = LogicalPlanConfiguration.getObjectProperties(operator);
       @SuppressWarnings("rawtypes")
       Iterator entryIterator = operatorProperties.entryIterator();
       while (entryIterator.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b0360d45/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index cca45d8..53e81bc 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 import com.datatorrent.api.*;
@@ -48,7 +49,6 @@ import com.datatorrent.api.Operator.Unifier;
 import com.datatorrent.api.annotation.InputPortFieldAnnotation;
 import com.datatorrent.api.annotation.OperatorAnnotation;
 import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
-
 import com.datatorrent.common.experimental.AppData;
 import com.datatorrent.common.metric.MetricsAggregator;
 import com.datatorrent.common.metric.SingleMetricAggregator;
@@ -1067,7 +1067,7 @@ public class LogicalPlan implements Serializable, DAG
   public <T extends Operator> T addOperator(String name, T operator)
   {
     if (operators.containsKey(name)) {
-      if (operators.get(name) == (Object)operator) {
+      if (operators.get(name).operator == operator) {
         return operator;
       }
       throw new IllegalArgumentException("duplicate operator id: " + operators.get(name));
@@ -1079,16 +1079,109 @@ public class LogicalPlan implements Serializable, DAG
     return operator;
   }
 
-  @Override
-  public <T extends Module> T addModule(String name, Class<T> moduleClass)
+  public final class ModuleMeta implements DAG.ModuleMeta, Serializable
   {
-    throw new UnsupportedOperationException("Modules are not supported");
+    private final LinkedHashMap<InputPortMeta, StreamMeta> inputStreams = new LinkedHashMap<>();
+    private final LinkedHashMap<OutputPortMeta, StreamMeta> outputStreams = new LinkedHashMap<>();
+    private final Attribute.AttributeMap attributes;
+    @SuppressWarnings("unused")
+    private final int id;
+    @NotNull
+    private final String name;
+    private transient Integer nindex; // for cycle detection
+    private transient Integer lowlink; // for cycle detection
+    private transient Module module;
+
+    public ModuleMeta(String name, Module module)
+    {
+      this(name, module, new DefaultAttributeMap());
+    }
+
+    public ModuleMeta(String name, Module module, DefaultAttributeMap attributeMap)
+    {
+      LOG.debug("Initializing {} as {}", name, module.getClass().getName());
+      this.name = name;
+      this.module = module;
+      this.id = logicalOperatorSequencer.decrementAndGet();
+      this.attributes = attributeMap;
+    }
+
+    @Override
+    public String getName()
+    {
+      return name;
+    }
+
+    @Override
+    public Module getModule()
+    {
+      return module;
+    }
+
+    @Override
+    public DAG.InputPortMeta getMeta(InputPort<?> port)
+    {
+      return null;
+    }
+
+    @Override
+    public DAG.OutputPortMeta getMeta(OutputPort<?> port)
+    {
+      return null;
+    }
+
+    @Override
+    public Attribute.AttributeMap getAttributes()
+    {
+      return null;
+    }
+
+    @Override
+    public <T> T getValue(Attribute<T> key)
+    {
+      return null;
+    }
+
+    @Override
+    public void setCounters(Object counters)
+    {
+
+    }
+
+    @Override
+    public void sendMetrics(Collection<String> metricNames)
+    {
+
+    }
   }
 
+  public transient Map<String, ModuleMeta> modules = Maps.newHashMap();
+
   @Override
   public <T extends Module> T addModule(String name, T module)
   {
-    throw new UnsupportedOperationException("Modules are not supported");
+    if (modules.containsKey(name)) {
+      if (modules.get(name).module == module) {
+        return module;
+      }
+      throw new IllegalArgumentException("duplicate module is: " + modules.get(name));
+    }
+    ModuleMeta meta = new ModuleMeta(name, module);
+    modules.put(name, meta);
+    return module;
+  }
+
+  @Override
+  public <T extends Module> T addModule(String name, Class<T> clazz)
+  {
+    T instance;
+    try {
+      instance = clazz.newInstance();
+    } catch (Exception ex) {
+      throw new IllegalArgumentException(ex);
+    }
+    addModule(name, instance);
+    return instance;
   }
 
   public void removeOperator(Operator operator)
@@ -1231,6 +1324,10 @@ public class LogicalPlan implements Serializable, DAG
     return Collections.unmodifiableCollection(this.operators.values());
   }
 
+  public Collection<ModuleMeta> getAllModules() {
+    return Collections.unmodifiableCollection(this.modules.values());
+  }
+
   public Collection<StreamMeta> getAllStreams()
   {
     return Collections.unmodifiableCollection(this.streams.values());
@@ -1242,10 +1339,9 @@ public class LogicalPlan implements Serializable, DAG
     return this.operators.get(operatorName);
   }
 
-  @Override
   public ModuleMeta getModuleMeta(String moduleName)
   {
-    throw new UnsupportedOperationException("Modules are not supported");
+    return null;
   }
 
   @Override
@@ -1260,10 +1356,14 @@ public class LogicalPlan implements Serializable, DAG
     throw new IllegalArgumentException("Operator not associated with the DAG: " + operator);
   }
 
-  @Override
   public ModuleMeta getMeta(Module module)
   {
-    throw new UnsupportedOperationException("Modules are not supported");
+    for (ModuleMeta m : getAllModules()) {
+      if (m.module == module) {
+        return m;
+      }
+    }
+    throw new IllegalArgumentException("Module not associated with the DAG: " + module);
   }
 
   public int getMaxContainerCount()

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b0360d45/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
index 60bbdbe..9bbe85c 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
@@ -18,7 +18,6 @@
  */
 package com.datatorrent.stram.plan.logical;
 
-
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -32,7 +31,6 @@ import java.lang.reflect.Type;
 import java.util.*;
 import java.util.Map.Entry;
 
-
 import javax.validation.ValidationException;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -47,6 +45,7 @@ import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.commons.beanutils.BeanMap;
 import org.apache.commons.beanutils.BeanUtils;
 import org.apache.commons.collections.CollectionUtils;
@@ -61,10 +60,10 @@ import com.datatorrent.api.Context.DAGContext;
 import com.datatorrent.api.Context.OperatorContext;
 import com.datatorrent.api.Context.PortContext;
 import com.datatorrent.api.annotation.ApplicationAnnotation;
-
 import com.datatorrent.stram.StramUtils;
 import com.datatorrent.stram.client.StramClientUtils;
 import com.datatorrent.stram.plan.logical.LogicalPlan.InputPortMeta;
+import com.datatorrent.stram.plan.logical.LogicalPlan.ModuleMeta;
 import com.datatorrent.stram.plan.logical.LogicalPlan.OperatorMeta;
 import com.datatorrent.stram.plan.logical.LogicalPlan.OutputPortMeta;
 import com.datatorrent.stram.plan.logical.LogicalPlan.StreamMeta;
@@ -2114,6 +2113,10 @@ public class LogicalPlanConfiguration {
     if (dag.getAttributes().get(Context.DAGContext.APPLICATION_NAME) == null) {
       dag.setAttribute(Context.DAGContext.APPLICATION_NAME, appName);
     }
+
+    // Expand the modules within the dag recursively
+    setModuleProperties(dag, appName);
+
     // inject external operator configuration
     setOperatorConfiguration(dag, appConfs, appName);
     setStreamConfiguration(dag, appConfs, appName);
@@ -2138,7 +2141,7 @@ public class LogicalPlanConfiguration {
   public Map<String, String> getProperties(OperatorMeta ow, String appName) {
     List<AppConf> appConfs = stramConf.getMatchingChildConf(appName, StramElement.APPLICATION);
     List<OperatorConf> opConfs = getMatchingChildConf(appConfs, ow.getName(), StramElement.OPERATOR);
-    return getProperties(ow, opConfs, appName);
+    return getProperties(getPropertyArgs(ow), opConfs, appName);
   }
 
   private Map<String,String> getApplicationProperties(List<AppConf> appConfs){
@@ -2153,17 +2156,17 @@ public class LogicalPlanConfiguration {
   /**
    * Get the configuration opProps for the given operator.
    * These can be operator specific settings or settings from matching templates.
-   * @param ow
+   * @param pa
    * @param opConfs
    * @param appName
    */
-  private Map<String, String> getProperties(OperatorMeta ow, List<OperatorConf> opConfs, String appName)
+  private Map<String, String> getProperties(PropertyArgs pa, List<OperatorConf> opConfs, String appName)
   {
     Map<String, String> opProps = Maps.newHashMap();
     Map<String, TemplateConf> templates = stramConf.getChildren(StramElement.TEMPLATE);
     // list of all templates that match operator, ordered by priority
     if (!templates.isEmpty()) {
-      TreeMap<Integer, TemplateConf> matchingTemplates = getMatchingTemplates(ow, appName, templates);
+      TreeMap<Integer, TemplateConf> matchingTemplates = getMatchingTemplates(pa, appName, templates);
       if (matchingTemplates != null && !matchingTemplates.isEmpty()) {
         // combined map of prioritized template settings
         for (TemplateConf t : matchingTemplates.descendingMap().values()) {
@@ -2197,23 +2200,46 @@ public class LogicalPlanConfiguration {
     return refTemplates;
   }
 
+  private static class PropertyArgs
+  {
+    String name;
+    String className;
+
+    public PropertyArgs(String name, String className)
+    {
+      this.name = name;
+      this.className = className;
+    }
+  }
+
+  private PropertyArgs getPropertyArgs(OperatorMeta om)
+  {
+    return new PropertyArgs(om.getName(), om.getOperator().getClass().getName());
+  }
+
+  private PropertyArgs getPropertyArgs(ModuleMeta mm)
+  {
+    return new PropertyArgs(mm.getName(), mm.getModule().getClass().getName());
+  }
+
   /**
    * Produce the collections of templates that apply for the given id.
-   * @param ow
+   * @param pa
    * @param appName
    * @param templates
    * @return TreeMap<Integer, TemplateConf>
    */
-  private TreeMap<Integer, TemplateConf> getMatchingTemplates(OperatorMeta ow, String appName, Map<String, TemplateConf> templates) {
+  private TreeMap<Integer, TemplateConf> getMatchingTemplates(PropertyArgs pa, String appName, Map<String, TemplateConf> templates)
+  {
     TreeMap<Integer, TemplateConf> tm = Maps.newTreeMap();
     for (TemplateConf t : templates.values()) {
-      if ((t.idRegExp != null && ow.getName().matches(t.idRegExp))) {
+      if ((t.idRegExp != null && pa.name.matches(t.idRegExp))) {
         tm.put(1, t);
       } else if (appName != null && t.appNameRegExp != null
           && appName.matches(t.appNameRegExp)) {
         tm.put(2, t);
       } else if (t.classNameRegExp != null
-          && ow.getOperator().getClass().getName().matches(t.classNameRegExp)) {
+          && pa.className.matches(t.classNameRegExp)) {
         tm.put(3, t);
       }
     }
@@ -2238,6 +2264,26 @@ public class LogicalPlanConfiguration {
     }
   }
 
+  /**
+   * Generic helper function to inject properties on the object.
+   *
+   * @param obj
+   * @param properties
+   * @param <T>
+   * @return
+   */
+  public static <T> T setObjectProperties(T obj, Map<String, String> properties)
+  {
+    try {
+      BeanUtils.populate(obj, properties);
+      return obj;
+    } catch (IllegalAccessException e) {
+      throw new IllegalArgumentException("Error setting operator properties", e);
+    } catch (InvocationTargetException e) {
+      throw new IllegalArgumentException("Error setting operator properties", e);
+    }
+  }
+
   public static StreamingApplication setApplicationProperties(StreamingApplication application, Map<String, String> properties)
   {
     try {
@@ -2249,9 +2295,9 @@ public class LogicalPlanConfiguration {
     }
   }
 
-  public static BeanMap getOperatorProperties(Operator operator)
+  public static BeanMap getObjectProperties(Object obj)
   {
-    return new BeanMap(operator);
+    return new BeanMap(obj);
   }
 
   /**
@@ -2266,12 +2312,26 @@ public class LogicalPlanConfiguration {
     List<AppConf> appConfs = stramConf.getMatchingChildConf(applicationName, StramElement.APPLICATION);
     for (OperatorMeta ow : dag.getAllOperators()) {
       List<OperatorConf> opConfs = getMatchingChildConf(appConfs, ow.getName(), StramElement.OPERATOR);
-      Map<String, String> opProps = getProperties(ow, opConfs, applicationName);
+      Map<String, String> opProps = getProperties(getPropertyArgs(ow), opConfs, applicationName);
       setOperatorProperties(ow.getOperator(), opProps);
     }
   }
 
   /**
+   * Set any properties from configuration on the modules in the DAG. This
+   * method may throw unchecked exception if the configuration contains
+   * properties that are invalid for a module.
+   *
+   * @param dag
+   * @param applicationName
+   */
+  public void setModuleProperties(LogicalPlan dag, String applicationName)
+  {
+    List<AppConf> appConfs = stramConf.getMatchingChildConf(applicationName, StramElement.APPLICATION);
+    setModuleConfiguration(dag, appConfs, applicationName);
+  }
+
+  /**
    * Set the application configuration.
    * @param dag
    * @param appName
@@ -2298,7 +2358,7 @@ public class LogicalPlanConfiguration {
       // Set the operator attributes
       setAttributes(opConfs, ow.getAttributes());
       // Set the operator opProps
-      Map<String, String> opProps = getProperties(ow, opConfs, appName);
+      Map<String, String> opProps = getProperties(getPropertyArgs(ow), opConfs, appName);
       setOperatorProperties(ow.getOperator(), opProps);
 
       // Set the port attributes
@@ -2327,6 +2387,15 @@ public class LogicalPlanConfiguration {
     }
   }
 
+  private void setModuleConfiguration(final LogicalPlan dag, List<AppConf> appConfs, String appName)
+  {
+    for (final ModuleMeta mw : dag.getAllModules()) {
+      List<OperatorConf> opConfs = getMatchingChildConf(appConfs, mw.getName(), StramElement.OPERATOR);
+      Map<String, String> opProps = getProperties(getPropertyArgs(mw), opConfs, appName);
+      setObjectProperties(mw.getModule(), opProps);
+    }
+  }
+
   private void setStreamConfiguration(LogicalPlan dag, List<AppConf> appConfs, String appAlias) {
     for (StreamMeta sm : dag.getAllStreams()) {
       List<StreamConf> smConfs = getMatchingChildConf(appConfs, sm.getName(), StramElement.STREAM);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b0360d45/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java b/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java
index de085cd..6fdba00 100644
--- a/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java
+++ b/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java
@@ -715,7 +715,7 @@ public class StramWebServices
       throw new NotFoundException();
     }
 
-    BeanMap operatorProperties = LogicalPlanConfiguration.getOperatorProperties(logicalOperator.getOperator());
+    BeanMap operatorProperties = LogicalPlanConfiguration.getObjectProperties(logicalOperator.getOperator());
     Map<String, Object> m = new HashMap<String, Object>();
     @SuppressWarnings("rawtypes")
     Iterator entryIterator = operatorProperties.entryIterator();

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b0360d45/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleProperties.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleProperties.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleProperties.java
new file mode 100644
index 0000000..7951e26
--- /dev/null
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleProperties.java
@@ -0,0 +1,58 @@
+/**
+ * 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 com.datatorrent.stram.plan.logical.module;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.stram.plan.logical.LogicalPlan;
+import com.datatorrent.stram.plan.logical.LogicalPlanConfiguration;
+
+public class TestModuleProperties
+{
+  @Test
+  public void testModuleProperties()
+  {
+    Configuration conf = new Configuration(false);
+    conf.set(StreamingApplication.DT_PREFIX + "operator.o1.prop.myStringProperty", "myStringPropertyValue");
+    conf.set(StreamingApplication.DT_PREFIX + "operator.o2.prop.stringArrayField", "a,b,c");
+    conf.set(StreamingApplication.DT_PREFIX + "operator.o2.prop.mapProperty.key1", "key1Val");
+    conf.set(StreamingApplication.DT_PREFIX + "operator.o2.prop.mapProperty(key1.dot)", "key1dotVal");
+    conf.set(StreamingApplication.DT_PREFIX + "operator.o2.prop.mapProperty(key2.dot)", "key2dotVal");
+
+    LogicalPlan dag = new LogicalPlan();
+    TestModules.GenericModule o1 = dag.addModule("o1", new TestModules.GenericModule());
+    TestModules.ValidationTestModule o2 = dag.addModule("o2", new TestModules.ValidationTestModule());
+
+    LogicalPlanConfiguration pb = new LogicalPlanConfiguration(conf);
+
+    pb.setModuleProperties(dag, "testSetOperatorProperties");
+    Assert.assertEquals("o1.myStringProperty", "myStringPropertyValue", o1.getMyStringProperty());
+    Assert.assertArrayEquals("o2.stringArrayField", new String[]{"a", "b", "c"}, o2.getStringArrayField());
+
+    Assert.assertEquals("o2.mapProperty.key1", "key1Val", o2.getMapProperty().get("key1"));
+    Assert.assertEquals("o2.mapProperty(key1.dot)", "key1dotVal", o2.getMapProperty().get("key1.dot"));
+    Assert.assertEquals("o2.mapProperty(key2.dot)", "key2dotVal", o2.getMapProperty().get("key2.dot"));
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b0360d45/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModules.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModules.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModules.java
new file mode 100644
index 0000000..8fad613
--- /dev/null
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModules.java
@@ -0,0 +1,216 @@
+/**
+ * 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 com.datatorrent.stram.plan.logical.module;
+
+import java.util.Map;
+
+import javax.validation.Valid;
+import javax.validation.constraints.AssertTrue;
+import javax.validation.constraints.Min;
+import javax.validation.constraints.NotNull;
+import javax.validation.constraints.Pattern;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.collect.Maps;
+
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.Module;
+import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
+import com.datatorrent.stram.engine.GenericOperatorProperty;
+
+public class TestModules
+{
+
+  public static class GenericModule implements Module
+  {
+    private static final Logger LOG = LoggerFactory.getLogger(TestModules.class);
+
+    public volatile Object inport1Tuple = null;
+
+    @OutputPortFieldAnnotation(optional = true)
+    public final transient DefaultOutputPort<Object> outport1 = new DefaultOutputPort<Object>();
+
+    @OutputPortFieldAnnotation(optional = true)
+    public final transient DefaultOutputPort<Object> outport2 = new DefaultOutputPort<Object>();
+
+    private String emitFormat;
+
+    public boolean booleanProperty;
+
+    private String myStringProperty;
+
+    private transient GenericOperatorProperty genericOperatorProperty = new GenericOperatorProperty("test");
+
+    public String getMyStringProperty()
+    {
+      return myStringProperty;
+    }
+
+    public void setMyStringProperty(String myStringProperty)
+    {
+      this.myStringProperty = myStringProperty;
+    }
+
+    public boolean isBooleanProperty()
+    {
+      return booleanProperty;
+    }
+
+    public void setBooleanProperty(boolean booleanProperty)
+    {
+      this.booleanProperty = booleanProperty;
+    }
+
+    public String propertySetterOnly;
+
+    /**
+     * setter w/o getter defined
+     *
+     * @param v
+     */
+    public void setStringPropertySetterOnly(String v)
+    {
+      this.propertySetterOnly = v;
+    }
+
+    public String getEmitFormat()
+    {
+      return emitFormat;
+    }
+
+    public void setEmitFormat(String emitFormat)
+    {
+      this.emitFormat = emitFormat;
+    }
+
+    public GenericOperatorProperty getGenericOperatorProperty()
+    {
+      return genericOperatorProperty;
+    }
+
+    public void setGenericOperatorProperty(GenericOperatorProperty genericOperatorProperty)
+    {
+      this.genericOperatorProperty = genericOperatorProperty;
+    }
+
+    @Override
+    public void populateDAG(DAG dag, Configuration conf)
+    {
+      LOG.debug("populateDAG of module called");
+    }
+  }
+
+  public static class ValidationTestModule implements Module
+  {
+    @NotNull
+    @Pattern(regexp = ".*malhar.*", message = "Value has to contain 'malhar'!")
+    private String stringField1;
+
+    @Min(2)
+    private int intField1;
+
+    @AssertTrue(message = "stringField1 should end with intField1")
+    private boolean isValidConfiguration()
+    {
+      return stringField1.endsWith(String.valueOf(intField1));
+    }
+
+    private String getterProperty2 = "";
+
+    @NotNull
+    public String getProperty2()
+    {
+      return getterProperty2;
+    }
+
+    public void setProperty2(String s)
+    {
+      // annotations need to be on the getter
+      getterProperty2 = s;
+    }
+
+    private String[] stringArrayField;
+
+    public String[] getStringArrayField()
+    {
+      return stringArrayField;
+    }
+
+    public void setStringArrayField(String[] stringArrayField)
+    {
+      this.stringArrayField = stringArrayField;
+    }
+
+    public class Nested
+    {
+      @NotNull
+      private String property = "";
+
+      public String getProperty()
+      {
+        return property;
+      }
+
+      public void setProperty(String property)
+      {
+        this.property = property;
+      }
+
+    }
+
+    @Valid
+    private final Nested nestedBean = new Nested();
+
+    private String stringProperty2;
+
+    public String getStringProperty2()
+    {
+      return stringProperty2;
+    }
+
+    public void setStringProperty2(String stringProperty2)
+    {
+      this.stringProperty2 = stringProperty2;
+    }
+
+    private Map<String, String> mapProperty = Maps.newHashMap();
+
+    public Map<String, String> getMapProperty()
+    {
+      return mapProperty;
+    }
+
+    public void setMapProperty(Map<String, String> mapProperty)
+    {
+      this.mapProperty = mapProperty;
+    }
+
+    @Override
+    public void populateDAG(DAG dag, Configuration conf)
+    {
+
+    }
+  }
+
+}


[37/50] incubator-apex-core git commit: APEXCORE-305 - Enable checkstyle violations logging to console during maven build

Posted by th...@apache.org.
APEXCORE-305 - Enable checkstyle violations logging to console during maven build


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

Branch: refs/heads/master
Commit: 59770d0647bedbe81e794e3f3fd476e581c4339c
Parents: 3b6c252
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Mon Jan 11 15:04:46 2016 -0800
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Mon Jan 11 15:14:01 2016 -0800

----------------------------------------------------------------------
 api/pom.xml          | 3 ---
 bufferserver/pom.xml | 4 +---
 common/pom.xml       | 3 ---
 engine/pom.xml       | 5 +++++
 pom.xml              | 6 ------
 5 files changed, 6 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/59770d06/api/pom.xml
----------------------------------------------------------------------
diff --git a/api/pom.xml b/api/pom.xml
index 4c8f00b..d5ea932 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -70,9 +70,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
-        <configuration>
-          <logViolationsToConsole>true</logViolationsToConsole>
-        </configuration>
       </plugin>
     </plugins>
   </build>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/59770d06/bufferserver/pom.xml
----------------------------------------------------------------------
diff --git a/bufferserver/pom.xml b/bufferserver/pom.xml
index efc5b5e..aa79c0e 100644
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@ -48,10 +48,8 @@
       </plugin>
       -->
       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
-        <configuration>
-          <consoleOutput>true</consoleOutput>
-        </configuration>
       </plugin>
     </plugins>
   </build>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/59770d06/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 5c2c98f..a749cea 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -55,9 +55,6 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
-        <configuration>
-          <logViolationsToConsole>true</logViolationsToConsole>
-        </configuration>
       </plugin>
     </plugins>
   </build>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/59770d06/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 3e43184..2b70aa6 100644
--- a/engine/pom.xml
+++ b/engine/pom.xml
@@ -33,6 +33,10 @@
 
   <name>Apache Apex (incubating) Stream Processing Engine</name>
 
+  <properties>
+    <checkstyle.console>false</checkstyle.console>
+  </properties>
+
   <build>
     <finalName>${project.artifactId}</finalName>
     <plugins>
@@ -146,6 +150,7 @@
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
           <maxAllowedViolations>3224</maxAllowedViolations>
+          <logViolationsToConsole>${checkstyle.console}</logViolationsToConsole>
         </configuration>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/59770d06/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1972d37..164645c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -64,7 +64,6 @@
     <!-- do not change jetty version as later versions have problems with DefaultServlet -->
     <jetty.version>8.1.10.v20130312</jetty.version>
     <license.skip>true</license.skip>
-    <checkstyle.console>false</checkstyle.console>
     <postNoticeText>The initial developer of the original code is&#xA;DataTorrent, Inc. (http://www.datatorrent.com)&#xA;Copyright (c) 2012 - 2015. All Rights Reserved.</postNoticeText>
   </properties>
 
@@ -316,16 +315,11 @@
               <goals>
                 <goal>check</goal>
               </goals>
-              <configuration>
-                <failOnViolation>true</failOnViolation>
-                <logViolationsToConsole>${checkstyle.console}</logViolationsToConsole>
-              </configuration>
             </execution>
           </executions>
           <configuration>
             <configLocation>apex_checks.xml</configLocation>
             <suppressionsLocation>checkstyle-suppressions.xml</suppressionsLocation>
-            <suppressionsFileExpression>checkstyle.suppressions.file</suppressionsFileExpression>
             <includeTestSourceDirectory>true</includeTestSourceDirectory>
           </configuration>
         </plugin>


[36/50] incubator-apex-core git commit: Merge branch 'APEXCORE-267' of https://github.com/chandnisingh/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'APEXCORE-267' of https://github.com/chandnisingh/incubator-apex-core into devel-3


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

Branch: refs/heads/master
Commit: 3b6c252fe2a1855a6044b122a69fbb6f9376a1fe
Parents: aded30c 9da2e85
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Mon Jan 11 08:35:15 2016 -0800
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Mon Jan 11 08:35:15 2016 -0800

----------------------------------------------------------------------
 api/pom.xml                                     |  2 +-
 .../java/com/datatorrent/api/Attribute.java     | 20 +++++++-------------
 .../java/com/datatorrent/api/AutoMetric.java    |  6 +++++-
 .../java/com/datatorrent/api/Component.java     |  2 +-
 .../main/java/com/datatorrent/api/Context.java  |  9 +++++++--
 api/src/main/java/com/datatorrent/api/DAG.java  |  3 ++-
 .../com/datatorrent/api/DefaultInputPort.java   |  3 +--
 .../com/datatorrent/api/DefaultOutputPort.java  |  4 ++--
 .../com/datatorrent/api/DefaultPartition.java   | 19 +++++++++----------
 .../java/com/datatorrent/api/LocalMode.java     |  8 ++++----
 .../java/com/datatorrent/api/Partitioner.java   |  4 ++--
 .../main/java/com/datatorrent/api/Stats.java    |  1 +
 .../java/com/datatorrent/api/StatsListener.java | 13 ++++++++++++-
 .../datatorrent/api/StreamingApplication.java   |  2 +-
 .../java/com/datatorrent/api/StringCodec.java   | 14 ++++++--------
 .../api/annotation/ApplicationAnnotation.java   |  3 ++-
 .../annotation/InputPortFieldAnnotation.java    |  6 +++++-
 .../com/datatorrent/api/annotation/Name.java    |  6 +++++-
 .../annotation/OutputPortFieldAnnotation.java   |  5 ++---
 .../datatorrent/api/annotation/RecordField.java |  6 +++++-
 .../datatorrent/api/annotation/Stateless.java   |  6 +++++-
 .../com/datatorrent/api/AttributeMapTest.java   |  8 ++++----
 .../com/datatorrent/api/Object2StringTest.java  | 13 +++++++------
 23 files changed, 96 insertions(+), 67 deletions(-)
----------------------------------------------------------------------



[12/50] incubator-apex-core git commit: APEXCORE-104 Added flattening of module into parent DAG

Posted by th...@apache.org.
APEXCORE-104 Added flattening of module into parent DAG


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

Branch: refs/heads/master
Commit: 14a09bb51cb30585e7979ec022ec762ac2ba91e5
Parents: b0360d4
Author: chinmaykolhatkar <ch...@datatorrent.com>
Authored: Wed Oct 7 15:06:36 2015 +0530
Committer: chinmaykolhatkar <ch...@datatorrent.com>
Committed: Tue Dec 22 01:42:43 2015 +0530

----------------------------------------------------------------------
 api/src/main/java/com/datatorrent/api/DAG.java  |   4 -
 .../stram/plan/logical/LogicalPlan.java         | 169 +++++++++++++++----
 .../plan/logical/LogicalPlanConfiguration.java  |   8 +
 3 files changed, 146 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/14a09bb5/api/src/main/java/com/datatorrent/api/DAG.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/DAG.java b/api/src/main/java/com/datatorrent/api/DAG.java
index 1dce402..7c793f9 100644
--- a/api/src/main/java/com/datatorrent/api/DAG.java
+++ b/api/src/main/java/com/datatorrent/api/DAG.java
@@ -165,10 +165,6 @@ public interface DAG extends DAGContext, Serializable
     String getName();
 
     Module getModule();
-
-    InputPortMeta getMeta(Operator.InputPort<?> port);
-
-    OutputPortMeta getMeta(Operator.OutputPort<?> port);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/14a09bb5/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index 53e81bc..5a3e167 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -80,6 +80,7 @@ public class LogicalPlan implements Serializable, DAG
   public static final String SER_FILE_NAME = "dt-conf.ser";
   public static final String LAUNCH_CONFIG_FILE_NAME = "dt-launch-config.xml";
   private static final transient AtomicInteger logicalOperatorSequencer = new AtomicInteger();
+  public static final String MODULE_NAMESPACE_SEPARATOR = "$";
 
   /**
    * Constant
@@ -146,6 +147,7 @@ public class LogicalPlan implements Serializable, DAG
 
   private final Map<String, StreamMeta> streams = new HashMap<String, StreamMeta>();
   private final Map<String, OperatorMeta> operators = new HashMap<String, OperatorMeta>();
+  public final Map<String, ModuleMeta> modules = new LinkedHashMap<>();
   private final List<OperatorMeta> rootOperators = new ArrayList<OperatorMeta>();
   private final Attribute.AttributeMap attributes = new DefaultAttributeMap();
   private transient int nodeIndex = 0; // used for cycle validation
@@ -733,6 +735,7 @@ public class LogicalPlan implements Serializable, DAG
     private transient Integer lowlink; // for cycle detection
     private transient Operator operator;
     private MetricAggregatorMeta metricAggregatorMeta;
+    private String moduleName;  // Name of the module which has this operator. null if this is a top level operator.
 
     /*
      * Used for  OIO validation,
@@ -819,6 +822,16 @@ public class LogicalPlan implements Serializable, DAG
       return metricAggregatorMeta;
     }
 
+    public String getModuleName()
+    {
+      return moduleName;
+    }
+
+    public void setModuleName(String moduleName)
+    {
+      this.moduleName = moduleName;
+    }
+
     protected void populateAggregatorMeta()
     {
       AutoMetric.Aggregator aggregator = getValue(OperatorContext.METRICS_AGGREGATOR);
@@ -1073,37 +1086,38 @@ public class LogicalPlan implements Serializable, DAG
       throw new IllegalArgumentException("duplicate operator id: " + operators.get(name));
     }
 
+    // Avoid name conflict with module.
+    if (modules.containsKey(name))
+      throw new IllegalArgumentException("duplicate operator id: " + operators.get(name));
+
     OperatorMeta decl = new OperatorMeta(name, operator);
     rootOperators.add(decl); // will be removed when a sink is added to an input port for this operator
     operators.put(name, decl);
     return operator;
   }
 
+  /**
+   * Module meta object.
+   */
   public final class ModuleMeta implements DAG.ModuleMeta, Serializable
   {
     private final LinkedHashMap<InputPortMeta, StreamMeta> inputStreams = new LinkedHashMap<>();
     private final LinkedHashMap<OutputPortMeta, StreamMeta> outputStreams = new LinkedHashMap<>();
     private final Attribute.AttributeMap attributes;
-    @SuppressWarnings("unused")
-    private final int id;
     @NotNull
-    private final String name;
-    private transient Integer nindex; // for cycle detection
-    private transient Integer lowlink; // for cycle detection
+    private String name;
     private transient Module module;
+    private ModuleMeta parent;
+    private LogicalPlan dag = null;
+    private transient String fullName;
 
-    public ModuleMeta(String name, Module module)
-    {
-      this(name, module, new DefaultAttributeMap());
-    }
-
-    public ModuleMeta(String name, Module module, DefaultAttributeMap attributeMap)
+    private ModuleMeta(String name, Module module)
     {
       LOG.debug("Initializing {} as {}", name, module.getClass().getName());
       this.name = name;
       this.module = module;
-      this.id = logicalOperatorSequencer.decrementAndGet();
-      this.attributes = attributeMap;
+      this.attributes = new DefaultAttributeMap();
+      this.dag = new LogicalPlan();
     }
 
     @Override
@@ -1119,43 +1133,104 @@ public class LogicalPlan implements Serializable, DAG
     }
 
     @Override
-    public DAG.InputPortMeta getMeta(InputPort<?> port)
+    public Attribute.AttributeMap getAttributes()
     {
-      return null;
+      return attributes;
     }
 
     @Override
-    public DAG.OutputPortMeta getMeta(OutputPort<?> port)
+    public <T> T getValue(Attribute<T> key)
     {
-      return null;
+      return attributes.get(key);
     }
 
     @Override
-    public Attribute.AttributeMap getAttributes()
+    public void setCounters(Object counters)
     {
-      return null;
+
     }
 
     @Override
-    public <T> T getValue(Attribute<T> key)
+    public void sendMetrics(Collection<String> metricNames)
     {
-      return null;
+
     }
 
-    @Override
-    public void setCounters(Object counters)
+    public LinkedHashMap<InputPortMeta, StreamMeta> getInputStreams()
     {
+      return inputStreams;
+    }
 
+    public LinkedHashMap<OutputPortMeta, StreamMeta> getOutputStreams()
+    {
+      return outputStreams;
     }
 
-    @Override
-    public void sendMetrics(Collection<String> metricNames)
+    public LogicalPlan getDag()
     {
+      return dag;
+    }
 
+    private void writeObject(ObjectOutputStream out) throws IOException
+    {
+      out.defaultWriteObject();
+      FSStorageAgent.store(out, module);
     }
-  }
 
-  public transient Map<String, ModuleMeta> modules = Maps.newHashMap();
+    private void readObject(ObjectInputStream input) throws IOException, ClassNotFoundException
+    {
+      input.defaultReadObject();
+      module = (Module)FSStorageAgent.retrieve(input);
+    }
+
+    /**
+     * Expand the module and add its operator to the parentDAG. After this method finishes the module is expanded fully
+     * with all its submodules also expanded. The parentDAG contains the operator added by all the modules.
+     *
+     * @param parentDAG parent dag to populate with operators from this and inner modules.
+     * @param conf      configuration object.
+     */
+    public void flattenModule(LogicalPlan parentDAG, org.apache.hadoop.conf.Configuration conf)
+    {
+      module.populateDAG(dag, conf);
+      for (ModuleMeta subModuleMeta : dag.getAllModules()) {
+        subModuleMeta.setParent(this);
+        subModuleMeta.flattenModule(dag, conf);
+      }
+      parentDAG.addDAGToCurrentDAG(this);
+    }
+
+    /**
+     * Return full name of the module. If this is a inner module, i.e module inside of module this method will traverse
+     * till the top level module, and construct the name by concatenating name of modules in the chain in reverse order
+     * separated by MODULE_NAMESPACE_SEPARATO.
+     *
+     * For example If there is module M1, which adds another module M2 in the DAG. Then the full name of the module M2
+     * is ("M1" ++ MODULE_NAMESPACE_SEPARATO + "M2")
+     *
+     * @return full name of the module.
+     */
+    public String getFullName()
+    {
+      if (fullName != null) {
+        return fullName;
+      }
+
+      if (parent == null) {
+        fullName = name;
+      } else {
+        fullName = parent.getFullName() + MODULE_NAMESPACE_SEPARATOR + name;
+      }
+      return fullName;
+    }
+
+    private void setParent(ModuleMeta meta)
+    {
+      this.parent = meta;
+    }
+
+    private static final long serialVersionUID = 7562277769188329223L;
+  }
 
   @Override
   public <T extends Module> T addModule(String name, T module)
@@ -1166,6 +1241,10 @@ public class LogicalPlan implements Serializable, DAG
       }
       throw new IllegalArgumentException("duplicate module is: " + modules.get(name));
     }
+    if (operators.containsKey(name)) {
+      throw new IllegalArgumentException("duplicate module is: " + modules.get(name));
+    }
+
     ModuleMeta meta = new ModuleMeta(name, module);
     modules.put(name, meta);
     return module;
@@ -1228,6 +1307,33 @@ public class LogicalPlan implements Serializable, DAG
     return s;
   }
 
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  private void addDAGToCurrentDAG(ModuleMeta moduleMeta)
+  {
+    LogicalPlan subDag = moduleMeta.getDag();
+    String subDAGName = moduleMeta.getName();
+    String name;
+    for (OperatorMeta operatorMeta : subDag.getAllOperators()) {
+      name = subDAGName + MODULE_NAMESPACE_SEPARATOR + operatorMeta.getName();
+      this.addOperator(name, operatorMeta.getOperator());
+      OperatorMeta operatorMetaNew = this.getOperatorMeta(name);
+      operatorMetaNew.setModuleName(operatorMeta.getModuleName() == null ? subDAGName :
+          subDAGName + MODULE_NAMESPACE_SEPARATOR + operatorMeta.getModuleName());
+    }
+
+    for (StreamMeta streamMeta : subDag.getAllStreams()) {
+      OutputPortMeta sourceMeta = streamMeta.getSource();
+      List<InputPort<?>> ports = new LinkedList<>();
+      for (InputPortMeta inputPortMeta : streamMeta.getSinks()) {
+        ports.add(inputPortMeta.getPortObject());
+      }
+      InputPort[] inputPorts = ports.toArray(new InputPort[]{});
+
+      name = subDAGName + MODULE_NAMESPACE_SEPARATOR + streamMeta.getName();
+      this.addStream(name, sourceMeta.getPortObject(), inputPorts);
+    }
+  }
+
   @Override
   @SuppressWarnings("unchecked")
   public <T> StreamMeta addStream(String id, Operator.OutputPort<? extends T> source, Operator.InputPort<? super T> sink1)
@@ -1276,7 +1382,7 @@ public class LogicalPlan implements Serializable, DAG
 
   private OutputPortMeta assertGetPortMeta(Operator.OutputPort<?> port)
   {
-    for (OperatorMeta o: getAllOperators()) {
+    for (OperatorMeta o : getAllOperators()) {
       OutputPortMeta opm = o.getPortMapping().outPortMap.get(port);
       if (opm != null) {
         return opm;
@@ -1287,7 +1393,7 @@ public class LogicalPlan implements Serializable, DAG
 
   private InputPortMeta assertGetPortMeta(Operator.InputPort<?> port)
   {
-    for (OperatorMeta o: getAllOperators()) {
+    for (OperatorMeta o : getAllOperators()) {
       InputPortMeta opm = o.getPortMapping().inPortMap.get(port);
       if (opm != null) {
         return opm;
@@ -1324,7 +1430,8 @@ public class LogicalPlan implements Serializable, DAG
     return Collections.unmodifiableCollection(this.operators.values());
   }
 
-  public Collection<ModuleMeta> getAllModules() {
+  public Collection<ModuleMeta> getAllModules()
+  {
     return Collections.unmodifiableCollection(this.modules.values());
   }
 
@@ -1341,7 +1448,7 @@ public class LogicalPlan implements Serializable, DAG
 
   public ModuleMeta getModuleMeta(String moduleName)
   {
-    return null;
+    return this.modules.get(moduleName);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/14a09bb5/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
index 9bbe85c..6dc4c0c 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
@@ -2116,12 +2116,20 @@ public class LogicalPlanConfiguration {
 
     // Expand the modules within the dag recursively
     setModuleProperties(dag, appName);
+    flattenDAG(dag, conf);
 
     // inject external operator configuration
     setOperatorConfiguration(dag, appConfs, appName);
     setStreamConfiguration(dag, appConfs, appName);
   }
 
+  private void flattenDAG(LogicalPlan dag, Configuration conf)
+  {
+    for (ModuleMeta moduleMeta : dag.getAllModules()) {
+      moduleMeta.flattenModule(dag, conf);
+    }
+  }
+
   public static Properties readProperties(String filePath) throws IOException
   {
     InputStream is = new FileInputStream(filePath);


[21/50] incubator-apex-core git commit: APEXCORE-272 copy operator and port attributes from module dag to parent dag.

Posted by th...@apache.org.
APEXCORE-272 copy operator and port attributes from module dag to parent dag.


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

Branch: refs/heads/master
Commit: be075b81d5f5a7b4d5b556cfa3c5f28f349333f5
Parents: 3f76dcb
Author: Tushar R. Gosavi <tu...@apache.org>
Authored: Wed Dec 23 22:01:50 2015 +0530
Committer: Tushar R. Gosavi <tu...@apache.org>
Committed: Thu Dec 31 12:00:29 2015 +0530

----------------------------------------------------------------------
 .../stram/plan/logical/LogicalPlan.java         |  47 ++++++-
 .../logical/module/TestModuleExpansion.java     | 128 ++++++++++++++++++-
 2 files changed, 169 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/be075b81/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index 347e94f..867f814 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -910,6 +910,45 @@ public class LogicalPlan implements Serializable, DAG
         getValue(OperatorContext.METRICS_DIMENSIONS_SCHEME));
     }
 
+    /**
+     * Copy attribute from source attributeMap to destination attributeMap.
+     *
+     * @param dest  destination attribute map.
+     * @param source source attribute map.
+     */
+    private void copyAttributes(AttributeMap dest, AttributeMap source)
+    {
+      for (Entry<Attribute<?>, ?> a : source.entrySet()) {
+        dest.put((Attribute<Object>)a.getKey(), a.getValue());
+      }
+    }
+
+    /**
+     * Copy attribute of operator and port from provided operatorMeta. This function requires
+     * operatorMeta argument is for the same operator.
+     *
+     * @param operatorMeta copy attribute from this OperatorMeta to the object.
+     */
+    private void copyAttributesFrom(OperatorMeta operatorMeta)
+    {
+      if (operator != operatorMeta.getOperator()) {
+        throw new IllegalArgumentException("Operator meta is not for the same operator ");
+      }
+
+      // copy operator attributes
+      copyAttributes(attributes, operatorMeta.getAttributes());
+
+      // copy Input port attributes
+      for (Map.Entry<InputPort<?>, InputPortMeta> entry : operatorMeta.getPortMapping().inPortMap.entrySet()) {
+        copyAttributes(getPortMapping().inPortMap.get(entry.getKey()).attributes, entry.getValue().attributes);
+      }
+
+      // copy Output port attributes
+      for (Map.Entry<OutputPort<?>, OutputPortMeta> entry : operatorMeta.getPortMapping().outPortMap.entrySet()) {
+        copyAttributes(getPortMapping().outPortMap.get(entry.getKey()).attributes, entry.getValue().attributes);
+      }
+    }
+
     private class PortMapping implements Operators.OperatorDescriptor
     {
       private final Map<Operator.InputPort<?>, InputPortMeta> inPortMap = new HashMap<Operator.InputPort<?>, InputPortMeta>();
@@ -1362,8 +1401,9 @@ public class LogicalPlan implements Serializable, DAG
     String name;
     for (OperatorMeta operatorMeta : subDag.getAllOperators()) {
       name = subDAGName + MODULE_NAMESPACE_SEPARATOR + operatorMeta.getName();
-      this.addOperator(name, operatorMeta.getOperator());
-      OperatorMeta operatorMetaNew = this.getOperatorMeta(name);
+      Operator op = this.addOperator(name, operatorMeta.getOperator());
+      OperatorMeta operatorMetaNew = this.getMeta(op);
+      operatorMetaNew.copyAttributesFrom(operatorMeta);
       operatorMetaNew.setModuleName(operatorMeta.getModuleName() == null ? subDAGName :
           subDAGName + MODULE_NAMESPACE_SEPARATOR + operatorMeta.getModuleName());
     }
@@ -1377,7 +1417,8 @@ public class LogicalPlan implements Serializable, DAG
       InputPort[] inputPorts = ports.toArray(new InputPort[]{});
 
       name = subDAGName + MODULE_NAMESPACE_SEPARATOR + streamMeta.getName();
-      this.addStream(name, sourceMeta.getPortObject(), inputPorts);
+      StreamMeta streamMetaNew = this.addStream(name, sourceMeta.getPortObject(), inputPorts);
+      streamMetaNew.setLocality(streamMeta.getLocality());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/be075b81/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java
index 5bfd8f1..d5af67b 100644
--- a/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java
@@ -18,8 +18,11 @@
  */
 package com.datatorrent.stram.plan.logical.module;
 
+import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 
 import org.junit.Assert;
@@ -27,15 +30,19 @@ import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
 
+import com.datatorrent.api.Attribute;
+import com.datatorrent.api.Context;
 import com.datatorrent.api.DAG;
 import com.datatorrent.api.DefaultInputPort;
 import com.datatorrent.api.DefaultOutputPort;
 import com.datatorrent.api.InputOperator;
 import com.datatorrent.api.Module;
+import com.datatorrent.api.Partitioner;
 import com.datatorrent.api.StreamingApplication;
 import com.datatorrent.api.annotation.InputPortFieldAnnotation;
 import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
 import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.stram.engine.OperatorContext;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
 import com.datatorrent.stram.plan.logical.LogicalPlanConfiguration;
 
@@ -97,6 +104,23 @@ public class TestModuleExpansion
     }
   }
 
+  static class TestPartitioner implements Partitioner<DummyOperator>, Serializable
+  {
+    @Override
+    public Collection<Partition<DummyOperator>> definePartitions(Collection<Partition<DummyOperator>> partitions, PartitioningContext context)
+    {
+      ArrayList<Partition<DummyOperator>> lst = new ArrayList();
+      lst.add(partitions.iterator().next());
+      return lst;
+    }
+
+    @Override
+    public void partitioned(Map<Integer, Partition<DummyOperator>> partitions)
+    {
+
+    }
+  }
+
   static class Level1Module implements Module
   {
     private int level1ModuleProp = 0;
@@ -105,12 +129,26 @@ public class TestModuleExpansion
     public final transient ProxyInputPort<Integer> mIn = new ProxyInputPort<>();
     @OutputPortFieldAnnotation(optional = true)
     public final transient ProxyOutputPort<Integer> mOut = new ProxyOutputPort<>();
+    private int memory = 512;
+    private int portMemory = 2;
 
     @Override
     public void populateDAG(DAG dag, Configuration conf)
     {
       DummyOperator o1 = dag.addOperator("O1", new DummyOperator());
       o1.setOperatorProp(level1ModuleProp);
+
+      /** set various attribute on the operator for testing */
+      Attribute.AttributeMap attr = dag.getMeta(o1).getAttributes();
+      attr.put(OperatorContext.MEMORY_MB, memory);
+      attr.put(OperatorContext.APPLICATION_WINDOW_COUNT, 2);
+      attr.put(OperatorContext.LOCALITY_HOST, "host1");
+      attr.put(OperatorContext.PARTITIONER, new TestPartitioner());
+      attr.put(OperatorContext.CHECKPOINT_WINDOW_COUNT, 120);
+      attr.put(OperatorContext.STATELESS, true);
+      attr.put(OperatorContext.SPIN_MILLIS, 20);
+
+      dag.setInputPortAttribute(o1.in, Context.PortContext.BUFFER_MEMORY_MB, portMemory);
       mIn.set(o1.in);
       mOut.set(o1.out1);
     }
@@ -124,6 +162,26 @@ public class TestModuleExpansion
     {
       this.level1ModuleProp = level1ModuleProp;
     }
+
+    public int getMemory()
+    {
+      return memory;
+    }
+
+    public void setMemory(int memory)
+    {
+      this.memory = memory;
+    }
+
+    public int getPortMemory()
+    {
+      return portMemory;
+    }
+
+    public void setPortMemory(int portMemory)
+    {
+      this.portMemory = portMemory;
+    }
   }
 
   static class Level2ModuleA implements Module
@@ -145,15 +203,19 @@ public class TestModuleExpansion
     public void populateDAG(DAG dag, Configuration conf)
     {
       Level1Module m1 = dag.addModule("M1", new Level1Module());
+      m1.setMemory(1024);
+      m1.setPortMemory(1);
       m1.setLevel1ModuleProp(level2ModuleAProp1);
 
       Level1Module m2 = dag.addModule("M2", new Level1Module());
+      m2.setMemory(2048);
+      m2.setPortMemory(2);
       m2.setLevel1ModuleProp(level2ModuleAProp2);
 
       DummyOperator o1 = dag.addOperator("O1", new DummyOperator());
       o1.setOperatorProp(level2ModuleAProp3);
 
-      dag.addStream("M1_M2&O1", m1.mOut, m2.mIn, o1.in);
+      dag.addStream("M1_M2&O1", m1.mOut, m2.mIn, o1.in).setLocality(DAG.Locality.CONTAINER_LOCAL);
 
       mIn.set(m1.mIn);
       mOut1.set(m2.mOut);
@@ -213,13 +275,15 @@ public class TestModuleExpansion
       o1.setOperatorProp(level2ModuleBProp1);
 
       Level1Module m1 = dag.addModule("M1", new Level1Module());
+      m1.setMemory(4096);
+      m1.setPortMemory(3);
       m1.setLevel1ModuleProp(level2ModuleBProp2);
 
       DummyOperator o2 = dag.addOperator("O2", new DummyOperator());
       o2.setOperatorProp(level2ModuleBProp3);
 
-      dag.addStream("O1_M1", o1.out1, m1.mIn);
-      dag.addStream("O1_O2", o1.out2, o2.in);
+      dag.addStream("O1_M1", o1.out1, m1.mIn).setLocality(DAG.Locality.THREAD_LOCAL);
+      dag.addStream("O1_O2", o1.out2, o2.in).setLocality(DAG.Locality.RACK_LOCAL);
 
       mIn.set(o1.in);
       mOut1.set(m1.mOut);
@@ -370,6 +434,15 @@ public class TestModuleExpansion
     validateSeperateStream(dag, componentName("Md", "O1_O2"), componentName("Md", "O1"), componentName("Md", "O2"));
     validateSeperateStream(dag, "Ma_Mb", componentName("Ma", "M2", "O1"), componentName("Mb", "O1"));
     validateSeperateStream(dag, "O1_O2", "O1", "O2", componentName("Me", "O1"));
+
+    /* Verify that stream locality is set correctly in top level dag */
+    validateStreamLocality(dag, componentName("Mc", "M1_M2&O1"), DAG.Locality.CONTAINER_LOCAL);
+    validateStreamLocality(dag, componentName("Mb", "O1_M1"), DAG.Locality.THREAD_LOCAL);
+    validateStreamLocality(dag, componentName("Mb", "O1_O2"), DAG.Locality.RACK_LOCAL);
+    validateStreamLocality(dag, componentName("Mc", "M1_M2&O1"), DAG.Locality.CONTAINER_LOCAL);
+    validateStreamLocality(dag, componentName("Md", "O1_M1"), DAG.Locality.THREAD_LOCAL);
+    validateStreamLocality(dag, componentName("Me", "s1"), null);
+
   }
 
   private void validateSeperateStream(LogicalPlan dag, String streamName, String inputOperatorName,
@@ -441,6 +514,18 @@ public class TestModuleExpansion
     validateOperatorParent(dag, componentName("Md", "O1"), "Md");
     validateOperatorParent(dag, componentName("Md", "M1", "O1"), componentName("Md", "M1"));
     validateOperatorParent(dag, componentName("Md", "O2"), "Md");
+
+    validateOperatorAttribute(dag, componentName("Ma", "M1", "O1"), 1024);
+    validateOperatorAttribute(dag, componentName("Ma", "M2", "O1"), 2048);
+    validateOperatorAttribute(dag, componentName("Mb", "M1", "O1"), 4096);
+    validateOperatorAttribute(dag, componentName("Mc", "M1", "O1"), 1024);
+    validateOperatorAttribute(dag, componentName("Mc", "M2", "O1"), 2048);
+
+    validatePortAttribute(dag, componentName("Ma", "M1", "O1"), 1);
+    validatePortAttribute(dag, componentName("Ma", "M2", "O1"), 2);
+    validatePortAttribute(dag, componentName("Mb", "M1", "O1"), 3);
+    validatePortAttribute(dag, componentName("Mc", "M1", "O1"), 1);
+    validatePortAttribute(dag, componentName("Mc", "M2", "O1"), 2);
   }
 
   private void validateOperatorParent(LogicalPlan dag, String operatorName, String parentModuleName)
@@ -549,4 +634,41 @@ public class TestModuleExpansion
     lpc.prepareDAG(dag, null, "ModuleApp");
     dag.validate();
   }
+
+  /**
+   * Verify attributes populated on DummyOperator from Level1 module
+   */
+  private void validateOperatorAttribute(LogicalPlan dag, String name, int memory)
+  {
+    LogicalPlan.OperatorMeta oMeta = dag.getOperatorMeta(name);
+    Attribute.AttributeMap attrs = oMeta.getAttributes();
+    Assert.assertEquals((int)attrs.get(OperatorContext.MEMORY_MB), memory);
+    Assert.assertEquals("Application window id is 2 ", (int)attrs.get(OperatorContext.APPLICATION_WINDOW_COUNT), 2);
+    Assert.assertEquals("Locality host is host1", attrs.get(OperatorContext.LOCALITY_HOST), "host1");
+    Assert.assertEquals(attrs.get(OperatorContext.PARTITIONER).getClass(), TestPartitioner.class);
+    Assert.assertEquals("Checkpoint window count ", (int)attrs.get(OperatorContext.CHECKPOINT_WINDOW_COUNT), 120);
+    Assert.assertEquals("Operator is stateless ", attrs.get(OperatorContext.STATELESS), true);
+    Assert.assertEquals("SPIN MILLIS is set to 20 ", (int)attrs.get(OperatorContext.SPIN_MILLIS), 20);
+
+  }
+
+  /**
+   * Validate attribute set on the port of DummyOperator in Level1Module
+   */
+  private void validatePortAttribute(LogicalPlan dag, String name, int memory)
+  {
+    LogicalPlan.InputPortMeta imeta = dag.getOperatorMeta(name).getInputStreams().keySet().iterator().next();
+    Assert.assertEquals(memory, (int)imeta.getAttributes().get(Context.PortContext.BUFFER_MEMORY_MB));
+  }
+
+  /**
+   * validate if stream attributes are copied or not
+   */
+  private void validateStreamLocality(LogicalPlan dag, String name, DAG.Locality locality)
+  {
+    LogicalPlan.StreamMeta meta = dag.getStream(name);
+    Assert.assertTrue("Metadata for stream is available ", meta != null);
+    Assert.assertEquals("Locality is " + locality, meta.getLocality(), locality);
+  }
+
 }


[46/50] incubator-apex-core git commit: APEXCORE-306 Skip recovery checkpoint upgrade for entire group during deploy.

Posted by th...@apache.org.
APEXCORE-306 Skip recovery checkpoint upgrade for entire group during deploy.


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

Branch: refs/heads/master
Commit: 5371bc7b7cf33e5d5c96775e241af002fc22d5ed
Parents: b3402be
Author: Thomas Weise <th...@datatorrent.com>
Authored: Sat Jan 23 00:34:42 2016 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sat Jan 23 00:34:42 2016 -0800

----------------------------------------------------------------------
 .../java/com/datatorrent/stram/StreamingContainerManager.java    | 4 +++-
 .../com/datatorrent/stram/plan/logical/DelayOperatorTest.java    | 1 -
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/5371bc7b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
index a687a37..df3bfc4 100644
--- a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
+++ b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
@@ -1939,6 +1939,7 @@ public class StreamingContainerManager implements PlanContext
       commonCheckpoints.addAll(operator.checkpoints);
     }
     Set<PTOperator> groupOpers = new HashSet<>(checkpointGroup.size());
+    boolean pendingDeploy = operator.getState() == PTOperator.State.PENDING_DEPLOY;
     if (checkpointGroup.size() > 1) {
       for (OperatorMeta om : checkpointGroup) {
         Collection<PTOperator> operators = plan.getAllOperators(om);
@@ -1949,6 +1950,7 @@ public class StreamingContainerManager implements PlanContext
           // visit all downstream operators of the group
           ctx.visited.add(groupOper);
           groupOpers.add(groupOper);
+          pendingDeploy |= operator.getState() == PTOperator.State.PENDING_DEPLOY;
         }
       }
       // highest common checkpoint
@@ -2004,7 +2006,7 @@ public class StreamingContainerManager implements PlanContext
 
     for (PTOperator groupOper : groupOpers) {
       // checkpoint frozen during deployment
-      if (ctx.recovery || groupOper.getState() != PTOperator.State.PENDING_DEPLOY) {
+      if (!pendingDeploy || ctx.recovery) {
         // remove previous checkpoints
         Checkpoint c1 = Checkpoint.INITIAL_CHECKPOINT;
         LinkedList<Checkpoint> checkpoints = groupOper.checkpoints;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/5371bc7b/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
index 06f184f..cb4222a 100644
--- a/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
@@ -321,7 +321,6 @@ public class DelayOperatorTest
         FibonacciOperator.results.subList(0, 10).toArray());
   }
 
-  @Ignore // Out of sequence BEGIN_WINDOW tuple on Travis. Will tackle in the next version
   @Test
   public void testFibonacciRecovery1() throws Exception
   {


[33/50] incubator-apex-core git commit: APEXCORE-303 #resolve evaluate the launch properties at application launch

Posted by th...@apache.org.
APEXCORE-303 #resolve evaluate the launch properties at application launch


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

Branch: refs/heads/master
Commit: ec0c88b183d82257da185d42e664bd95474ad206
Parents: 56b55fe
Author: David Yan <da...@datatorrent.com>
Authored: Fri Jan 8 16:23:52 2016 -0800
Committer: David Yan <da...@datatorrent.com>
Committed: Fri Jan 8 16:23:52 2016 -0800

----------------------------------------------------------------------
 .../com/datatorrent/stram/client/StramAppLauncher.java |  1 +
 .../com/datatorrent/stram/client/StramClientUtils.java | 13 ++++++++++++-
 2 files changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ec0c88b1/engine/src/main/java/com/datatorrent/stram/client/StramAppLauncher.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/client/StramAppLauncher.java b/engine/src/main/java/com/datatorrent/stram/client/StramAppLauncher.java
index f1e7261..d3666d4 100644
--- a/engine/src/main/java/com/datatorrent/stram/client/StramAppLauncher.java
+++ b/engine/src/main/java/com/datatorrent/stram/client/StramAppLauncher.java
@@ -435,6 +435,7 @@ public class StramAppLauncher
         conf.set(entry.getKey(), entry.getValue());
       }
     }
+    StramClientUtils.evalConfiguration(conf);
     return conf;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ec0c88b1/engine/src/main/java/com/datatorrent/stram/client/StramClientUtils.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/client/StramClientUtils.java b/engine/src/main/java/com/datatorrent/stram/client/StramClientUtils.java
index 27f58f0..7e23fa5 100644
--- a/engine/src/main/java/com/datatorrent/stram/client/StramClientUtils.java
+++ b/engine/src/main/java/com/datatorrent/stram/client/StramClientUtils.java
@@ -659,7 +659,6 @@ public class StramClientUtils
       engine.eval("var _prop = {}");
       for (Map.Entry<String, String> entry : vars) {
         String evalString = String.format("_prop[\"%s\"] = \"%s\"", StringEscapeUtils.escapeJava(entry.getKey()), StringEscapeUtils.escapeJava(entry.getValue()));
-        LOG.debug("Evaluating: {}", evalString);
         engine.eval(evalString);
       }
     } catch (ScriptException ex) {
@@ -709,6 +708,18 @@ public class StramClientUtils
     }
   }
 
+  public static void evalConfiguration(Configuration conf)
+  {
+    Properties props = new Properties();
+    for (Map.Entry entry : conf) {
+      props.put(entry.getKey(), entry.getValue());
+    }
+    evalProperties(props, conf);
+    for (Map.Entry<Object, Object> entry : props.entrySet()) {
+      conf.set((String)entry.getKey(), (String)entry.getValue());
+    }
+  }
+
   public static <T> T doAs(String userName, PrivilegedExceptionAction<T> action) throws Exception
   {
     if (StringUtils.isNotBlank(userName) && !userName.equals(UserGroupInformation.getLoginUser().getShortUserName())) {


[24/50] incubator-apex-core git commit: APEXCORE-298 #resolve #comment reducing the severity level of line length check

Posted by th...@apache.org.
APEXCORE-298 #resolve #comment reducing the severity level of line length check


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

Branch: refs/heads/master
Commit: fadb6f4f702315ea7123911d330e3e9711f439db
Parents: 2da1000
Author: Chandni Singh <cs...@apache.org>
Authored: Mon Jan 4 11:27:06 2016 -0800
Committer: Chandni Singh <cs...@apache.org>
Committed: Mon Jan 4 11:27:06 2016 -0800

----------------------------------------------------------------------
 apex_checks.xml | 1 +
 api/pom.xml     | 2 +-
 common/pom.xml  | 2 +-
 engine/pom.xml  | 2 +-
 4 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/fadb6f4f/apex_checks.xml
----------------------------------------------------------------------
diff --git a/apex_checks.xml b/apex_checks.xml
index 60d99fe..136a0d0 100644
--- a/apex_checks.xml
+++ b/apex_checks.xml
@@ -101,6 +101,7 @@
 
     <module name="LineLength">
       <property name="max" value="120"/>
+      <property name="severity" value="warning"/>
     </module>
 
     <module name="OneStatementPerLine"/>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/fadb6f4f/api/pom.xml
----------------------------------------------------------------------
diff --git a/api/pom.xml b/api/pom.xml
index 79af4b3..f6e1ada 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -71,7 +71,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>114</maxAllowedViolations>
+          <maxAllowedViolations>61</maxAllowedViolations>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/fadb6f4f/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index f1bae56..94f317f 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -56,7 +56,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>149</maxAllowedViolations>
+          <maxAllowedViolations>114</maxAllowedViolations>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/fadb6f4f/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index f345410..3e43184 100644
--- a/engine/pom.xml
+++ b/engine/pom.xml
@@ -145,7 +145,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>4403</maxAllowedViolations>
+          <maxAllowedViolations>3224</maxAllowedViolations>
         </configuration>
       </plugin>
       <plugin>


[08/50] incubator-apex-core git commit: APEXCORE-288 #resolve added group id information to apex app package

Posted by th...@apache.org.
APEXCORE-288 #resolve added group id information to apex app package


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

Branch: refs/heads/master
Commit: c5780754c0f5b5ffc9fdb2a75f52ea58ba7ceb86
Parents: 0b0f914
Author: David Yan <da...@datatorrent.com>
Authored: Tue Dec 15 16:46:58 2015 -0800
Committer: David Yan <da...@datatorrent.com>
Committed: Tue Dec 15 16:46:58 2015 -0800

----------------------------------------------------------------------
 .../src/main/resources/archetype-resources/pom.xml        |  1 +
 .../src/main/resources/archetype-resources/pom.xml        |  2 ++
 engine/src/main/java/com/datatorrent/stram/cli/DTCli.java |  5 +++++
 .../java/com/datatorrent/stram/client/AppPackage.java     | 10 +++++++++-
 .../java/com/datatorrent/stram/client/ConfigPackage.java  |  8 ++++++++
 .../java/com/datatorrent/stram/client/AppPackageTest.java |  1 +
 .../com/datatorrent/stram/client/ConfigPackageTest.java   |  1 +
 engine/src/test/resources/testAppPackage/mydtapp/pom.xml  |  1 +
 .../testConfigPackageSrc/META-INF/MANIFEST.MF             |  1 +
 9 files changed, 29 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c5780754/apex-app-archetype/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/apex-app-archetype/src/main/resources/archetype-resources/pom.xml b/apex-app-archetype/src/main/resources/archetype-resources/pom.xml
index 74f320c..d690034 100644
--- a/apex-app-archetype/src/main/resources/archetype-resources/pom.xml
+++ b/apex-app-archetype/src/main/resources/archetype-resources/pom.xml
@@ -80,6 +80,7 @@
                  <manifestEntries>
                    <Class-Path>${apex.apppackage.classpath}</Class-Path>
                    <DT-Engine-Version>${apex.version}</DT-Engine-Version>
+                   <DT-App-Package-Group-Id>${project.groupId}</DT-App-Package-Group-Id>
                    <DT-App-Package-Name>${project.artifactId}</DT-App-Package-Name>
                    <DT-App-Package-Version>${project.version}</DT-App-Package-Version>
                    <DT-App-Package-Display-Name>${project.name}</DT-App-Package-Display-Name>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c5780754/apex-conf-archetype/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/apex-conf-archetype/src/main/resources/archetype-resources/pom.xml b/apex-conf-archetype/src/main/resources/archetype-resources/pom.xml
index 42ec2bf..624fb0d 100644
--- a/apex-conf-archetype/src/main/resources/archetype-resources/pom.xml
+++ b/apex-conf-archetype/src/main/resources/archetype-resources/pom.xml
@@ -19,6 +19,7 @@
   </licenses>
   
   <properties>
+    <apex.apppackage.groupid>${groupId}</apex.apppackage.groupid>
     <apex.apppackage.name>myapexapp</apex.apppackage.name>
     <apex.apppackage.minversion>1.0.0</apex.apppackage.minversion>
     <apex.apppackage.maxversion>1.9999.9999</apex.apppackage.maxversion>
@@ -106,6 +107,7 @@
                </archiverConfig>                  
                <archive>
                  <manifestEntries>
+                   <DT-App-Package-Group-Id>${apex.apppackage.groupid}</DT-App-Package-Group-Id>
                    <DT-App-Package-Name>${apex.apppackage.name}</DT-App-Package-Name>
                    <DT-App-Package-Min-Version>${apex.apppackage.minversion}</DT-App-Package-Min-Version>
                    <DT-App-Package-Max-Version>${apex.apppackage.maxversion}</DT-App-Package-Max-Version>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c5780754/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java b/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java
index deb0967..7f01b75 100644
--- a/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java
+++ b/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java
@@ -3435,9 +3435,14 @@ public class DTCli
       return;
     }
     String requiredAppPackageName = cp.getAppPackageName();
+    String requiredAppPackageGroupId = cp.getAppPackageGroupId();
     if (requiredAppPackageName != null && !requiredAppPackageName.equals(ap.getAppPackageName())) {
       throw new CliException("Config package requires an app package name of \"" + requiredAppPackageName + "\". The app package given has the name of \"" + ap.getAppPackageName() + "\"");
     }
+    if (requiredAppPackageGroupId != null && !requiredAppPackageGroupId.equals(ap.getAppPackageGroupId())) {
+      throw new CliException("Config package requires an app package group id of \"" + requiredAppPackageGroupId +
+          "\". The app package given has the groupId of \"" + ap.getAppPackageGroupId() + "\"");
+    }
     String requiredAppPackageMinVersion = cp.getAppPackageMinVersion();
     if (requiredAppPackageMinVersion != null && VersionInfo.compare(requiredAppPackageMinVersion, ap.getAppPackageVersion()) > 0) {
       throw new CliException("Config package requires an app package minimum version of \"" + requiredAppPackageMinVersion + "\". The app package given is of version \"" + ap.getAppPackageVersion() + "\"");

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c5780754/engine/src/main/java/com/datatorrent/stram/client/AppPackage.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/client/AppPackage.java b/engine/src/main/java/com/datatorrent/stram/client/AppPackage.java
index cbe63f2..86ca915 100644
--- a/engine/src/main/java/com/datatorrent/stram/client/AppPackage.java
+++ b/engine/src/main/java/com/datatorrent/stram/client/AppPackage.java
@@ -44,12 +44,14 @@ public class AppPackage extends JarFile
   public static final String ATTRIBUTE_DT_ENGINE_VERSION = "DT-Engine-Version";
   public static final String ATTRIBUTE_DT_APP_PACKAGE_NAME = "DT-App-Package-Name";
   public static final String ATTRIBUTE_DT_APP_PACKAGE_VERSION = "DT-App-Package-Version";
+  public static final String ATTRIBUTE_DT_APP_PACKAGE_GROUP_ID = "DT-App-Package-Group-Id";
   public static final String ATTRIBUTE_CLASS_PATH = "Class-Path";
   public static final String ATTRIBUTE_DT_APP_PACKAGE_DISPLAY_NAME = "DT-App-Package-Display-Name";
   public static final String ATTRIBUTE_DT_APP_PACKAGE_DESCRIPTION = "DT-App-Package-Description";
 
   private final String appPackageName;
   private final String appPackageVersion;
+  private final String appPackageGroupId;
   private final String dtEngineVersion;
   private final String appPackageDescription;
   private final String appPackageDisplayName;
@@ -129,12 +131,13 @@ public class AppPackage extends JarFile
     Attributes attr = manifest.getMainAttributes();
     appPackageName = attr.getValue(ATTRIBUTE_DT_APP_PACKAGE_NAME);
     appPackageVersion = attr.getValue(ATTRIBUTE_DT_APP_PACKAGE_VERSION);
+    appPackageGroupId = attr.getValue(ATTRIBUTE_DT_APP_PACKAGE_GROUP_ID);
     dtEngineVersion = attr.getValue(ATTRIBUTE_DT_ENGINE_VERSION);
     appPackageDisplayName = attr.getValue(ATTRIBUTE_DT_APP_PACKAGE_DISPLAY_NAME);
     appPackageDescription = attr.getValue(ATTRIBUTE_DT_APP_PACKAGE_DESCRIPTION);
     String classPathString = attr.getValue(ATTRIBUTE_CLASS_PATH);
     if (appPackageName == null || appPackageVersion == null || classPathString == null) {
-      throw new IOException("Not a valid app package.  Class-Path is missing from MANIFEST.MF");
+      throw new IOException("Not a valid app package.  App Package Name or Version or Class-Path is missing from MANIFEST.MF");
     }
     classPath.addAll(Arrays.asList(StringUtils.split(classPathString, " ")));
     directory = contentFolder;
@@ -234,6 +237,11 @@ public class AppPackage extends JarFile
     return appPackageVersion;
   }
 
+  public String getAppPackageGroupId()
+  {
+    return appPackageGroupId;
+  }
+
   public String getAppPackageDescription()
   {
     return appPackageDescription;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c5780754/engine/src/main/java/com/datatorrent/stram/client/ConfigPackage.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/client/ConfigPackage.java b/engine/src/main/java/com/datatorrent/stram/client/ConfigPackage.java
index 622665b..185cb7a 100644
--- a/engine/src/main/java/com/datatorrent/stram/client/ConfigPackage.java
+++ b/engine/src/main/java/com/datatorrent/stram/client/ConfigPackage.java
@@ -39,6 +39,7 @@ public class ConfigPackage extends JarFile implements Closeable
 
   public static final String ATTRIBUTE_DT_CONF_PACKAGE_NAME = "DT-Conf-Package-Name";
   public static final String ATTRIBUTE_DT_APP_PACKAGE_NAME = "DT-App-Package-Name";
+  public static final String ATTRIBUTE_DT_APP_PACKAGE_GROUP_ID = "DT-App-Package-Group-Id";
   public static final String ATTRIBUTE_DT_APP_PACKAGE_MIN_VERSION = "DT-App-Package-Min-Version";
   public static final String ATTRIBUTE_DT_APP_PACKAGE_MAX_VERSION = "DT-App-Package-Max-Version";
   public static final String ATTRIBUTE_DT_CONF_PACKAGE_DESCRIPTION = "DT-Conf-Package-Description";
@@ -47,6 +48,7 @@ public class ConfigPackage extends JarFile implements Closeable
 
   private final String configPackageName;
   private final String appPackageName;
+  private final String appPackageGroupId;
   private final String appPackageMinVersion;
   private final String appPackageMaxVersion;
   private final String configPackageDescription;
@@ -74,6 +76,7 @@ public class ConfigPackage extends JarFile implements Closeable
     Attributes attr = manifest.getMainAttributes();
     configPackageName = attr.getValue(ATTRIBUTE_DT_CONF_PACKAGE_NAME);
     appPackageName = attr.getValue(ATTRIBUTE_DT_APP_PACKAGE_NAME);
+    appPackageGroupId = attr.getValue(ATTRIBUTE_DT_APP_PACKAGE_GROUP_ID);
     appPackageMinVersion = attr.getValue(ATTRIBUTE_DT_APP_PACKAGE_MIN_VERSION);
     appPackageMaxVersion = attr.getValue(ATTRIBUTE_DT_APP_PACKAGE_MAX_VERSION);
     configPackageDescription = attr.getValue(ATTRIBUTE_DT_CONF_PACKAGE_DESCRIPTION);
@@ -122,6 +125,11 @@ public class ConfigPackage extends JarFile implements Closeable
     return appPackageName;
   }
 
+  public String getAppPackageGroupId()
+  {
+    return appPackageGroupId;
+  }
+
   public String getAppPackageMinVersion()
   {
     return appPackageMinVersion;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c5780754/engine/src/test/java/com/datatorrent/stram/client/AppPackageTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/client/AppPackageTest.java b/engine/src/test/java/com/datatorrent/stram/client/AppPackageTest.java
index 0d284cf..5400930 100644
--- a/engine/src/test/java/com/datatorrent/stram/client/AppPackageTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/client/AppPackageTest.java
@@ -82,6 +82,7 @@ public class AppPackageTest
   @Test
   public void testAppPackage() throws Exception
   {
+    Assert.assertEquals("com.example", json.getString("appPackageGroupId"));
     Assert.assertEquals("mydtapp", json.getString("appPackageName"));
     Assert.assertEquals("1.0-SNAPSHOT", json.getString("appPackageVersion"));
     Assert.assertEquals("3.2.0-incubating", json.getString("dtEngineVersion"));

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c5780754/engine/src/test/java/com/datatorrent/stram/client/ConfigPackageTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/client/ConfigPackageTest.java b/engine/src/test/java/com/datatorrent/stram/client/ConfigPackageTest.java
index 651f56d..c09bc84 100644
--- a/engine/src/test/java/com/datatorrent/stram/client/ConfigPackageTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/client/ConfigPackageTest.java
@@ -98,6 +98,7 @@ public class ConfigPackageTest
   @Test
   public void testConfigPackage() throws Exception
   {
+    Assert.assertEquals("com.example", json.getString("appPackageGroupId"));
     Assert.assertEquals("mydtapp", json.getString("appPackageName"));
     Assert.assertEquals("1.0", json.getString("appPackageMinVersion"));
     Assert.assertEquals("1.9999", json.getString("appPackageMaxVersion"));

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c5780754/engine/src/test/resources/testAppPackage/mydtapp/pom.xml
----------------------------------------------------------------------
diff --git a/engine/src/test/resources/testAppPackage/mydtapp/pom.xml b/engine/src/test/resources/testAppPackage/mydtapp/pom.xml
index 9d2ea7d..4ba5870 100644
--- a/engine/src/test/resources/testAppPackage/mydtapp/pom.xml
+++ b/engine/src/test/resources/testAppPackage/mydtapp/pom.xml
@@ -119,6 +119,7 @@
                  <manifestEntries>
                    <Class-Path>${apex.apppackage.classpath}</Class-Path>
                    <DT-Engine-Version>${apex.version}</DT-Engine-Version>
+                   <DT-App-Package-Group-Id>${project.groupId}</DT-App-Package-Group-Id>
                    <DT-App-Package-Name>${project.artifactId}</DT-App-Package-Name>
                    <DT-App-Package-Version>${project.version}</DT-App-Package-Version>
                    <DT-App-Package-Display-Name>${project.name}</DT-App-Package-Display-Name>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/c5780754/engine/src/test/resources/testConfigPackage/testConfigPackageSrc/META-INF/MANIFEST.MF
----------------------------------------------------------------------
diff --git a/engine/src/test/resources/testConfigPackage/testConfigPackageSrc/META-INF/MANIFEST.MF b/engine/src/test/resources/testConfigPackage/testConfigPackageSrc/META-INF/MANIFEST.MF
index efacab4..7edd658 100644
--- a/engine/src/test/resources/testConfigPackage/testConfigPackageSrc/META-INF/MANIFEST.MF
+++ b/engine/src/test/resources/testConfigPackage/testConfigPackageSrc/META-INF/MANIFEST.MF
@@ -5,6 +5,7 @@ Built-By: dtdeveloper
 Build-Jdk: 1.6.0_45
 DT-App-Package-Max-Version: 1.9999
 DT-App-Package-Min-Version: 1.0
+DT-App-Package-Group-Id: com.example
 DT-App-Package-Name: mydtapp
 DT-Conf-Package-Description: My DataTorrent Application Configuration 
  Description


[43/50] incubator-apex-core git commit: APEXCORE-60 Iteration support in Apex Core

Posted by th...@apache.org.
APEXCORE-60 Iteration support in Apex Core


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

Branch: refs/heads/master
Commit: f7e1ccf14154eca92b24c5f6b5387fe56c516829
Parents: d0908e4
Author: David Yan <da...@datatorrent.com>
Authored: Wed Dec 9 15:52:26 2015 -0800
Committer: David Yan <da...@datatorrent.com>
Committed: Fri Jan 22 19:03:46 2016 -0800

----------------------------------------------------------------------
 .../main/java/com/datatorrent/api/Context.java  |   7 +
 .../main/java/com/datatorrent/api/Operator.java |  19 +
 .../common/util/DefaultDelayOperator.java       |  75 ++++
 .../datatorrent/stram/StramLocalCluster.java    |  15 +
 .../stram/StreamingContainerManager.java        |  56 ++-
 .../datatorrent/stram/engine/GenericNode.java   | 190 +++++++---
 .../java/com/datatorrent/stram/engine/Node.java |   6 +-
 .../stram/engine/StreamingContainer.java        |   2 +
 .../stram/engine/WindowGenerator.java           |  14 +-
 .../stram/plan/logical/LogicalPlan.java         |  53 +++
 .../stram/plan/physical/PTOperator.java         |   4 +-
 .../stram/plan/physical/PhysicalPlan.java       |  19 +-
 .../stram/plan/physical/StreamMapping.java      |   4 +-
 .../java/com/datatorrent/stram/tuple/Tuple.java |   5 +
 .../stram/debug/TupleRecorderTest.java          | 208 +++++-----
 .../stram/engine/GenericNodeTest.java           |  18 +-
 .../stram/engine/GenericTestOperator.java       |   3 +
 .../stram/plan/logical/DelayOperatorTest.java   | 377 +++++++++++++++++++
 18 files changed, 888 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/api/src/main/java/com/datatorrent/api/Context.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Context.java b/api/src/main/java/com/datatorrent/api/Context.java
index ceed8a2..58bc552 100644
--- a/api/src/main/java/com/datatorrent/api/Context.java
+++ b/api/src/main/java/com/datatorrent/api/Context.java
@@ -166,6 +166,13 @@ public interface Context
      */
     Attribute<Class<?>> TUPLE_CLASS = new Attribute<>(new Class2String<>());
 
+    /**
+     * Attribute of input port.
+     * This is a read-only attribute to query whether the input port is connected to a DelayOperator
+     * This is for iterative processing.
+     */
+    Attribute<Boolean> IS_CONNECTED_TO_DELAY_OPERATOR = new Attribute<>(false);
+
     @SuppressWarnings("FieldNameHidesFieldInSuperclass")
     long serialVersionUID = AttributeMap.AttributeInitializer.initialize(PortContext.class);
   }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/api/src/main/java/com/datatorrent/api/Operator.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Operator.java b/api/src/main/java/com/datatorrent/api/Operator.java
index 785c60b..d4a6a90 100644
--- a/api/src/main/java/com/datatorrent/api/Operator.java
+++ b/api/src/main/java/com/datatorrent/api/Operator.java
@@ -99,6 +99,25 @@ public interface Operator extends Component<OperatorContext>
   }
 
   /**
+   * DelayOperator is an operator of which the outgoing streaming window id is incremented by *one* by the
+   * engine, thus allowing loops in the "DAG". The output ports of a DelayOperator, if connected, *must*
+   * immediately connect to an upstream operator in the data flow path. Note that at least one output port of
+   * DelayOperator should be connected in order for the DelayOperator to serve its purpose.
+   *
+   * This is meant for iterative algorithms in the topology. A larger window increment can be simulated by an
+   * implementation of this interface.
+   */
+  interface DelayOperator extends Operator
+  {
+    /**
+     * This method gets called at the first window of the execution.
+     * The implementation is expected to emit tuples for initialization and/or
+     * recovery.
+     */
+    void firstWindow();
+  }
+
+  /**
    * A operator provides ports as a means to consume and produce data tuples.
    * Concrete ports implement derived interfaces.
    */

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/common/src/main/java/com/datatorrent/common/util/DefaultDelayOperator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/DefaultDelayOperator.java b/common/src/main/java/com/datatorrent/common/util/DefaultDelayOperator.java
new file mode 100644
index 0000000..ff676d4
--- /dev/null
+++ b/common/src/main/java/com/datatorrent/common/util/DefaultDelayOperator.java
@@ -0,0 +1,75 @@
+/**
+ * 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 com.datatorrent.common.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.Operator;
+
+/**
+ * DefaultDelayOperator. This is an implementation of the DelayOperator that has one input port and one output
+ * port, and does a simple pass-through from the input port to the output port, while recording the tuples in memory
+ * as checkpoint state.  Subclass of this operator can override this behavior by overriding processTuple(T tuple).
+ *
+ * Note that the engine automatically does a +1 on the output window ID since it is a DelayOperator.
+ *
+ * This DelayOperator provides no data loss during recovery, but it incurs a run-time cost per tuple, and all tuples
+ * of the checkpoint window will be part of the checkpoint state.
+ */
+public class DefaultDelayOperator<T> extends BaseOperator implements Operator.DelayOperator
+{
+  public transient DefaultInputPort<T> input = new DefaultInputPort<T>()
+  {
+    @Override
+    public void process(T tuple)
+    {
+      processTuple(tuple);
+    }
+  };
+
+  public transient DefaultOutputPort<T> output = new DefaultOutputPort<T>();
+
+  protected List<T> lastWindowTuples = new ArrayList<>();
+
+  protected void processTuple(T tuple)
+  {
+    lastWindowTuples.add(tuple);
+    output.emit(tuple);
+  }
+
+  @Override
+  public void beginWindow(long windowId)
+  {
+    lastWindowTuples.clear();
+  }
+
+  @Override
+  public void firstWindow()
+  {
+    for (T tuple : lastWindowTuples) {
+      output.emit(tuple);
+    }
+  }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/StramLocalCluster.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/StramLocalCluster.java b/engine/src/main/java/com/datatorrent/stram/StramLocalCluster.java
index 29e8e03..cda2a38 100644
--- a/engine/src/main/java/com/datatorrent/stram/StramLocalCluster.java
+++ b/engine/src/main/java/com/datatorrent/stram/StramLocalCluster.java
@@ -27,6 +27,7 @@ import java.net.SocketAddress;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -80,6 +81,7 @@ public class StramLocalCluster implements Runnable, Controller
   private boolean appDone = false;
   private final Map<String, StreamingContainer> injectShutdown = new ConcurrentHashMap<String, StreamingContainer>();
   private boolean heartbeatMonitoringEnabled = true;
+  private Callable<Boolean> exitCondition;
 
   public interface MockComponentFactory
   {
@@ -427,6 +429,11 @@ public class StramLocalCluster implements Runnable, Controller
     this.perContainerBufferServer = perContainerBufferServer;
   }
 
+  public void setExitCondition(Callable<Boolean> exitCondition)
+  {
+    this.exitCondition = exitCondition;
+  }
+
   @Override
   public void run()
   {
@@ -476,6 +483,14 @@ public class StramLocalCluster implements Runnable, Controller
         appDone = true;
       }
 
+      try {
+        if (exitCondition != null && exitCondition.call()) {
+          appDone = true;
+        }
+      } catch (Exception ex) {
+        break;
+      }
+
       if (Thread.interrupted()) {
         break;
       }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
index 4b79589..6233697 100644
--- a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
+++ b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
@@ -1008,7 +1008,7 @@ public class StreamingContainerManager implements PlanContext
       return operatorStatus.latencyMA.getAvg();
     }
     for (PTOperator.PTInput input : maxOperator.getInputs()) {
-      if (null != input.source.source) {
+      if (null != input.source.source && !input.delay) {
         operators.add(input.source.source);
       }
     }
@@ -1896,6 +1896,19 @@ public class StreamingContainerManager implements PlanContext
 
   }
 
+  private void addVisited(PTOperator operator, UpdateCheckpointsContext ctx)
+  {
+    ctx.visited.add(operator);
+    for (PTOperator.PTOutput out : operator.getOutputs()) {
+      for (PTOperator.PTInput sink : out.sinks) {
+        PTOperator sinkOperator = sink.target;
+        if (!ctx.visited.contains(sinkOperator)) {
+          addVisited(sinkOperator, ctx);
+        }
+      }
+    }
+  }
+
   /**
    * Compute checkpoints required for a given operator instance to be recovered.
    * This is done by looking at checkpoints available for downstream dependencies first,
@@ -1913,6 +1926,9 @@ public class StreamingContainerManager implements PlanContext
     if (operator.getState() == PTOperator.State.ACTIVE && (ctx.currentTms - operator.stats.lastWindowIdChangeTms) > operator.stats.windowProcessingTimeoutMillis) {
       // if the checkpoint is ahead, then it is not blocked but waiting for activation (state-less recovery, at-most-once)
       if (ctx.committedWindowId.longValue() >= operator.getRecoveryCheckpoint().windowId) {
+        LOG.debug("Marking operator {} blocked committed window {}, recovery window {}", operator,
+            Codec.getStringWindowId(ctx.committedWindowId.longValue()),
+            Codec.getStringWindowId(operator.getRecoveryCheckpoint().windowId));
         ctx.blocked.add(operator);
       }
     }
@@ -1922,25 +1938,30 @@ public class StreamingContainerManager implements PlanContext
       long currentWindowId = WindowGenerator.getWindowId(ctx.currentTms, this.vars.windowStartMillis, this.getLogicalPlan().getValue(LogicalPlan.STREAMING_WINDOW_SIZE_MILLIS));
       maxCheckpoint = currentWindowId;
     }
+    ctx.visited.add(operator);
 
     // DFS downstream operators
-    for (PTOperator.PTOutput out : operator.getOutputs()) {
-      for (PTOperator.PTInput sink : out.sinks) {
-        PTOperator sinkOperator = sink.target;
-        if (!ctx.visited.contains(sinkOperator)) {
-          // downstream traversal
-          updateRecoveryCheckpoints(sinkOperator, ctx);
-        }
-        // recovery window id cannot move backwards
-        // when dynamically adding new operators
-        if (sinkOperator.getRecoveryCheckpoint().windowId >= operator.getRecoveryCheckpoint().windowId) {
-          maxCheckpoint = Math.min(maxCheckpoint, sinkOperator.getRecoveryCheckpoint().windowId);
-        }
+    if (operator.getOperatorMeta().getOperator() instanceof Operator.DelayOperator) {
+      addVisited(operator, ctx);
+    } else {
+      for (PTOperator.PTOutput out : operator.getOutputs()) {
+        for (PTOperator.PTInput sink : out.sinks) {
+          PTOperator sinkOperator = sink.target;
+          if (!ctx.visited.contains(sinkOperator)) {
+            // downstream traversal
+            updateRecoveryCheckpoints(sinkOperator, ctx);
+          }
+          // recovery window id cannot move backwards
+          // when dynamically adding new operators
+          if (sinkOperator.getRecoveryCheckpoint().windowId >= operator.getRecoveryCheckpoint().windowId) {
+            maxCheckpoint = Math.min(maxCheckpoint, sinkOperator.getRecoveryCheckpoint().windowId);
+          }
 
-        if (ctx.blocked.contains(sinkOperator)) {
-          if (sinkOperator.stats.getCurrentWindowId() == operator.stats.getCurrentWindowId()) {
-            // downstream operator is blocked by this operator
-            ctx.blocked.remove(sinkOperator);
+          if (ctx.blocked.contains(sinkOperator)) {
+            if (sinkOperator.stats.getCurrentWindowId() == operator.stats.getCurrentWindowId()) {
+              // downstream operator is blocked by this operator
+              ctx.blocked.remove(sinkOperator);
+            }
           }
         }
       }
@@ -1975,7 +1996,6 @@ public class StreamingContainerManager implements PlanContext
       LOG.debug("Skipping checkpoint update {} during {}", operator, operator.getState());
     }
 
-    ctx.visited.add(operator);
   }
 
   public long windowIdToMillis(long windowId)

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java b/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java
index 93cee49..4777f93 100644
--- a/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java
+++ b/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java
@@ -34,11 +34,14 @@ import com.datatorrent.api.Operator.ShutdownException;
 import com.datatorrent.api.Sink;
 import com.datatorrent.api.annotation.Stateless;
 
+import com.datatorrent.bufferserver.packet.MessageType;
 import com.datatorrent.bufferserver.util.Codec;
 import com.datatorrent.netlet.util.DTThrowable;
 import com.datatorrent.netlet.util.CircularBuffer;
 import com.datatorrent.stram.api.StreamingContainerUmbilicalProtocol.ContainerStats;
 import com.datatorrent.stram.debug.TappedReservoir;
+import com.datatorrent.stram.plan.logical.Operators;
+import com.datatorrent.stram.tuple.ResetWindowTuple;
 import com.datatorrent.stram.tuple.Tuple;
 
 /**
@@ -198,6 +201,15 @@ public class GenericNode extends Node<Operator>
     insideWindow = applicationWindowCount != 0;
   }
 
+  private boolean isInputPortConnectedToDelayOperator(String portName)
+  {
+    Operators.PortContextPair<InputPort<?>> pcPair = descriptor.inputPorts.get(portName);
+    if (pcPair == null || pcPair.context == null) {
+      return false;
+    }
+    return pcPair.context.getValue(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR);
+  }
+
   /**
    * Originally this method was defined in an attempt to implement the interface Runnable.
    *
@@ -212,30 +224,67 @@ public class GenericNode extends Node<Operator>
     long spinMillis = context.getValue(OperatorContext.SPIN_MILLIS);
     final boolean handleIdleTime = operator instanceof IdleTimeHandler;
     int totalQueues = inputs.size();
+    int regularQueues = totalQueues;
+    // regularQueues is the number of queues that are not connected to a DelayOperator
+    for (String portName : inputs.keySet()) {
+      if (isInputPortConnectedToDelayOperator(portName)) {
+        regularQueues--;
+      }
+    }
 
-    ArrayList<SweepableReservoir> activeQueues = new ArrayList<SweepableReservoir>();
-    activeQueues.addAll(inputs.values());
+    ArrayList<Map.Entry<String, SweepableReservoir>> activeQueues = new ArrayList<>();
+    activeQueues.addAll(inputs.entrySet());
 
     int expectingBeginWindow = activeQueues.size();
     int receivedEndWindow = 0;
+    long firstWindowId = -1;
 
     TupleTracker tracker;
     LinkedList<TupleTracker> resetTupleTracker = new LinkedList<TupleTracker>();
-
     try {
       do {
-        Iterator<SweepableReservoir> buffers = activeQueues.iterator();
+        Iterator<Map.Entry<String, SweepableReservoir>> buffers = activeQueues.iterator();
   activequeue:
         while (buffers.hasNext()) {
-          SweepableReservoir activePort = buffers.next();
+          Map.Entry<String, SweepableReservoir> activePortEntry = buffers.next();
+          SweepableReservoir activePort = activePortEntry.getValue();
           Tuple t = activePort.sweep();
           if (t != null) {
+            boolean delay = (operator instanceof Operator.DelayOperator);
+            long windowAhead = 0;
+            if (delay) {
+              windowAhead = WindowGenerator.getAheadWindowId(t.getWindowId(), firstWindowMillis, windowWidthMillis, 1);
+            }
             switch (t.getType()) {
               case BEGIN_WINDOW:
                 if (expectingBeginWindow == totalQueues) {
+                  // This is the first begin window tuple among all ports
+                  if (isInputPortConnectedToDelayOperator(activePortEntry.getKey())) {
+                    // We need to wait for the first BEGIN_WINDOW from a port not connected to DelayOperator before
+                    // we can do anything with it, because otherwise if a CHECKPOINT tuple arrives from
+                    // upstream after the BEGIN_WINDOW tuple for the next window from the delay operator, it would end
+                    // up checkpointing in the middle of the window.  This code is assuming we have at least one
+                    // input port that is not connected to a DelayOperator, and we might have to change this later.
+                    // In the future, this condition will not be needed if we get rid of the CHECKPOINT tuple.
+                    continue;
+                  }
                   activePort.remove();
                   expectingBeginWindow--;
+                  receivedEndWindow = 0;
                   currentWindowId = t.getWindowId();
+                  if (delay) {
+                    if (WindowGenerator.getBaseSecondsFromWindowId(windowAhead) > t.getBaseSeconds()) {
+                      // Buffer server code strips out the base seconds from BEGIN_WINDOW and END_WINDOW tuples for
+                      // serialization optimization.  That's why we need a reset window here to tell the buffer
+                      // server we are having a new baseSeconds now.
+                      Tuple resetWindowTuple = new ResetWindowTuple(windowAhead);
+                      for (int s = sinks.length; s-- > 0; ) {
+                        sinks[s].put(resetWindowTuple);
+                      }
+                      controlTupleCount++;
+                    }
+                    t.setWindowId(windowAhead);
+                  }
                   for (int s = sinks.length; s-- > 0; ) {
                     sinks[s].put(t);
                   }
@@ -245,7 +294,6 @@ public class GenericNode extends Node<Operator>
                     insideWindow = true;
                     operator.beginWindow(currentWindowId);
                   }
-                  receivedEndWindow = 0;
                 }
                 else if (t.getWindowId() == currentWindowId) {
                   activePort.remove();
@@ -253,17 +301,7 @@ public class GenericNode extends Node<Operator>
                 }
                 else {
                   buffers.remove();
-
-                  /* find the name of the port which got out of sequence tuple */
-                  String port = null;
-                  for (Entry<String, SweepableReservoir> e : inputs.entrySet()) {
-                    if (e.getValue() == activePort) {
-                      port = e.getKey();
-                    }
-                  }
-
-                  assert (port != null); /* we should always find the port */
-
+                  String port = activePortEntry.getKey();
                   if (PROCESSING_MODE == ProcessingMode.AT_MOST_ONCE) {
                     if (t.getWindowId() < currentWindowId) {
                       /*
@@ -279,21 +317,21 @@ public class GenericNode extends Node<Operator>
                       WindowIdActivatedReservoir wiar = new WindowIdActivatedReservoir(port, activePort, currentWindowId);
                       wiar.setSink(sink);
                       inputs.put(port, wiar);
-                      activeQueues.add(wiar);
+                      activeQueues.add(new AbstractMap.SimpleEntry<String, SweepableReservoir>(port, wiar));
                       break activequeue;
                     }
                     else {
                       expectingBeginWindow--;
                       if (++receivedEndWindow == totalQueues) {
                         processEndWindow(null);
-                        activeQueues.addAll(inputs.values());
+                        activeQueues.addAll(inputs.entrySet());
                         expectingBeginWindow = activeQueues.size();
                         break activequeue;
                       }
                     }
                   }
                   else {
-                    logger.error("Catastrophic Error: Out of sequence tuple {} on port {} while expecting {}", Codec.getStringWindowId(t.getWindowId()), port, Codec.getStringWindowId(currentWindowId));
+                    logger.error("Catastrophic Error: Out of sequence {} tuple {} on port {} while expecting {}", t.getType(), Codec.getStringWindowId(t.getWindowId()), port, Codec.getStringWindowId(currentWindowId));
                     System.exit(2);
                   }
                 }
@@ -306,8 +344,11 @@ public class GenericNode extends Node<Operator>
                   endWindowDequeueTimes.put(activePort, System.currentTimeMillis());
                   if (++receivedEndWindow == totalQueues) {
                     assert (activeQueues.isEmpty());
+                    if (delay) {
+                      t.setWindowId(windowAhead);
+                    }
                     processEndWindow(t);
-                    activeQueues.addAll(inputs.values());
+                    activeQueues.addAll(inputs.entrySet());
                     expectingBeginWindow = activeQueues.size();
                     break activequeue;
                   }
@@ -330,11 +371,12 @@ public class GenericNode extends Node<Operator>
                       doCheckpoint = true;
                     }
                   }
-
-                  for (int s = sinks.length; s-- > 0; ) {
-                    sinks[s].put(t);
+                  if (!delay) {
+                    for (int s = sinks.length; s-- > 0; ) {
+                      sinks[s].put(t);
+                    }
+                    controlTupleCount++;
                   }
-                  controlTupleCount++;
                 }
                 break;
 
@@ -343,12 +385,14 @@ public class GenericNode extends Node<Operator>
                  * we will receive tuples which are equal to the number of input streams.
                  */
                 activePort.remove();
-                buffers.remove();
+                if (isInputPortConnectedToDelayOperator(activePortEntry.getKey())) {
+                  break; // breaking out of the switch/case
+                }
 
+                buffers.remove();
                 int baseSeconds = t.getBaseSeconds();
                 tracker = null;
-                Iterator<TupleTracker> trackerIterator = resetTupleTracker.iterator();
-                while (trackerIterator.hasNext()) {
+                for (Iterator<TupleTracker> trackerIterator = resetTupleTracker.iterator(); trackerIterator.hasNext(); ) {
                   tracker = trackerIterator.next();
                   if (tracker.tuple.getBaseSeconds() == baseSeconds) {
                     break;
@@ -356,7 +400,7 @@ public class GenericNode extends Node<Operator>
                 }
 
                 if (tracker == null) {
-                  tracker = new TupleTracker(t, totalQueues);
+                  tracker = new TupleTracker(t, regularQueues);
                   resetTupleTracker.add(tracker);
                 }
                 int trackerIndex = 0;
@@ -364,29 +408,50 @@ public class GenericNode extends Node<Operator>
                   if (tracker.ports[trackerIndex] == null) {
                     tracker.ports[trackerIndex++] = activePort;
                     break;
-                  }
-                  else if (tracker.ports[trackerIndex] == activePort) {
+                  } else if (tracker.ports[trackerIndex] == activePort) {
                     break;
                   }
 
                   trackerIndex++;
                 }
 
-                if (trackerIndex == totalQueues) {
-                  trackerIterator = resetTupleTracker.iterator();
+                if (trackerIndex == regularQueues) {
+                  Iterator<TupleTracker> trackerIterator = resetTupleTracker.iterator();
                   while (trackerIterator.hasNext()) {
                     if (trackerIterator.next().tuple.getBaseSeconds() <= baseSeconds) {
                       trackerIterator.remove();
                     }
                   }
-                  for (int s = sinks.length; s-- > 0; ) {
-                    sinks[s].put(t);
+                  if (!delay) {
+                    for (int s = sinks.length; s-- > 0; ) {
+                      sinks[s].put(t);
+                    }
+                    controlTupleCount++;
                   }
-                  controlTupleCount++;
-
-                  assert (activeQueues.isEmpty());
-                  activeQueues.addAll(inputs.values());
+                  if (!activeQueues.isEmpty()) {
+                    // make sure they are all queues from DelayOperator
+                    for (Map.Entry<String, SweepableReservoir> entry : activeQueues) {
+                      if (!isInputPortConnectedToDelayOperator(entry.getKey())) {
+                        assert (false);
+                      }
+                    }
+                    activeQueues.clear();
+                  }
+                  activeQueues.addAll(inputs.entrySet());
                   expectingBeginWindow = activeQueues.size();
+
+                  if (firstWindowId == -1) {
+                    if (delay) {
+                      for (int s = sinks.length; s-- > 0; ) {
+                        sinks[s].put(t);
+                      }
+                      controlTupleCount++;
+                      // if it's a DelayOperator and this is the first RESET_WINDOW (start) or END_STREAM
+                      // (recovery), fabricate the first window
+                      fabricateFirstWindow((Operator.DelayOperator)operator, windowAhead);
+                    }
+                    firstWindowId = t.getWindowId();
+                  }
                   break activequeue;
                 }
                 break;
@@ -394,6 +459,15 @@ public class GenericNode extends Node<Operator>
               case END_STREAM:
                 activePort.remove();
                 buffers.remove();
+                if (firstWindowId == -1) {
+                  // this is for recovery from a checkpoint for DelayOperator
+                  if (delay) {
+                    // if it's a DelayOperator and this is the first RESET_WINDOW (start) or END_STREAM (recovery),
+                    // fabricate the first window
+                    fabricateFirstWindow((Operator.DelayOperator)operator, windowAhead);
+                  }
+                  firstWindowId = t.getWindowId();
+                }
                 for (Iterator<Entry<String, SweepableReservoir>> it = inputs.entrySet().iterator(); it.hasNext(); ) {
                   Entry<String, SweepableReservoir> e = it.next();
                   if (e.getValue() == activePort) {
@@ -409,7 +483,7 @@ public class GenericNode extends Node<Operator>
                       if (e.getKey().equals(dic.portname)) {
                         connectInputPort(dic.portname, dic.reservoir);
                         dici.remove();
-                        activeQueues.add(dic.reservoir);
+                        activeQueues.add(new AbstractMap.SimpleEntry<>(dic.portname, dic.reservoir));
                         break activequeue;
                       }
                     }
@@ -427,17 +501,18 @@ public class GenericNode extends Node<Operator>
                  * Since one of the operators we care about it gone, we should relook at our ports.
                  * We need to make sure that the END_STREAM comes outside of the window.
                  */
+                regularQueues--;
                 totalQueues--;
 
                 boolean break_activequeue = false;
-                if (totalQueues == 0) {
+                if (regularQueues == 0) {
                   alive = false;
                   break_activequeue = true;
                 }
                 else if (activeQueues.isEmpty()) {
                   assert (!inputs.isEmpty());
                   processEndWindow(null);
-                  activeQueues.addAll(inputs.values());
+                  activeQueues.addAll(inputs.entrySet());
                   expectingBeginWindow = activeQueues.size();
                   break_activequeue = true;
                 }
@@ -450,22 +525,22 @@ public class GenericNode extends Node<Operator>
                  * it's the only one which has not, then we consider it delivered and release the reset tuple downstream.
                  */
                 Tuple tuple = null;
-                for (trackerIterator = resetTupleTracker.iterator(); trackerIterator.hasNext(); ) {
+                for (Iterator<TupleTracker> trackerIterator = resetTupleTracker.iterator(); trackerIterator.hasNext(); ) {
                   tracker = trackerIterator.next();
 
                   trackerIndex = 0;
                   while (trackerIndex < tracker.ports.length) {
                     if (tracker.ports[trackerIndex] == activePort) {
-                      SweepableReservoir[] ports = new SweepableReservoir[totalQueues];
+                      SweepableReservoir[] ports = new SweepableReservoir[regularQueues];
                       System.arraycopy(tracker.ports, 0, ports, 0, trackerIndex);
-                      if (trackerIndex < totalQueues) {
+                      if (trackerIndex < regularQueues) {
                         System.arraycopy(tracker.ports, trackerIndex + 1, ports, trackerIndex, tracker.ports.length - trackerIndex - 1);
                       }
                       tracker.ports = ports;
                       break;
                     }
                     else if (tracker.ports[trackerIndex] == null) {
-                      if (trackerIndex == totalQueues) { /* totalQueues is already adjusted above */
+                      if (trackerIndex == regularQueues) { /* regularQueues is already adjusted above */
                         if (tuple == null || tuple.getBaseSeconds() < tracker.tuple.getBaseSeconds()) {
                           tuple = tracker.tuple;
                         }
@@ -475,7 +550,7 @@ public class GenericNode extends Node<Operator>
                       break;
                     }
                     else {
-                      tracker.ports = Arrays.copyOf(tracker.ports, totalQueues);
+                      tracker.ports = Arrays.copyOf(tracker.ports, regularQueues);
                     }
 
                     trackerIndex++;
@@ -485,7 +560,7 @@ public class GenericNode extends Node<Operator>
                 /*
                  * Since we were waiting for a reset tuple on this stream, we should not any longer.
                  */
-                if (tuple != null) {
+                if (tuple != null && !delay) {
                   for (int s = sinks.length; s-- > 0; ) {
                     sinks[s].put(tuple);
                   }
@@ -509,8 +584,8 @@ public class GenericNode extends Node<Operator>
         }
         else {
           boolean need2sleep = true;
-          for (SweepableReservoir cb : activeQueues) {
-            if (cb.size() > 0) {
+          for (Map.Entry<String, SweepableReservoir> cb : activeQueues) {
+            if (cb.getValue().size() > 0) {
               need2sleep = false;
               break;
             }
@@ -582,6 +657,21 @@ public class GenericNode extends Node<Operator>
 
   }
 
+  private void fabricateFirstWindow(Operator.DelayOperator delayOperator, long windowAhead)
+  {
+    Tuple beginWindowTuple = new Tuple(MessageType.BEGIN_WINDOW, windowAhead);
+    Tuple endWindowTuple = new Tuple(MessageType.END_WINDOW, windowAhead);
+    for (Sink<Object> sink : outputs.values()) {
+      sink.put(beginWindowTuple);
+    }
+    controlTupleCount++;
+    delayOperator.firstWindow();
+    for (Sink<Object> sink : outputs.values()) {
+      sink.put(endWindowTuple);
+    }
+    controlTupleCount++;
+  }
+
   /**
    * End window dequeue times may not have been saved for all the input ports during deactivate,
    * so save them for reporting. SPOI-1324.

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/engine/Node.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/engine/Node.java b/engine/src/main/java/com/datatorrent/stram/engine/Node.java
index 068a325..d4970cd 100644
--- a/engine/src/main/java/com/datatorrent/stram/engine/Node.java
+++ b/engine/src/main/java/com/datatorrent/stram/engine/Node.java
@@ -126,6 +126,8 @@ public abstract class Node<OPERATOR extends Operator> implements Component<Opera
   private ExecutorService executorService;
   private Queue<Pair<FutureTask<Stats.CheckpointStats>, CheckpointWindowInfo>> taskQueue;
   protected Stats.CheckpointStats checkpointStats;
+  public long firstWindowMillis;
+  public long windowWidthMillis;
 
   public Node(OPERATOR operator, OperatorContext context)
   {
@@ -354,7 +356,9 @@ public abstract class Node<OPERATOR extends Operator> implements Component<Opera
 
   protected void emitEndWindow()
   {
-    EndWindowTuple ewt = new EndWindowTuple(currentWindowId);
+    long windowId = (operator instanceof Operator.DelayOperator) ?
+        WindowGenerator.getAheadWindowId(currentWindowId, firstWindowMillis, windowWidthMillis, 1) : currentWindowId;
+    EndWindowTuple ewt = new EndWindowTuple(windowId);
     for (int s = sinks.length; s-- > 0; ) {
       sinks[s].put(ewt);
     }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/engine/StreamingContainer.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/engine/StreamingContainer.java b/engine/src/main/java/com/datatorrent/stram/engine/StreamingContainer.java
index 14e00a9..79d9037 100644
--- a/engine/src/main/java/com/datatorrent/stram/engine/StreamingContainer.java
+++ b/engine/src/main/java/com/datatorrent/stram/engine/StreamingContainer.java
@@ -894,6 +894,8 @@ public class StreamingContainer extends YarnContainerMain
       Node<?> node = Node.retrieveNode(backupAgent.load(ndi.id, ctx.stateless ? Stateless.WINDOW_ID : ndi.checkpoint.windowId), ctx, ndi.type);
       node.currentWindowId = ndi.checkpoint.windowId;
       node.applicationWindowCount = ndi.checkpoint.applicationWindowCount;
+      node.firstWindowMillis = firstWindowMillis;
+      node.windowWidthMillis = windowWidthMillis;
 
       node.setId(ndi.id);
       nodes.put(ndi.id, node);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/engine/WindowGenerator.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/engine/WindowGenerator.java b/engine/src/main/java/com/datatorrent/stram/engine/WindowGenerator.java
index 5610112..ea429af 100644
--- a/engine/src/main/java/com/datatorrent/stram/engine/WindowGenerator.java
+++ b/engine/src/main/java/com/datatorrent/stram/engine/WindowGenerator.java
@@ -314,13 +314,25 @@ public class WindowGenerator extends MuxReservoir implements Stream, Runnable
     long baseMillis = (windowId >> 32) * 1000;
     long diff = baseMillis - firstWindowMillis;
     long baseChangeInterval = windowWidthMillis * (WindowGenerator.MAX_WINDOW_ID + 1);
+    assert (baseChangeInterval > 0);
     long multiplier = diff / baseChangeInterval;
     if (diff % baseChangeInterval > 0) {
       multiplier++;
     }
     assert (multiplier >= 0);
     windowId = windowId & WindowGenerator.WINDOW_MASK;
-    return firstWindowMillis + (multiplier * windowWidthMillis * (WindowGenerator.MAX_WINDOW_ID + 1)) + windowId * windowWidthMillis;
+    return firstWindowMillis + (multiplier * baseChangeInterval) + (windowId * windowWidthMillis);
+  }
+
+  /**
+   * Utility function to get the base seconds from a window id
+   *
+   * @param windowId
+   * @return the base seconds for the given window id
+   */
+  public static long getBaseSecondsFromWindowId(long windowId)
+  {
+    return windowId >>> 32;
   }
 
   private class MasterReservoir extends CircularBuffer<Tuple> implements Reservoir

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index 867f814..3c26118 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -1758,6 +1758,14 @@ public class LogicalPlan implements Serializable, DAG
       throw new ValidationException("Loops in graph: " + cycles);
     }
 
+    List<List<String>> invalidDelays = new ArrayList<>();
+    for (OperatorMeta n : rootOperators) {
+      findInvalidDelays(n, invalidDelays);
+    }
+    if (!invalidDelays.isEmpty()) {
+      throw new ValidationException("Invalid delays in graph: " + invalidDelays);
+    }
+
     for (StreamMeta s: streams.values()) {
       if (s.source == null) {
         throw new ValidationException("Stream source not connected: " + s.getName());
@@ -1814,6 +1822,11 @@ public class LogicalPlan implements Serializable, DAG
       return;
     }
 
+    if (om.getOperator() instanceof Operator.DelayOperator) {
+      String msg = String.format("Locality %s invalid for delay operator %s", Locality.THREAD_LOCAL, om);
+      throw new ValidationException(msg);
+    }
+
     for (StreamMeta sm: om.inputStreams.values()){
       // validation fail as each input stream should be OIO
       if (sm.locality != Locality.THREAD_LOCAL){
@@ -1822,6 +1835,10 @@ public class LogicalPlan implements Serializable, DAG
         throw new ValidationException(msg);
       }
 
+      if (sm.source.operatorMeta.getOperator() instanceof Operator.DelayOperator) {
+        String msg = String.format("Locality %s invalid for delay operator %s", Locality.THREAD_LOCAL, sm.source.operatorMeta);
+        throw new ValidationException(msg);
+      }
       // gets oio root for input operator for the stream
       Integer oioStreamRoot = getOioRoot(sm.source.operatorMeta);
 
@@ -1895,6 +1912,11 @@ public class LogicalPlan implements Serializable, DAG
     // depth first successors traversal
     for (StreamMeta downStream: om.outputStreams.values()) {
       for (InputPortMeta sink: downStream.sinks) {
+        if (om.getOperator() instanceof Operator.DelayOperator) {
+          // this is an iteration loop, do not treat it as downstream when detecting cycles
+          sink.attributes.put(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR, true);
+          continue;
+        }
         OperatorMeta successor = sink.getOperatorWrapper();
         if (successor == null) {
           continue;
@@ -1932,6 +1954,37 @@ public class LogicalPlan implements Serializable, DAG
     }
   }
 
+  public void findInvalidDelays(OperatorMeta om, List<List<String>> invalidDelays)
+  {
+    stack.push(om);
+
+    // depth first successors traversal
+    boolean isDelayOperator = om.getOperator() instanceof Operator.DelayOperator;
+    if (isDelayOperator) {
+      if (om.getValue(OperatorContext.APPLICATION_WINDOW_COUNT) != 1) {
+        LOG.debug("detected DelayOperator having APPLICATION_WINDOW_COUNT not equal to 1");
+        invalidDelays.add(Collections.singletonList(om.getName()));
+      }
+    }
+
+    for (StreamMeta downStream: om.outputStreams.values()) {
+      for (InputPortMeta sink : downStream.sinks) {
+        OperatorMeta successor = sink.getOperatorWrapper();
+        if (isDelayOperator) {
+          // Check whether all downstream operators are already visited in the path
+          if (successor != null && !stack.contains(successor)) {
+            LOG.debug("detected DelayOperator does not immediately output to a visited operator {}.{}->{}.{}",
+                om.getName(), downStream.getSource().getPortName(), successor.getName(), sink.getPortName());
+            invalidDelays.add(Arrays.asList(om.getName(), successor.getName()));
+          }
+        } else {
+          findInvalidDelays(successor, invalidDelays);
+        }
+      }
+    }
+    stack.pop();
+  }
+
   private void validateProcessingMode(OperatorMeta om, Set<OperatorMeta> visited)
   {
     for (StreamMeta is : om.getInputStreams().values()) {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/plan/physical/PTOperator.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/physical/PTOperator.java b/engine/src/main/java/com/datatorrent/stram/plan/physical/PTOperator.java
index 6adfd64..ae276d8 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/physical/PTOperator.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/physical/PTOperator.java
@@ -81,6 +81,7 @@ public class PTOperator implements java.io.Serializable
     public final PartitionKeys partitions;
     public final PTOutput source;
     public final String portName;
+    public final boolean delay;
 
     /**
      *
@@ -90,7 +91,7 @@ public class PTOperator implements java.io.Serializable
      * @param partitions
      * @param source
      */
-    protected PTInput(String portName, StreamMeta logicalStream, PTOperator target, PartitionKeys partitions, PTOutput source)
+    protected PTInput(String portName, StreamMeta logicalStream, PTOperator target, PartitionKeys partitions, PTOutput source, boolean delay)
     {
       this.logicalStream = logicalStream;
       this.target = target;
@@ -98,6 +99,7 @@ public class PTOperator implements java.io.Serializable
       this.source = source;
       this.portName = portName;
       this.source.sinks.add(this);
+      this.delay = delay;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
index 829a6fd..da96ef3 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
@@ -328,8 +328,11 @@ public class PhysicalPlan implements Serializable
 
       boolean upstreamDeployed = true;
 
-      for (StreamMeta s : n.getInputStreams().values()) {
-        if (s.getSource() != null && !this.logicalToPTOperator.containsKey(s.getSource().getOperatorMeta())) {
+      for (Map.Entry<InputPortMeta, StreamMeta> entry : n.getInputStreams().entrySet()) {
+        StreamMeta s = entry.getValue();
+        boolean delay = entry.getKey().getValue(LogicalPlan.IS_CONNECTED_TO_DELAY_OPERATOR);
+        // skip delay sources since it's going to be handled as downstream
+        if (!delay && s.getSource() != null && !this.logicalToPTOperator.containsKey(s.getSource().getOperatorMeta())) {
           pendingNodes.push(n);
           pendingNodes.push(s.getSource().getOperatorMeta());
           upstreamDeployed = false;
@@ -907,7 +910,10 @@ public class PhysicalPlan implements Serializable
 
     for (Map.Entry<InputPortMeta, StreamMeta> ipm : m.logicalOperator.getInputStreams().entrySet()) {
       PMapping sourceMapping = this.logicalToPTOperator.get(ipm.getValue().getSource().getOperatorMeta());
-
+      if (ipm.getValue().getSource().getOperatorMeta().getOperator() instanceof Operator.DelayOperator) {
+        // skip if the source is a DelayOperator
+        continue;
+      }
       if (ipm.getKey().getValue(PortContext.PARTITION_PARALLEL)) {
         if (sourceMapping.partitions.size() < m.partitions.size()) {
           throw new AssertionError("Number of partitions don't match in parallel mapping " + sourceMapping.logicalOperator.getName() + " -> " + m.logicalOperator.getName() + ", " + sourceMapping.partitions.size() + " -> " + m.partitions.size());
@@ -942,11 +948,11 @@ public class PhysicalPlan implements Serializable
                 PTOperator slidingUnifier = StreamMapping.createSlidingUnifier(sourceOut.logicalStream, this,
                   sourceOM.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT), slidingWindowCount);
                 StreamMapping.addInput(slidingUnifier, sourceOut, null);
-                input = new PTInput(ipm.getKey().getPortName(), ipm.getValue(), oper, null, slidingUnifier.outputs.get(0));
+                input = new PTInput(ipm.getKey().getPortName(), ipm.getValue(), oper, null, slidingUnifier.outputs.get(0), ipm.getKey().getValue(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR));
                 sourceMapping.outputStreams.get(ipm.getValue().getSource()).slidingUnifiers.add(slidingUnifier);
               }
               else {
-                input = new PTInput(ipm.getKey().getPortName(), ipm.getValue(), oper, null, sourceOut);
+                input = new PTInput(ipm.getKey().getPortName(), ipm.getValue(), oper, null, sourceOut, ipm.getKey().getValue(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR));
               }
               oper.inputs.add(input);
             }
@@ -1445,6 +1451,9 @@ public class PhysicalPlan implements Serializable
     PMapping upstreamPartitioned = null;
 
     for (Map.Entry<LogicalPlan.InputPortMeta, StreamMeta> e : om.getInputStreams().entrySet()) {
+      if (e.getValue().getSource().getOperatorMeta().getOperator() instanceof Operator.DelayOperator) {
+        continue;
+      }
       PMapping m = logicalToPTOperator.get(e.getValue().getSource().getOperatorMeta());
       if (e.getKey().getValue(PortContext.PARTITION_PARALLEL).equals(true)) {
         // operator partitioned with upstream

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java b/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java
index d42c327..91c6eef 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java
@@ -347,7 +347,7 @@ public class StreamMapping implements java.io.Serializable
     // link to upstream output(s) for this stream
     for (PTOutput upstreamOut : sourceOper.outputs) {
       if (upstreamOut.logicalStream == streamMeta) {
-        PTInput input = new PTInput(ipm.getPortName(), streamMeta, oper, pks, upstreamOut);
+        PTInput input = new PTInput(ipm.getPortName(), streamMeta, oper, pks, upstreamOut, ipm.getValue(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR));
         oper.inputs.add(input);
       }
     }
@@ -356,7 +356,7 @@ public class StreamMapping implements java.io.Serializable
   public static void addInput(PTOperator target, PTOutput upstreamOut, PartitionKeys pks)
   {
     StreamMeta lStreamMeta = upstreamOut.logicalStream;
-    PTInput input = new PTInput("<merge#" + lStreamMeta.getSource().getPortName() + ">", lStreamMeta, target, pks, upstreamOut);
+    PTInput input = new PTInput("<merge#" + lStreamMeta.getSource().getPortName() + ">", lStreamMeta, target, pks, upstreamOut, false);
     target.inputs.add(input);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/main/java/com/datatorrent/stram/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/tuple/Tuple.java b/engine/src/main/java/com/datatorrent/stram/tuple/Tuple.java
index 23c197b..9191b65 100644
--- a/engine/src/main/java/com/datatorrent/stram/tuple/Tuple.java
+++ b/engine/src/main/java/com/datatorrent/stram/tuple/Tuple.java
@@ -52,6 +52,11 @@ public class Tuple
     return windowId;
   }
 
+  public void setWindowId(long windowId)
+  {
+    this.windowId = windowId;
+  }
+
   public final int getBaseSeconds()
   {
     return (int)(windowId >> 32);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/test/java/com/datatorrent/stram/debug/TupleRecorderTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/debug/TupleRecorderTest.java b/engine/src/test/java/com/datatorrent/stram/debug/TupleRecorderTest.java
index 3f97b54..1c17d68 100644
--- a/engine/src/test/java/com/datatorrent/stram/debug/TupleRecorderTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/debug/TupleRecorderTest.java
@@ -76,8 +76,7 @@ public class TupleRecorderTest
   public TupleRecorder getTupleRecorder(final StramLocalCluster localCluster, final PTOperator op)
   {
     TupleRecorderCollection instance = (TupleRecorderCollection)localCluster.getContainer(op).getInstance(classname);
-    TupleRecorder tupleRecorder = instance.getTupleRecorder(op.getId(), null);
-    return tupleRecorder;
+    return instance.getTupleRecorder(op.getId(), null);
   }
 
   public class Tuple
@@ -89,8 +88,7 @@ public class TupleRecorderTest
   @Test
   public void testRecorder() throws IOException
   {
-    FileSystem fs = new LocalFileSystem();
-    try {
+    try (FileSystem fs = new LocalFileSystem()) {
       TupleRecorder recorder = new TupleRecorder(null, "application_test_id_1");
       recorder.getStorage().setBytesPerPartFile(4096);
       recorder.getStorage().setLocalMode(true);
@@ -132,80 +130,76 @@ public class TupleRecorderTest
 
       fs.initialize((new Path(recorder.getStorage().getBasePath()).toUri()), new Configuration());
       Path path;
-      FSDataInputStream is;
       String line;
-      BufferedReader br;
 
       path = new Path(recorder.getStorage().getBasePath(), FSPartFileCollection.INDEX_FILE);
-      is = fs.open(path);
-      br = new BufferedReader(new InputStreamReader(is));
-
-      line = br.readLine();
-      //    Assert.assertEquals("check index", "B:1000:T:0:part0.txt", line);
-      Assert.assertTrue("check index", line.matches("F:part0.txt:\\d+-\\d+:4:T:1000-1000:33:\\{\"3\":\"1\",\"1\":\"1\",\"0\":\"1\",\"2\":\"1\"\\}"));
+      try (FSDataInputStream is = fs.open(path);
+          BufferedReader br = new BufferedReader(new InputStreamReader(is))) {
 
+        line = br.readLine();
+        //    Assert.assertEquals("check index", "B:1000:T:0:part0.txt", line);
+        Assert.assertTrue("check index", line
+            .matches("F:part0.txt:\\d+-\\d+:4:T:1000-1000:33:\\{\"3\":\"1\",\"1\":\"1\",\"0\":\"1\",\"2\":\"1\"\\}"));
+      }
       path = new Path(recorder.getStorage().getBasePath(), FSPartFileCollection.META_FILE);
-      is = fs.open(path);
-      br = new BufferedReader(new InputStreamReader(is));
-
-      ObjectMapper mapper = new ObjectMapper();
-      line = br.readLine();
-      Assert.assertEquals("check version", "1.2", line);
-      br.readLine(); // RecordInfo
-      //RecordInfo ri = mapper.readValue(line, RecordInfo.class);
-      line = br.readLine();
-      PortInfo pi = mapper.readValue(line, PortInfo.class);
-      Assert.assertEquals("port1", recorder.getPortInfoMap().get(pi.name).id, pi.id);
-      Assert.assertEquals("port1", recorder.getPortInfoMap().get(pi.name).type, pi.type);
-      line = br.readLine();
-      pi = mapper.readValue(line, PortInfo.class);
-      Assert.assertEquals("port2", recorder.getPortInfoMap().get(pi.name).id, pi.id);
-      Assert.assertEquals("port2", recorder.getPortInfoMap().get(pi.name).type, pi.type);
-      line = br.readLine();
-      pi = mapper.readValue(line, PortInfo.class);
-      Assert.assertEquals("port3", recorder.getPortInfoMap().get(pi.name).id, pi.id);
-      Assert.assertEquals("port3", recorder.getPortInfoMap().get(pi.name).type, pi.type);
-      line = br.readLine();
-      pi = mapper.readValue(line, PortInfo.class);
-      Assert.assertEquals("port4", recorder.getPortInfoMap().get(pi.name).id, pi.id);
-      Assert.assertEquals("port4", recorder.getPortInfoMap().get(pi.name).type, pi.type);
-      Assert.assertEquals("port size", 4, recorder.getPortInfoMap().size());
-      //line = br.readLine();
-
+      try (FSDataInputStream is = fs.open(path);
+          BufferedReader br = new BufferedReader(new InputStreamReader(is))) {
+
+        ObjectMapper mapper = new ObjectMapper();
+        line = br.readLine();
+        Assert.assertEquals("check version", "1.2", line);
+        br.readLine(); // RecordInfo
+        //RecordInfo ri = mapper.readValue(line, RecordInfo.class);
+        line = br.readLine();
+        PortInfo pi = mapper.readValue(line, PortInfo.class);
+        Assert.assertEquals("port1", recorder.getPortInfoMap().get(pi.name).id, pi.id);
+        Assert.assertEquals("port1", recorder.getPortInfoMap().get(pi.name).type, pi.type);
+        line = br.readLine();
+        pi = mapper.readValue(line, PortInfo.class);
+        Assert.assertEquals("port2", recorder.getPortInfoMap().get(pi.name).id, pi.id);
+        Assert.assertEquals("port2", recorder.getPortInfoMap().get(pi.name).type, pi.type);
+        line = br.readLine();
+        pi = mapper.readValue(line, PortInfo.class);
+        Assert.assertEquals("port3", recorder.getPortInfoMap().get(pi.name).id, pi.id);
+        Assert.assertEquals("port3", recorder.getPortInfoMap().get(pi.name).type, pi.type);
+        line = br.readLine();
+        pi = mapper.readValue(line, PortInfo.class);
+        Assert.assertEquals("port4", recorder.getPortInfoMap().get(pi.name).id, pi.id);
+        Assert.assertEquals("port4", recorder.getPortInfoMap().get(pi.name).type, pi.type);
+        Assert.assertEquals("port size", 4, recorder.getPortInfoMap().size());
+        //line = br.readLine();
+      }
       path = new Path(recorder.getStorage().getBasePath(), "part0.txt");
-      is = fs.open(path);
-      br = new BufferedReader(new InputStreamReader(is));
+      try (FSDataInputStream is = fs.open(path);
+          BufferedReader br = new BufferedReader(new InputStreamReader(is))) {
 
-      line = br.readLine();
-      Assert.assertTrue("check part0", line.startsWith("B:"));
-      Assert.assertTrue("check part0", line.endsWith(":1000"));
+        line = br.readLine();
+        Assert.assertTrue("check part0", line.startsWith("B:"));
+        Assert.assertTrue("check part0", line.endsWith(":1000"));
 
-      line = br.readLine();
-      Assert.assertTrue("check part0 1", line.startsWith("T:"));
-      Assert.assertTrue("check part0 1", line.endsWith(":0:30:{\"key\":\"speed\",\"value\":\"5m/h\"}"));
+        line = br.readLine();
+        Assert.assertTrue("check part0 1", line.startsWith("T:"));
+        Assert.assertTrue("check part0 1", line.endsWith(":0:30:{\"key\":\"speed\",\"value\":\"5m/h\"}"));
 
-      line = br.readLine();
-      Assert.assertTrue("check part0 2", line.startsWith("T:"));
-      Assert.assertTrue("check part0 2", line.endsWith(":2:30:{\"key\":\"speed\",\"value\":\"4m/h\"}"));
+        line = br.readLine();
+        Assert.assertTrue("check part0 2", line.startsWith("T:"));
+        Assert.assertTrue("check part0 2", line.endsWith(":2:30:{\"key\":\"speed\",\"value\":\"4m/h\"}"));
 
-      line = br.readLine();
-      Assert.assertTrue("check part0 3", line.startsWith("T:"));
-      Assert.assertTrue("check part0 3", line.endsWith(":1:30:{\"key\":\"speed\",\"value\":\"6m/h\"}"));
+        line = br.readLine();
+        Assert.assertTrue("check part0 3", line.startsWith("T:"));
+        Assert.assertTrue("check part0 3", line.endsWith(":1:30:{\"key\":\"speed\",\"value\":\"6m/h\"}"));
 
-      line = br.readLine();
-      Assert.assertTrue("check part0 4", line.startsWith("T:"));
-      Assert.assertTrue("check part0 4", line.endsWith(":3:30:{\"key\":\"speed\",\"value\":\"2m/h\"}"));
+        line = br.readLine();
+        Assert.assertTrue("check part0 4", line.startsWith("T:"));
+        Assert.assertTrue("check part0 4", line.endsWith(":3:30:{\"key\":\"speed\",\"value\":\"2m/h\"}"));
 
-      line = br.readLine();
-      Assert.assertTrue("check part0 5", line.startsWith("E:"));
-      Assert.assertTrue("check part0 5", line.endsWith(":1000"));
-    }
-    catch (IOException ex) {
+        line = br.readLine();
+        Assert.assertTrue("check part0 5", line.startsWith("E:"));
+        Assert.assertTrue("check part0 5", line.endsWith(":1000"));
+      }
+    } catch (IOException ex) {
       throw new RuntimeException(ex);
     }
-    finally {
-      fs.close();
-    }
   }
 
   private static final File testWorkDir = new File("target", TupleRecorderTest.class.getName());
@@ -234,17 +228,17 @@ public class TupleRecorderTest
     final PTOperator ptOp2 = localCluster.findByLogicalNode(dag.getMeta(op2));
     StramTestSupport.waitForActivation(localCluster, ptOp2);
 
-    testRecordingOnOperator(localCluster, ptOp2, 2);
+    testRecordingOnOperator(localCluster, ptOp2);
 
     final PTOperator ptOp1 = localCluster.findByLogicalNode(dag.getMeta(op1));
     StramTestSupport.waitForActivation(localCluster, ptOp1);
 
-    testRecordingOnOperator(localCluster, ptOp1, 1);
+    testRecordingOnOperator(localCluster, ptOp1);
 
     localCluster.shutdown();
   }
 
-  private void testRecordingOnOperator(final StramLocalCluster localCluster, final PTOperator op, int numPorts) throws Exception
+  private void testRecordingOnOperator(final StramLocalCluster localCluster, final PTOperator op) throws Exception
   {
     String id = "xyz";
     localCluster.getStreamingContainerManager().startRecording(id, op.getId(), null, 0);
@@ -259,25 +253,30 @@ public class TupleRecorderTest
 
     };
     Assert.assertTrue("Should get a tuple recorder within 10 seconds", StramTestSupport.awaitCompletion(c, 10000));
-    TupleRecorder tupleRecorder = getTupleRecorder(localCluster, op);
+    final TupleRecorder tupleRecorder = getTupleRecorder(localCluster, op);
     long startTime = tupleRecorder.getStartTime();
-    BufferedReader br;
     String line;
     File dir = new File(testWorkDir, "recordings/" + op.getId() + "/" + id);
     File file;
 
-    file = new File(dir, "meta.txt");
+    file = new File(dir, FSPartFileCollection.META_FILE);
     Assert.assertTrue("meta file should exist", file.exists());
-    br = new BufferedReader(new FileReader(file));
-    line = br.readLine();
-    Assert.assertEquals("version should be 1.2", "1.2", line);
-    line = br.readLine();
-    JSONObject json = new JSONObject(line);
-    Assert.assertEquals("Start time verification", startTime, json.getLong("startTime"));
-    
-    for (int i = 0; i < numPorts; i++) {
+    int numPorts = tupleRecorder.getSinkMap().size();
+
+    try (BufferedReader br = new BufferedReader(new FileReader(file))) {
       line = br.readLine();
-      Assert.assertTrue("should contain name, streamName, type and id", line != null && line.contains("\"name\"") && line.contains("\"streamName\"") && line.contains("\"type\"") && line.contains("\"id\""));
+      Assert.assertEquals("version should be 1.2", "1.2", line);
+      line = br.readLine();
+      JSONObject json = new JSONObject(line);
+      Assert.assertEquals("Start time verification", startTime, json.getLong("startTime"));
+      Assert.assertTrue(numPorts > 0);
+
+      for (int i = 0; i < numPorts; i++) {
+        line = br.readLine();
+        Assert.assertTrue("should contain name, streamName, type and id", line != null && line
+            .contains("\"name\"") && line.contains("\"streamName\"") && line.contains("\"type\"") && line
+            .contains("\"id\""));
+      }
     }
 
     c = new WaitCondition()
@@ -285,7 +284,6 @@ public class TupleRecorderTest
       @Override
       public boolean isComplete()
       {
-        TupleRecorder tupleRecorder = getTupleRecorder(localCluster, op);
         return (tupleRecorder.getTotalTupleCount() >= testTupleCount);
       }
 
@@ -306,24 +304,23 @@ public class TupleRecorderTest
     };
     Assert.assertTrue("Tuple recorder shouldn't exist any more after stopping", StramTestSupport.awaitCompletion(c, 5000));
 
-    file = new File(dir, "index.txt");
+    file = new File(dir, FSPartFileCollection.INDEX_FILE);
     Assert.assertTrue("index file should exist", file.exists());
-    br = new BufferedReader(new FileReader(file));
 
-    ArrayList<String> partFiles = new ArrayList<String>();
+    ArrayList<String> partFiles = new ArrayList<>();
     int indexCount = 0;
-    while ((line = br.readLine()) != null) {
-      String partFile = "part" + indexCount + ".txt";
-      if (line.startsWith("F:" + partFile + ":")) {
-        partFiles.add(partFile);
-        indexCount++;
-      }
-      else if (line.startsWith("E")) {
-        Assert.assertEquals("index file should end after E line", br.readLine(), null);
-        break;
-      }
-      else {
-        Assert.fail("index file line is not starting with F or E");
+    try (BufferedReader br = new BufferedReader(new FileReader(file))) {
+      while ((line = br.readLine()) != null) {
+        String partFile = "part" + indexCount + ".txt";
+        if (line.startsWith("F:" + partFile + ":")) {
+          partFiles.add(partFile);
+          indexCount++;
+        } else if (line.startsWith("E")) {
+          Assert.assertEquals("index file should end after E line", br.readLine(), null);
+          break;
+        } else {
+          Assert.fail("index file line is not starting with F or E");
+        }
       }
     }
 
@@ -337,17 +334,16 @@ public class TupleRecorderTest
         Assert.assertTrue(partFile + " should be greater than 1KB", file.length() >= 1024);
       }
       Assert.assertTrue(partFile + " should exist", file.exists());
-      br = new BufferedReader(new FileReader(file));
-      while ((line = br.readLine()) != null) {
-        if (line.startsWith("B:")) {
-          beginWindowExists = true;
-        }
-        else if (line.startsWith("E:")) {
-          endWindowExists = true;
-        }
-        else if (line.startsWith("T:")) {
-          String[] parts = line.split(":");
-          tupleCount[Integer.valueOf(parts[2])]++;
+      try (BufferedReader br = new BufferedReader(new FileReader(file))) {
+        while ((line = br.readLine()) != null) {
+          if (line.startsWith("B:")) {
+            beginWindowExists = true;
+          } else if (line.startsWith("E:")) {
+            endWindowExists = true;
+          } else if (line.startsWith("T:")) {
+            String[] parts = line.split(":");
+            tupleCount[Integer.valueOf(parts[2])]++;
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java b/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java
index c7e8ccc..2577504 100644
--- a/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/engine/GenericNodeTest.java
@@ -277,6 +277,8 @@ public class GenericNodeTest
     gn.connectInputPort("ip1", reservoir1);
     gn.connectInputPort("ip2", reservoir2);
     gn.connectOutputPort("op", output);
+    gn.firstWindowMillis = 0;
+    gn.windowWidthMillis = 100;
 
     final AtomicBoolean ab = new AtomicBoolean(false);
     Thread t = new Thread()
@@ -382,6 +384,8 @@ public class GenericNodeTest
     gn.connectInputPort("ip1", reservoir1);
     gn.connectInputPort("ip2", reservoir2);
     gn.connectOutputPort("op", Sink.BLACKHOLE);
+    gn.firstWindowMillis = 0;
+    gn.windowWidthMillis = 100;
 
     final AtomicBoolean ab = new AtomicBoolean(false);
     Thread t = new Thread()
@@ -493,6 +497,8 @@ public class GenericNodeTest
 
     in.connectInputPort("ip1", windowGenerator.acquireReservoir(String.valueOf(in.id), 1024));
     in.connectOutputPort("output", testSink);
+    in.firstWindowMillis = 0;
+    in.windowWidthMillis = 100;
 
     windowGenerator.activate(null);
 
@@ -551,9 +557,13 @@ public class GenericNodeTest
     final long sleepTime = 25L;
 
     WindowGenerator windowGenerator = new WindowGenerator(new ScheduledThreadPoolExecutor(1, "WindowGenerator"), 1024);
-    windowGenerator.setResetWindow(0L);
-    windowGenerator.setFirstWindow(1448909287863L);
-    windowGenerator.setWindowWidth(100);
+    long resetWindow = 0L;
+    long firstWindowMillis = 1448909287863L;
+    int windowWidth = 100;
+
+    windowGenerator.setResetWindow(resetWindow);
+    windowGenerator.setFirstWindow(firstWindowMillis);
+    windowGenerator.setWindowWidth(windowWidth);
     windowGenerator.setCheckpointCount(1, 0);
 
     GenericOperator go = new GenericOperator();
@@ -576,6 +586,8 @@ public class GenericNodeTest
 
     gn.connectInputPort("ip1", windowGenerator.acquireReservoir(String.valueOf(gn.id), 1024));
     gn.connectOutputPort("output", testSink);
+    gn.firstWindowMillis = firstWindowMillis;
+    gn.windowWidthMillis = windowWidth;
 
     windowGenerator.activate(null);
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/test/java/com/datatorrent/stram/engine/GenericTestOperator.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/engine/GenericTestOperator.java b/engine/src/test/java/com/datatorrent/stram/engine/GenericTestOperator.java
index 0c8ae62..a3b0c53 100644
--- a/engine/src/test/java/com/datatorrent/stram/engine/GenericTestOperator.java
+++ b/engine/src/test/java/com/datatorrent/stram/engine/GenericTestOperator.java
@@ -132,6 +132,9 @@ public class GenericTestOperator extends BaseOperator {
     if (outport1.isConnected()) {
       outport1.emit(o);
     }
+    if (outport2.isConnected()) {
+      outport2.emit(o);
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/f7e1ccf1/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
new file mode 100644
index 0000000..359da17
--- /dev/null
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/DelayOperatorTest.java
@@ -0,0 +1,377 @@
+/**
+ * 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 com.datatorrent.stram.plan.logical;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.TreeSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import javax.validation.ValidationException;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.Operator;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.common.util.DefaultDelayOperator;
+import com.datatorrent.stram.StramLocalCluster;
+import com.datatorrent.stram.engine.GenericTestOperator;
+import com.datatorrent.stram.engine.TestGeneratorInputOperator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * Unit tests for topologies with delay operator
+ */
+public class DelayOperatorTest
+{
+  private static Lock sequential = new ReentrantLock();
+
+  @Before
+  public void setup()
+  {
+    sequential.lock();
+  }
+
+  @After
+  public void teardown()
+  {
+    sequential.unlock();
+  }
+
+  @Test
+  public void testInvalidDelayDetection()
+  {
+    LogicalPlan dag = new LogicalPlan();
+
+    GenericTestOperator opB = dag.addOperator("B", GenericTestOperator.class);
+    GenericTestOperator opC = dag.addOperator("C", GenericTestOperator.class);
+    GenericTestOperator opD = dag.addOperator("D", GenericTestOperator.class);
+    DefaultDelayOperator opDelay = dag.addOperator("opDelay", DefaultDelayOperator.class);
+
+    dag.addStream("BtoC", opB.outport1, opC.inport1);
+    dag.addStream("CtoD", opC.outport1, opD.inport1);
+    dag.addStream("CtoDelay", opC.outport2, opDelay.input);
+    dag.addStream("DelayToD", opDelay.output, opD.inport2);
+
+    List<List<String>> invalidDelays = new ArrayList<>();
+    dag.findInvalidDelays(dag.getMeta(opB), invalidDelays);
+    assertEquals("operator invalid delay", 1, invalidDelays.size());
+
+    try {
+      dag.validate();
+      fail("validation should fail");
+    } catch (ValidationException e) {
+      // expected
+    }
+
+    dag = new LogicalPlan();
+
+    opB = dag.addOperator("B", GenericTestOperator.class);
+    opC = dag.addOperator("C", GenericTestOperator.class);
+    opD = dag.addOperator("D", GenericTestOperator.class);
+    opDelay = dag.addOperator("opDelay", DefaultDelayOperator.class);
+    dag.setAttribute(opDelay, Context.OperatorContext.APPLICATION_WINDOW_COUNT, 2);
+    dag.addStream("BtoC", opB.outport1, opC.inport1);
+    dag.addStream("CtoD", opC.outport1, opD.inport1);
+    dag.addStream("CtoDelay", opC.outport2, opDelay.input);
+    dag.addStream("DelayToC", opDelay.output, opC.inport2);
+
+    invalidDelays = new ArrayList<>();
+    dag.findInvalidDelays(dag.getMeta(opB), invalidDelays);
+    assertEquals("operator invalid delay", 1, invalidDelays.size());
+
+    try {
+      dag.validate();
+      fail("validation should fail");
+    } catch (ValidationException e) {
+      // expected
+    }
+
+    dag = new LogicalPlan();
+
+    opB = dag.addOperator("B", GenericTestOperator.class);
+    opC = dag.addOperator("C", GenericTestOperator.class);
+    opD = dag.addOperator("D", GenericTestOperator.class);
+    opDelay = dag.addOperator("opDelay", DefaultDelayOperator.class);
+    dag.addStream("BtoC", opB.outport1, opC.inport1);
+    dag.addStream("CtoD", opC.outport1, opD.inport1);
+    dag.addStream("CtoDelay", opC.outport2, opDelay.input).setLocality(DAG.Locality.THREAD_LOCAL);
+    dag.addStream("DelayToC", opDelay.output, opC.inport2).setLocality(DAG.Locality.THREAD_LOCAL);
+
+    try {
+      dag.validate();
+      fail("validation should fail");
+    } catch (ValidationException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testValidDelay()
+  {
+    LogicalPlan dag = new LogicalPlan();
+
+    TestGeneratorInputOperator opA = dag.addOperator("A", TestGeneratorInputOperator.class);
+    GenericTestOperator opB = dag.addOperator("B", GenericTestOperator.class);
+    GenericTestOperator opC = dag.addOperator("C", GenericTestOperator.class);
+    GenericTestOperator opD = dag.addOperator("D", GenericTestOperator.class);
+    DefaultDelayOperator opDelay = dag.addOperator("opDelay", DefaultDelayOperator.class);
+
+    dag.addStream("AtoB", opA.outport, opB.inport1);
+    dag.addStream("BtoC", opB.outport1, opC.inport1);
+    dag.addStream("CtoD", opC.outport1, opD.inport1);
+    dag.addStream("CtoDelay", opC.outport2, opDelay.input);
+    dag.addStream("DelayToB", opDelay.output, opB.inport2);
+    dag.validate();
+  }
+
+  public static final Long[] FIBONACCI_NUMBERS = new Long[]{
+      1L, 1L, 2L, 3L, 5L, 8L, 13L, 21L, 34L, 55L, 89L, 144L, 233L, 377L, 610L, 987L, 1597L, 2584L, 4181L, 6765L,
+      10946L, 17711L, 28657L, 46368L, 75025L, 121393L, 196418L, 317811L, 514229L, 832040L, 1346269L, 2178309L,
+      3524578L, 5702887L, 9227465L, 14930352L, 24157817L, 39088169L, 63245986L, 102334155L
+  };
+
+  public static class FibonacciOperator extends BaseOperator
+  {
+    public static List<Long> results = new ArrayList<>();
+    public long currentNumber = 1;
+    private transient long tempNum;
+
+    public transient DefaultInputPort<Object> dummyInputPort = new DefaultInputPort<Object>()
+    {
+      @Override
+      public void process(Object tuple)
+      {
+      }
+    };
+    public transient DefaultInputPort<Long> input = new DefaultInputPort<Long>()
+    {
+      @Override
+      public void process(Long tuple)
+      {
+        tempNum = tuple;
+      }
+    };
+    public transient DefaultOutputPort<Long> output = new DefaultOutputPort<>();
+
+    @Override
+    public void endWindow()
+    {
+      output.emit(currentNumber);
+      results.add(currentNumber);
+      currentNumber += tempNum;
+      if (currentNumber <= 0) {
+        // overflow
+        currentNumber = 1;
+      }
+    }
+
+  }
+
+  public static class FailableFibonacciOperator extends FibonacciOperator implements Operator.CheckpointListener
+  {
+    private boolean committed = false;
+    private int simulateFailureWindows = 0;
+    private boolean simulateFailureAfterCommit = false;
+    private int windowCount = 0;
+    public static boolean failureSimulated = false;
+
+    @Override
+    public void beginWindow(long windowId)
+    {
+      if (simulateFailureWindows > 0 && ((simulateFailureAfterCommit && committed) || !simulateFailureAfterCommit) &&
+          !failureSimulated) {
+        if (windowCount++ == simulateFailureWindows) {
+          failureSimulated = true;
+          throw new RuntimeException("simulating failure");
+        }
+      }
+    }
+
+    @Override
+    public void checkpointed(long windowId)
+    {
+    }
+
+    @Override
+    public void committed(long windowId)
+    {
+      committed = true;
+    }
+
+    public void setSimulateFailureWindows(int windows, boolean afterCommit)
+    {
+      this.simulateFailureAfterCommit = afterCommit;
+      this.simulateFailureWindows = windows;
+    }
+  }
+
+  public static class FailableDelayOperator extends DefaultDelayOperator implements Operator.CheckpointListener
+  {
+    private boolean committed = false;
+    private int simulateFailureWindows = 0;
+    private boolean simulateFailureAfterCommit = false;
+    private int windowCount = 0;
+    private static boolean failureSimulated = false;
+
+    @Override
+    public void beginWindow(long windowId)
+    {
+      super.beginWindow(windowId);
+      if (simulateFailureWindows > 0 && ((simulateFailureAfterCommit && committed) || !simulateFailureAfterCommit) &&
+          !failureSimulated) {
+        if (windowCount++ == simulateFailureWindows) {
+          failureSimulated = true;
+          throw new RuntimeException("simulating failure");
+        }
+      }
+    }
+
+    @Override
+    public void checkpointed(long windowId)
+    {
+    }
+
+    @Override
+    public void committed(long windowId)
+    {
+      committed = true;
+    }
+
+    public void setSimulateFailureWindows(int windows, boolean afterCommit)
+    {
+      this.simulateFailureAfterCommit = afterCommit;
+      this.simulateFailureWindows = windows;
+    }
+  }
+
+
+  @Test
+  public void testFibonacci() throws Exception
+  {
+    LogicalPlan dag = new LogicalPlan();
+
+    TestGeneratorInputOperator dummyInput = dag.addOperator("DUMMY", TestGeneratorInputOperator.class);
+    FibonacciOperator fib = dag.addOperator("FIB", FibonacciOperator.class);
+    DefaultDelayOperator opDelay = dag.addOperator("opDelay", DefaultDelayOperator.class);
+
+    dag.addStream("dummy_to_operator", dummyInput.outport, fib.dummyInputPort);
+    dag.addStream("operator_to_delay", fib.output, opDelay.input);
+    dag.addStream("delay_to_operator", opDelay.output, fib.input);
+    FibonacciOperator.results.clear();
+    final StramLocalCluster localCluster = new StramLocalCluster(dag);
+    localCluster.setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return FibonacciOperator.results.size() >= 10;
+      }
+    });
+    localCluster.run(10000);
+    Assert.assertArrayEquals(Arrays.copyOfRange(FIBONACCI_NUMBERS, 0, 10),
+        FibonacciOperator.results.subList(0, 10).toArray());
+  }
+
+  @Ignore // Out of sequence BEGIN_WINDOW tuple on Travis. Will tackle in the next version
+  @Test
+  public void testFibonacciRecovery1() throws Exception
+  {
+    LogicalPlan dag = new LogicalPlan();
+
+    TestGeneratorInputOperator dummyInput = dag.addOperator("DUMMY", TestGeneratorInputOperator.class);
+    FailableFibonacciOperator fib = dag.addOperator("FIB", FailableFibonacciOperator.class);
+    DefaultDelayOperator opDelay = dag.addOperator("opDelay", DefaultDelayOperator.class);
+
+    fib.setSimulateFailureWindows(3, true);
+
+    dag.addStream("dummy_to_operator", dummyInput.outport, fib.dummyInputPort);
+    dag.addStream("operator_to_delay", fib.output, opDelay.input);
+    dag.addStream("delay_to_operator", opDelay.output, fib.input);
+    dag.getAttributes().put(LogicalPlan.CHECKPOINT_WINDOW_COUNT, 2);
+    dag.getAttributes().put(LogicalPlan.STREAMING_WINDOW_SIZE_MILLIS, 300);
+    FailableFibonacciOperator.results.clear();
+    FailableFibonacciOperator.failureSimulated = false;
+    final StramLocalCluster localCluster = new StramLocalCluster(dag);
+    localCluster.setPerContainerBufferServer(true);
+    localCluster.setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return FailableFibonacciOperator.results.size() >= 30;
+      }
+    });
+    localCluster.run(60000);
+    Assert.assertTrue("failure should be invoked", FailableFibonacciOperator.failureSimulated);
+    Assert.assertArrayEquals(Arrays.copyOfRange(new TreeSet<>(Arrays.asList(FIBONACCI_NUMBERS)).toArray(), 0, 20),
+        Arrays.copyOfRange(new TreeSet<>(FibonacciOperator.results).toArray(), 0, 20));
+  }
+
+  @Ignore // Out of sequence BEGIN_WINDOW tuple on Travis. Will tackle in the next version
+  @Test
+  public void testFibonacciRecovery2() throws Exception
+  {
+    LogicalPlan dag = new LogicalPlan();
+
+    TestGeneratorInputOperator dummyInput = dag.addOperator("DUMMY", TestGeneratorInputOperator.class);
+    FibonacciOperator fib = dag.addOperator("FIB", FibonacciOperator.class);
+    FailableDelayOperator opDelay = dag.addOperator("opDelay", FailableDelayOperator.class);
+
+    opDelay.setSimulateFailureWindows(5, true);
+
+    dag.addStream("dummy_to_operator", dummyInput.outport, fib.dummyInputPort);
+    dag.addStream("operator_to_delay", fib.output, opDelay.input);
+    dag.addStream("delay_to_operator", opDelay.output, fib.input);
+    dag.getAttributes().put(LogicalPlan.CHECKPOINT_WINDOW_COUNT, 2);
+    dag.getAttributes().put(LogicalPlan.STREAMING_WINDOW_SIZE_MILLIS, 300);
+    FibonacciOperator.results.clear();
+    FailableDelayOperator.failureSimulated = false;
+    final StramLocalCluster localCluster = new StramLocalCluster(dag);
+    localCluster.setPerContainerBufferServer(true);
+    localCluster.setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return FibonacciOperator.results.size() >= 30;
+      }
+    });
+    localCluster.run(60000);
+
+    Assert.assertTrue("failure should be invoked", FailableDelayOperator.failureSimulated);
+    Assert.assertArrayEquals(Arrays.copyOfRange(new TreeSet<>(Arrays.asList(FIBONACCI_NUMBERS)).toArray(), 0, 20),
+        Arrays.copyOfRange(new TreeSet<>(FibonacciOperator.results).toArray(), 0, 20));
+  }
+
+
+}


[39/50] incubator-apex-core git commit: APEXCORE-296 #comment fixed the memory leak

Posted by th...@apache.org.
APEXCORE-296 #comment fixed the memory leak


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

Branch: refs/heads/master
Commit: d82d76698873facf58567ef42b84cad75ef8e810
Parents: 59770d0
Author: Gaurav <ga...@datatorrent.com>
Authored: Mon Jan 11 17:21:53 2016 -0800
Committer: Gaurav <ga...@datatorrent.com>
Committed: Fri Jan 15 09:33:30 2016 -0800

----------------------------------------------------------------------
 .../datatorrent/stram/StreamingContainerManager.java   | 13 +++++++++----
 .../stram/plan/physical/OperatorStatus.java            |  2 +-
 2 files changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d82d7669/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
index 162245b..4b79589 100644
--- a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
+++ b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
@@ -1027,6 +1027,14 @@ public class StreamingContainerManager implements PlanContext
         }
       }
       o.stats.lastWindowedStats = stats;
+      o.stats.operatorResponses = null;
+      if (!o.stats.responses.isEmpty()) {
+        o.stats.operatorResponses = new ArrayList<>();
+        StatsListener.OperatorResponse operatorResponse;
+        while ((operatorResponse = o.stats.responses.poll()) != null) {
+          o.stats.operatorResponses.add(operatorResponse);
+        }
+      }
       if (o.stats.lastWindowedStats != null) {
         // call listeners only with non empty window list
         if (o.statsListeners != null) {
@@ -1524,10 +1532,7 @@ public class StreamingContainerManager implements PlanContext
             LOG.debug(" Got back the response {} for the request {}", obj, obj.getResponseId());
           }
           else {       // This is to identify user requests
-            if (oper.stats.operatorResponses == null) {
-              oper.stats.operatorResponses = new ArrayList<StatsListener.OperatorResponse>();
-            }
-            oper.stats.operatorResponses.add(obj);
+            oper.stats.responses.add(obj);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d82d7669/engine/src/main/java/com/datatorrent/stram/plan/physical/OperatorStatus.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/physical/OperatorStatus.java b/engine/src/main/java/com/datatorrent/stram/plan/physical/OperatorStatus.java
index 4548173..063203e 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/physical/OperatorStatus.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/physical/OperatorStatus.java
@@ -34,7 +34,6 @@ import com.datatorrent.stram.api.StreamingContainerUmbilicalProtocol.OperatorHea
 import com.datatorrent.stram.engine.OperatorContext;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
 import com.datatorrent.stram.plan.physical.StatsRevisions.VersionedLong;
-import com.datatorrent.stram.util.MovingAverage;
 import com.datatorrent.stram.util.MovingAverage.MovingAverageLong;
 import com.datatorrent.stram.util.MovingAverage.TimedMovingAverageLong;
 
@@ -83,6 +82,7 @@ public class OperatorStatus implements BatchedOperatorStats, java.io.Serializabl
   public final ConcurrentLinkedQueue<List<OperatorStats>> listenerStats = new ConcurrentLinkedQueue<List<OperatorStats>>();
   public volatile long lastWindowIdChangeTms = 0;
   public final int windowProcessingTimeoutMillis;
+  public final ConcurrentLinkedQueue<StatsListener.OperatorResponse> responses = new ConcurrentLinkedQueue<>();
   public List<StatsListener.OperatorResponse> operatorResponses;
 
   private final LogicalPlan.OperatorMeta operatorMeta;


[42/50] incubator-apex-core git commit: Merge branch 'fix-warnings' of https://github.com/tweise/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'fix-warnings' of https://github.com/tweise/incubator-apex-core into devel-3


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

Branch: refs/heads/master
Commit: d0908e4bc226216805975ade997469dcd65b0b42
Parents: c284c39 d0b4bdb
Author: ishark <is...@datatorrent.com>
Authored: Wed Jan 20 13:57:56 2016 -0800
Committer: ishark <is...@datatorrent.com>
Committed: Wed Jan 20 13:57:56 2016 -0800

----------------------------------------------------------------------
 .../stram/plan/physical/PhysicalPlan.java       | 24 +++++++++++---------
 1 file changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[04/50] incubator-apex-core git commit: Merge branch 'checkstyle' into devel-3

Posted by th...@apache.org.
Merge branch 'checkstyle' into devel-3


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

Branch: refs/heads/master
Commit: 994f868e95f738c2a863c5440e7284ddc7a2a2fe
Parents: c336948 564888e
Author: Chandni Singh <cs...@apache.org>
Authored: Wed Dec 2 11:54:44 2015 -0800
Committer: Chandni Singh <cs...@apache.org>
Committed: Wed Dec 2 11:54:44 2015 -0800

----------------------------------------------------------------------
 apex_checks.xml | 2 +-
 pom.xml         | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[48/50] incubator-apex-core git commit: APEXCORE-308 - Implement findbugs plugin reporting

Posted by th...@apache.org.
APEXCORE-308 - Implement findbugs plugin reporting


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

Branch: refs/heads/master
Commit: d58de0c803131109f970885d69a9f6c97546d5c6
Parents: 50b0855
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Mon Jan 18 15:31:07 2016 -0800
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Mon Jan 25 14:58:08 2016 -0800

----------------------------------------------------------------------
 pom.xml | 22 ++++++++++++++++++++++
 1 file changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d58de0c8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 164645c..46c9b9c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -64,6 +64,9 @@
     <!-- do not change jetty version as later versions have problems with DefaultServlet -->
     <jetty.version>8.1.10.v20130312</jetty.version>
     <license.skip>true</license.skip>
+    <findbugs.xmlOutput>true</findbugs.xmlOutput>
+    <findbugs.failOnError>false</findbugs.failOnError>
+    <findbugs.skip>false</findbugs.skip>
     <postNoticeText>The initial developer of the original code is&#xA;DataTorrent, Inc. (http://www.datatorrent.com)&#xA;Copyright (c) 2012 - 2015. All Rights Reserved.</postNoticeText>
   </properties>
 
@@ -191,6 +194,25 @@
           </excludes>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <version>3.0.3</version>
+        <configuration>
+          <includeTests>true</includeTests>
+          <xmlOutput>${findbugs.xmlOutput}</xmlOutput>
+          <failOnError>${findbugs.failOnError}</failOnError>
+          <skip>${findbugs.skip}</skip>
+        </configuration>
+        <executions>
+          <execution>
+            <phase></phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     <pluginManagement>
       <plugins>


[44/50] incubator-apex-core git commit: moved attribute from context to logical plan

Posted by th...@apache.org.
moved attribute from context to logical plan


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

Branch: refs/heads/master
Commit: 4d5828c6ca48f5d28cd8c77c5706c6f72c7cd1ad
Parents: f7e1ccf
Author: Gaurav <ga...@datatorrent.com>
Authored: Wed Dec 16 06:33:54 2015 -0800
Committer: David Yan <da...@datatorrent.com>
Committed: Fri Jan 22 19:04:27 2016 -0800

----------------------------------------------------------------------
 api/src/main/java/com/datatorrent/api/Context.java           | 7 -------
 .../main/java/com/datatorrent/stram/engine/GenericNode.java  | 3 ++-
 .../java/com/datatorrent/stram/plan/logical/LogicalPlan.java | 8 +++++++-
 .../com/datatorrent/stram/plan/physical/PhysicalPlan.java    | 4 ++--
 .../com/datatorrent/stram/plan/physical/StreamMapping.java   | 2 +-
 5 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4d5828c6/api/src/main/java/com/datatorrent/api/Context.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/com/datatorrent/api/Context.java b/api/src/main/java/com/datatorrent/api/Context.java
index 58bc552..ceed8a2 100644
--- a/api/src/main/java/com/datatorrent/api/Context.java
+++ b/api/src/main/java/com/datatorrent/api/Context.java
@@ -166,13 +166,6 @@ public interface Context
      */
     Attribute<Class<?>> TUPLE_CLASS = new Attribute<>(new Class2String<>());
 
-    /**
-     * Attribute of input port.
-     * This is a read-only attribute to query whether the input port is connected to a DelayOperator
-     * This is for iterative processing.
-     */
-    Attribute<Boolean> IS_CONNECTED_TO_DELAY_OPERATOR = new Attribute<>(false);
-
     @SuppressWarnings("FieldNameHidesFieldInSuperclass")
     long serialVersionUID = AttributeMap.AttributeInitializer.initialize(PortContext.class);
   }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4d5828c6/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java b/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java
index 4777f93..1ccec31 100644
--- a/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java
+++ b/engine/src/main/java/com/datatorrent/stram/engine/GenericNode.java
@@ -40,6 +40,7 @@ import com.datatorrent.netlet.util.DTThrowable;
 import com.datatorrent.netlet.util.CircularBuffer;
 import com.datatorrent.stram.api.StreamingContainerUmbilicalProtocol.ContainerStats;
 import com.datatorrent.stram.debug.TappedReservoir;
+import com.datatorrent.stram.plan.logical.LogicalPlan;
 import com.datatorrent.stram.plan.logical.Operators;
 import com.datatorrent.stram.tuple.ResetWindowTuple;
 import com.datatorrent.stram.tuple.Tuple;
@@ -207,7 +208,7 @@ public class GenericNode extends Node<Operator>
     if (pcPair == null || pcPair.context == null) {
       return false;
     }
-    return pcPair.context.getValue(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR);
+    return pcPair.context.getValue(LogicalPlan.IS_CONNECTED_TO_DELAY_OPERATOR);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4d5828c6/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index 3c26118..883ad71 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -76,6 +76,12 @@ import com.datatorrent.stram.engine.Slider;
  */
 public class LogicalPlan implements Serializable, DAG
 {
+  /**
+   * Attribute of input port.
+   * This is a read-only attribute to query whether the input port is connected to a DelayOperator
+   * This is for iterative processing.
+   */
+  public static final Attribute<Boolean> IS_CONNECTED_TO_DELAY_OPERATOR = new Attribute<>(false);
   @SuppressWarnings("FieldNameHidesFieldInSuperclass")
   private static final long serialVersionUID = -2099729915606048704L;
   private static final Logger LOG = LoggerFactory.getLogger(LogicalPlan.class);
@@ -1914,7 +1920,7 @@ public class LogicalPlan implements Serializable, DAG
       for (InputPortMeta sink: downStream.sinks) {
         if (om.getOperator() instanceof Operator.DelayOperator) {
           // this is an iteration loop, do not treat it as downstream when detecting cycles
-          sink.attributes.put(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR, true);
+          sink.attributes.put(IS_CONNECTED_TO_DELAY_OPERATOR, true);
           continue;
         }
         OperatorMeta successor = sink.getOperatorWrapper();

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4d5828c6/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
index da96ef3..c696224 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/physical/PhysicalPlan.java
@@ -948,11 +948,11 @@ public class PhysicalPlan implements Serializable
                 PTOperator slidingUnifier = StreamMapping.createSlidingUnifier(sourceOut.logicalStream, this,
                   sourceOM.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT), slidingWindowCount);
                 StreamMapping.addInput(slidingUnifier, sourceOut, null);
-                input = new PTInput(ipm.getKey().getPortName(), ipm.getValue(), oper, null, slidingUnifier.outputs.get(0), ipm.getKey().getValue(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR));
+                input = new PTInput(ipm.getKey().getPortName(), ipm.getValue(), oper, null, slidingUnifier.outputs.get(0), ipm.getKey().getValue(LogicalPlan.IS_CONNECTED_TO_DELAY_OPERATOR));
                 sourceMapping.outputStreams.get(ipm.getValue().getSource()).slidingUnifiers.add(slidingUnifier);
               }
               else {
-                input = new PTInput(ipm.getKey().getPortName(), ipm.getValue(), oper, null, sourceOut, ipm.getKey().getValue(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR));
+                input = new PTInput(ipm.getKey().getPortName(), ipm.getValue(), oper, null, sourceOut, ipm.getKey().getValue(LogicalPlan.IS_CONNECTED_TO_DELAY_OPERATOR));
               }
               oper.inputs.add(input);
             }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4d5828c6/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java b/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java
index 91c6eef..f30ceb6 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/physical/StreamMapping.java
@@ -347,7 +347,7 @@ public class StreamMapping implements java.io.Serializable
     // link to upstream output(s) for this stream
     for (PTOutput upstreamOut : sourceOper.outputs) {
       if (upstreamOut.logicalStream == streamMeta) {
-        PTInput input = new PTInput(ipm.getPortName(), streamMeta, oper, pks, upstreamOut, ipm.getValue(PortContext.IS_CONNECTED_TO_DELAY_OPERATOR));
+        PTInput input = new PTInput(ipm.getPortName(), streamMeta, oper, pks, upstreamOut, ipm.getValue(LogicalPlan.IS_CONNECTED_TO_DELAY_OPERATOR));
         oper.inputs.add(input);
       }
     }


[16/50] incubator-apex-core git commit: APEXCORE-291 #resolve #comment using operator instance as default aggregator when it implements AutoMetric.Aggregator

Posted by th...@apache.org.
APEXCORE-291 #resolve #comment using operator instance as default aggregator when it implements AutoMetric.Aggregator


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

Branch: refs/heads/master
Commit: ecb96bad0aa2e5c959d7626ebdd93e0fb265c424
Parents: 56b55fe
Author: Chandni Singh <cs...@apache.org>
Authored: Tue Dec 22 00:45:22 2015 -0800
Committer: Chandni Singh <cs...@apache.org>
Committed: Tue Dec 29 23:40:22 2015 -0800

----------------------------------------------------------------------
 engine/pom.xml                                  |  2 +-
 .../stram/appdata/AppDataPushAgent.java         |  3 +-
 .../stram/plan/logical/LogicalPlan.java         | 38 +--------
 .../plan/logical/MetricAggregatorMeta.java      | 89 ++++++++++++++++++++
 .../stram/engine/AutoMetricTest.java            | 65 +++++++++++++-
 5 files changed, 156 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ecb96bad/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 34593ce..6bf8283 100644
--- a/engine/pom.xml
+++ b/engine/pom.xml
@@ -145,7 +145,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>4406</maxAllowedViolations>
+          <maxAllowedViolations>4400</maxAllowedViolations>
         </configuration>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ecb96bad/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java b/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
index 9389e3c..b47dc41 100644
--- a/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
+++ b/engine/src/main/java/com/datatorrent/stram/appdata/AppDataPushAgent.java
@@ -47,6 +47,7 @@ import com.datatorrent.stram.StreamingContainerManager;
 import com.datatorrent.stram.WebsocketAppDataPusher;
 import com.datatorrent.stram.api.AppDataPusher;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
+import com.datatorrent.stram.plan.logical.MetricAggregatorMeta;
 import com.datatorrent.stram.webapp.LogicalOperatorInfo;
 
 /**
@@ -253,7 +254,7 @@ public class AppDataPushAgent extends AbstractService
       result.put("appName", dnmgr.getApplicationAttributes().get(DAGContext.APPLICATION_NAME));
       result.put("logicalOperatorName", operatorMeta.getName());
 
-      LogicalPlan.MetricAggregatorMeta metricAggregatorMeta = operatorMeta.getMetricAggregatorMeta();
+      MetricAggregatorMeta metricAggregatorMeta = operatorMeta.getMetricAggregatorMeta();
       JSONArray valueSchemas = new JSONArray();
       for (Map.Entry<String, Object> entry : aggregates.entrySet()) {
         String metricName = entry.getKey();

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ecb96bad/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index 377fa6d..347e94f 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -839,6 +839,9 @@ public class LogicalPlan implements Serializable, DAG
     protected void populateAggregatorMeta()
     {
       AutoMetric.Aggregator aggregator = getValue(OperatorContext.METRICS_AGGREGATOR);
+      if (aggregator == null && operator instanceof AutoMetric.Aggregator) {
+        aggregator = new MetricAggregatorMeta.MetricsAggregatorProxy(this);
+      }
       if (aggregator == null) {
         MetricsAggregator defAggregator = null;
         Set<String> metricNames = Sets.newHashSet();
@@ -2015,39 +2018,4 @@ public class LogicalPlan implements Serializable, DAG
     return result;
   }
 
-  public final class MetricAggregatorMeta implements Serializable
-  {
-    private final AutoMetric.Aggregator aggregator;
-    private final AutoMetric.DimensionsScheme dimensionsScheme;
-
-    protected MetricAggregatorMeta(AutoMetric.Aggregator aggregator,
-                                   AutoMetric.DimensionsScheme dimensionsScheme)
-    {
-      this.aggregator = aggregator;
-      this.dimensionsScheme = dimensionsScheme;
-    }
-
-    public AutoMetric.Aggregator getAggregator()
-    {
-      return this.aggregator;
-    }
-
-    public String[] getDimensionAggregatorsFor(String logicalMetricName)
-    {
-      if (dimensionsScheme == null) {
-        return null;
-      }
-      return dimensionsScheme.getDimensionAggregationsFor(logicalMetricName);
-    }
-
-    public String[] getTimeBuckets()
-    {
-      if (dimensionsScheme == null) {
-        return null;
-      }
-      return dimensionsScheme.getTimeBuckets();
-    }
-
-    private static final long serialVersionUID = 201604271719L;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ecb96bad/engine/src/main/java/com/datatorrent/stram/plan/logical/MetricAggregatorMeta.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/MetricAggregatorMeta.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/MetricAggregatorMeta.java
new file mode 100644
index 0000000..65bc2a4
--- /dev/null
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/MetricAggregatorMeta.java
@@ -0,0 +1,89 @@
+/**
+ * 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 com.datatorrent.stram.plan.logical;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+
+import javax.validation.constraints.NotNull;
+
+import com.google.common.base.Preconditions;
+
+import com.datatorrent.api.AutoMetric;
+
+/**
+ * A class that encapsulates {@link AutoMetric.Aggregator} and {@link AutoMetric.DimensionsScheme} of a particular
+ * operator.
+ */
+public final class MetricAggregatorMeta implements Serializable
+{
+  private final AutoMetric.Aggregator aggregator;
+  private final AutoMetric.DimensionsScheme dimensionsScheme;
+
+  protected MetricAggregatorMeta(AutoMetric.Aggregator aggregator, AutoMetric.DimensionsScheme dimensionsScheme)
+  {
+    this.aggregator = aggregator;
+    this.dimensionsScheme = dimensionsScheme;
+  }
+
+  public AutoMetric.Aggregator getAggregator()
+  {
+    return this.aggregator;
+  }
+
+  public String[] getDimensionAggregatorsFor(String logicalMetricName)
+  {
+    if (dimensionsScheme == null) {
+      return null;
+    }
+    return dimensionsScheme.getDimensionAggregationsFor(logicalMetricName);
+  }
+
+  public String[] getTimeBuckets()
+  {
+    if (dimensionsScheme == null) {
+      return null;
+    }
+    return dimensionsScheme.getTimeBuckets();
+  }
+
+  private static final long serialVersionUID = 201604271719L;
+
+  /**
+   * Serves as a proxy for Aggregator when operator itself implements {@link AutoMetric.Aggregator}.
+   */
+  static final class MetricsAggregatorProxy implements AutoMetric.Aggregator, Serializable
+  {
+    private final LogicalPlan.OperatorMeta om;
+
+    MetricsAggregatorProxy(@NotNull LogicalPlan.OperatorMeta om)
+    {
+      this.om = Preconditions.checkNotNull(om);
+    }
+
+    @Override
+    public Map<String, Object> aggregate(long windowId, Collection<AutoMetric.PhysicalMetricsContext> physicalMetrics)
+    {
+      return ((AutoMetric.Aggregator)om.getOperator()).aggregate(windowId, physicalMetrics);
+    }
+
+    private static final long serialVersionUID = 201512221830L;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ecb96bad/engine/src/test/java/com/datatorrent/stram/engine/AutoMetricTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/engine/AutoMetricTest.java b/engine/src/test/java/com/datatorrent/stram/engine/AutoMetricTest.java
index 28e2e51..a76e6e0 100644
--- a/engine/src/test/java/com/datatorrent/stram/engine/AutoMetricTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/engine/AutoMetricTest.java
@@ -25,7 +25,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 
-import org.apache.hadoop.conf.Configuration;
+import javax.validation.constraints.NotNull;
+
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Ignore;
@@ -34,17 +35,21 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.conf.Configuration;
+
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-import com.datatorrent.api.*;
+import com.datatorrent.api.AutoMetric;
+import com.datatorrent.api.Context;
 import com.datatorrent.api.Context.OperatorContext;
 import com.datatorrent.api.DAG.Locality;
+import com.datatorrent.api.Operator;
+import com.datatorrent.api.Partitioner;
 import com.datatorrent.api.Stats.OperatorStats;
-
+import com.datatorrent.api.StatsListener;
 import com.datatorrent.common.partitioner.StatelessPartitioner;
-import com.datatorrent.common.util.AsyncFSStorageAgent;
 import com.datatorrent.stram.StramLocalCluster;
 import com.datatorrent.stram.engine.AutoMetricTest.TestOperator.TestStatsListener;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
@@ -307,6 +312,33 @@ public class AutoMetricTest
     Assert.assertNotNull("default aggregator injected", o1meta.getMetricAggregatorMeta().getAggregator());
   }
 
+  @Test
+  public void testDefaultMetricsAggregator() throws Exception
+  {
+    LogicalPlanConfiguration lpc = new LogicalPlanConfiguration(new Configuration());
+
+    TestGeneratorInputOperator inputOperator = dag.addOperator("input", TestGeneratorInputOperator.class);
+
+    CountDownLatch latch = new CountDownLatch(1);
+    OperatorAndAggregator o1 = dag.addOperator("o1", new OperatorAndAggregator(latch));
+
+    dag.setAttribute(Context.OperatorContext.STORAGE_AGENT, new StramTestSupport.MemoryStorageAgent());
+
+    dag.addStream("TestTuples", inputOperator.outport, o1.inport1);
+
+    lpc.prepareDAG(dag, null, "AutoMetricTest");
+
+    LogicalPlan.OperatorMeta o1meta = dag.getOperatorMeta("o1");
+    Assert.assertNotNull("default aggregator injected", o1meta.getMetricAggregatorMeta().getAggregator());
+
+    lpc.prepareDAG(dag, null, "AutoMetricTest");
+    StramLocalCluster lc = new StramLocalCluster(dag);
+    lc.runAsync();
+    latch.await();
+    Assert.assertEquals("progress", 1, o1.result.get("progress"));
+    lc.shutdown();
+  }
+
   private static class MockAggregator implements AutoMetric.Aggregator, Serializable
   {
     long cachedSum = -1;
@@ -361,6 +393,31 @@ public class AutoMetricTest
     }
   }
 
+  public static class OperatorAndAggregator extends OperatorWithMetrics implements AutoMetric.Aggregator
+  {
+    Map<String, Object> result = Maps.newHashMap();
+
+    private final transient CountDownLatch latch;
+
+    private OperatorAndAggregator()
+    {
+      latch = null;
+    }
+
+    OperatorAndAggregator(@NotNull CountDownLatch latch)
+    {
+      this.latch = Preconditions.checkNotNull(latch);
+    }
+
+    @Override
+    public Map<String, Object> aggregate(long windowId, Collection<AutoMetric.PhysicalMetricsContext> physicalMetrics)
+    {
+      result.put("progress", physicalMetrics.iterator().next().getMetrics().get("progress"));
+      latch.countDown();
+      return result;
+    }
+  }
+
   @Test
   public void testMetricsAnnotatedMethod() throws Exception
   {


[40/50] incubator-apex-core git commit: Merge branch 'APEXCORE-296' of https://github.com/gauravgopi123/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'APEXCORE-296' of https://github.com/gauravgopi123/incubator-apex-core into devel-3


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

Branch: refs/heads/master
Commit: c284c39cd7dd6ff3aba04290f9d4612a1108dbbe
Parents: ff198ae d82d766
Author: Thomas Weise <th...@datatorrent.com>
Authored: Fri Jan 15 12:41:56 2016 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Fri Jan 15 12:41:56 2016 -0800

----------------------------------------------------------------------
 .../datatorrent/stram/StreamingContainerManager.java   | 13 +++++++++----
 .../stram/plan/physical/OperatorStatus.java            |  2 +-
 2 files changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[14/50] incubator-apex-core git commit: APEXCORE-144, APEXCORE-145 Rest api changes to view module information

Posted by th...@apache.org.
APEXCORE-144, APEXCORE-145 Rest api changes to view module information


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

Branch: refs/heads/master
Commit: 47b3ce8e4019f325c2e91968cb120763f18bb174
Parents: c1314ea
Author: shubham <sh...@github.com>
Authored: Tue Nov 17 12:10:08 2015 +0530
Committer: Tushar R. Gosavi <tu...@apache.org>
Committed: Tue Dec 22 12:03:55 2015 +0530

----------------------------------------------------------------------
 .../stram/StreamingContainerManager.java        |  20 +++
 .../java/com/datatorrent/stram/cli/DTCli.java   |   8 +-
 .../stram/codec/LogicalPlanSerializer.java      |  47 ++++++-
 .../stram/plan/logical/LogicalPlan.java         |   7 +-
 .../plan/logical/LogicalPlanConfiguration.java  |   4 +-
 .../stram/webapp/StramWebServices.java          | 123 +++++++++++++------
 6 files changed, 158 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/47b3ce8e/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
index 29c6a2c..162245b 100644
--- a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
+++ b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
@@ -95,6 +95,7 @@ import com.datatorrent.stram.engine.WindowGenerator;
 import com.datatorrent.stram.plan.logical.LogicalOperatorStatus;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
 import com.datatorrent.stram.plan.logical.LogicalPlan.InputPortMeta;
+import com.datatorrent.stram.plan.logical.LogicalPlan.ModuleMeta;
 import com.datatorrent.stram.plan.logical.LogicalPlan.OperatorMeta;
 import com.datatorrent.stram.plan.logical.LogicalPlan.OutputPortMeta;
 import com.datatorrent.stram.plan.logical.LogicalPlanConfiguration;
@@ -2246,6 +2247,25 @@ public class StreamingContainerManager implements PlanContext
     return fillLogicalOperatorInfo(operatorMeta);
   }
 
+  public ModuleMeta getModuleMeta(String moduleName)
+  {
+    return getModuleMeta(moduleName, getLogicalPlan());
+  }
+
+  private ModuleMeta getModuleMeta(String moduleName, LogicalPlan dag)
+  {
+    for (ModuleMeta m : dag.getAllModules()) {
+      if (m.getFullName().equals(moduleName)) {
+        return m;
+      }
+      ModuleMeta res = getModuleMeta(moduleName, m.getDag());
+      if (res != null) {
+        return res;
+      }
+    }
+    return null;
+  }
+
   public List<LogicalOperatorInfo> getLogicalOperatorInfoList()
   {
     List<LogicalOperatorInfo> infoList = new ArrayList<LogicalOperatorInfo>();

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/47b3ce8e/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java b/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java
index deb0967..696f497 100644
--- a/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java
+++ b/engine/src/main/java/com/datatorrent/stram/cli/DTCli.java
@@ -2824,7 +2824,7 @@ public class DTCli
               }
               LogicalPlan logicalPlan = appFactory.createApp(submitApp.getLogicalPlanConfiguration());
               map.put("applicationName", appFactory.getName());
-              map.put("logicalPlan", LogicalPlanSerializer.convertToMap(logicalPlan));
+              map.put("logicalPlan", LogicalPlanSerializer.convertToMap(logicalPlan, false));
             } finally {
               if (raw) {
                 System.setOut(originalStream);
@@ -2840,7 +2840,7 @@ public class DTCli
             LogicalPlan logicalPlan = appFactory.createApp(submitApp.getLogicalPlanConfiguration());
             Map<String, Object> map = new HashMap<String, Object>();
             map.put("applicationName", appFactory.getName());
-            map.put("logicalPlan", LogicalPlanSerializer.convertToMap(logicalPlan));
+            map.put("logicalPlan", LogicalPlanSerializer.convertToMap(logicalPlan, false));
             printJson(map);
           } else if (filename.endsWith(".properties")) {
             File file = new File(filename);
@@ -2849,7 +2849,7 @@ public class DTCli
             LogicalPlan logicalPlan = appFactory.createApp(submitApp.getLogicalPlanConfiguration());
             Map<String, Object> map = new HashMap<String, Object>();
             map.put("applicationName", appFactory.getName());
-            map.put("logicalPlan", LogicalPlanSerializer.convertToMap(logicalPlan));
+            map.put("logicalPlan", LogicalPlanSerializer.convertToMap(logicalPlan, false));
             printJson(map);
           } else {
             StramAppLauncher submitApp = getStramAppLauncher(filename, config, commandLineInfo.ignorePom);
@@ -2893,7 +2893,7 @@ public class DTCli
               Map<String, Object> map = new HashMap<String, Object>();
               map.put("applicationName", appInfo.name);
               if (appInfo.dag != null) {
-                map.put("logicalPlan", LogicalPlanSerializer.convertToMap(appInfo.dag));
+                map.put("logicalPlan", LogicalPlanSerializer.convertToMap(appInfo.dag, false));
               }
               if (appInfo.error != null) {
                 map.put("error", appInfo.error);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/47b3ce8e/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java b/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java
index 90dd2b5..7b61d5b 100644
--- a/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java
+++ b/engine/src/main/java/com/datatorrent/stram/codec/LogicalPlanSerializer.java
@@ -88,7 +88,7 @@ public class LogicalPlanSerializer extends JsonSerializer<LogicalPlan>
    * @param dag
    * @return
    */
-  public static Map<String, Object> convertToMap(LogicalPlan dag)
+  public static Map<String, Object> convertToMap(LogicalPlan dag, boolean includeModules)
   {
     HashMap<String, Object> result = new HashMap<String, Object>();
     ArrayList<Object> operatorArray = new ArrayList< Object>();
@@ -200,6 +200,15 @@ public class LogicalPlanSerializer extends JsonSerializer<LogicalPlan>
         streamDetailMap.put("locality", streamMeta.getLocality().name());
       }
     }
+
+    if (includeModules) {
+      ArrayList<Map<String, Object>> modulesArray = new ArrayList<>();
+      result.put("modules", modulesArray);
+      for(LogicalPlan.ModuleMeta meta : dag.getAllModules()) {
+        modulesArray.add(getLogicalModuleDetails(dag, meta));
+      }
+    }
+
     return result;
   }
 
@@ -323,13 +332,43 @@ public class LogicalPlanSerializer extends JsonSerializer<LogicalPlan>
 
   public static JSONObject convertToJsonObject(LogicalPlan dag)
   {
-    return new JSONObject(convertToMap(dag));
+    return new JSONObject(convertToMap(dag, false));
   }
 
   @Override
-  public void serialize(LogicalPlan dag, JsonGenerator jg, SerializerProvider sp) throws IOException, JsonProcessingException
+  public void serialize(LogicalPlan dag, JsonGenerator jg, SerializerProvider sp) throws IOException,
+      JsonProcessingException
   {
-    jg.writeObject(convertToMap(dag));
+    jg.writeObject(convertToMap(dag, false));
+  }
+
+  /**
+   * Return information about operators and inner modules of a module.
+   *
+   * @param dag        top level DAG
+   * @param moduleMeta module information. DAG within module is used for constructing response.
+   * @return
+   */
+  private static Map<String, Object> getLogicalModuleDetails(LogicalPlan dag, LogicalPlan.ModuleMeta moduleMeta)
+  {
+    Map<String, Object> moduleDetailMap = new HashMap<String, Object>();
+    ArrayList<String> operatorArray = new ArrayList<>();
+    moduleDetailMap.put("name", moduleMeta.getName());
+    moduleDetailMap.put("className", moduleMeta.getModule().getClass().getName());
+
+    moduleDetailMap.put("operators", operatorArray);
+    for (OperatorMeta operatorMeta : moduleMeta.getDag().getAllOperators()) {
+      if (operatorMeta.getModuleName() == null) {
+        String fullName = moduleMeta.getFullName() + LogicalPlan.MODULE_NAMESPACE_SEPARATOR + operatorMeta.getName();
+        operatorArray.add(fullName);
+      }
+    }
+    ArrayList<Map<String, Object>> modulesArray = new ArrayList<>();
+    moduleDetailMap.put("modules", modulesArray);
+    for (LogicalPlan.ModuleMeta meta : moduleMeta.getDag().getAllModules()) {
+      modulesArray.add(getLogicalModuleDetails(dag, meta));
+    }
+    return moduleDetailMap;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/47b3ce8e/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
index 21039cc..377fa6d 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java
@@ -32,6 +32,7 @@ import javax.validation.constraints.NotNull;
 
 import org.apache.commons.lang.builder.ToStringBuilder;
 import org.apache.commons.lang.builder.ToStringStyle;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.slf4j.Logger;
@@ -1090,9 +1091,9 @@ public class LogicalPlan implements Serializable, DAG
     }
 
     // Avoid name conflict with module.
-    if (modules.containsKey(name))
+    if (modules.containsKey(name)) {
       throw new IllegalArgumentException("duplicate operator id: " + operators.get(name));
-
+    }
     OperatorMeta decl = new OperatorMeta(name, operator);
     rootOperators.add(decl); // will be removed when a sink is added to an input port for this operator
     operators.put(name, decl);
@@ -1193,7 +1194,7 @@ public class LogicalPlan implements Serializable, DAG
      * @param parentDAG parent dag to populate with operators from this and inner modules.
      * @param conf      configuration object.
      */
-    public void flattenModule(LogicalPlan parentDAG, org.apache.hadoop.conf.Configuration conf)
+    public void flattenModule(LogicalPlan parentDAG, Configuration conf)
     {
       module.populateDAG(dag, conf);
       for (ModuleMeta subModuleMeta : dag.getAllModules()) {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/47b3ce8e/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
index 483576a..dbd3bc3 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java
@@ -18,6 +18,7 @@
  */
 package com.datatorrent.stram.plan.logical;
 
+
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -31,6 +32,7 @@ import java.lang.reflect.Type;
 import java.util.*;
 import java.util.Map.Entry;
 
+
 import javax.validation.ValidationException;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -45,7 +47,6 @@ import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.commons.beanutils.BeanMap;
 import org.apache.commons.beanutils.BeanUtils;
 import org.apache.commons.collections.CollectionUtils;
@@ -60,6 +61,7 @@ import com.datatorrent.api.Context.DAGContext;
 import com.datatorrent.api.Context.OperatorContext;
 import com.datatorrent.api.Context.PortContext;
 import com.datatorrent.api.annotation.ApplicationAnnotation;
+
 import com.datatorrent.stram.StramUtils;
 import com.datatorrent.stram.client.StramClientUtils;
 import com.datatorrent.stram.plan.logical.LogicalPlan.InputPortMeta;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/47b3ce8e/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java b/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java
index 6fdba00..fd47d35 100644
--- a/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java
+++ b/engine/src/main/java/com/datatorrent/stram/webapp/StramWebServices.java
@@ -68,6 +68,7 @@ import com.datatorrent.stram.StreamingContainerManager;
 import com.datatorrent.stram.StringCodecs;
 import com.datatorrent.stram.codec.LogicalPlanSerializer;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
+import com.datatorrent.stram.plan.logical.LogicalPlan.ModuleMeta;
 import com.datatorrent.stram.plan.logical.LogicalPlan.OperatorMeta;
 import com.datatorrent.stram.plan.logical.LogicalPlanConfiguration;
 import com.datatorrent.stram.plan.logical.requests.LogicalPlanRequest;
@@ -561,17 +562,14 @@ public class StramWebServices
         LOG.debug("Setting property for {}: {}={}", operatorName, key, val);
         dagManager.setOperatorProperty(operatorName, key, val);
       }
-    }
-    catch (JSONException ex) {
+    } catch (JSONException ex) {
       LOG.warn("Got JSON Exception: ", ex);
-    }
-    catch (Exception ex) {
+    } catch (Exception ex) {
       LOG.error("Caught exception: ", ex);
       throw new RuntimeException(ex);
     }
     return response;
   }
-
   @POST // not supported by WebAppProxyServlet, can only be called directly
   @Path(PATH_PHYSICAL_PLAN_OPERATORS + "/{operatorId:\\d+}/properties")
   @Consumes(MediaType.APPLICATION_JSON)
@@ -633,60 +631,97 @@ public class StramWebServices
   public JSONObject getPorts(@PathParam("operatorName") String operatorName)
   {
     OperatorMeta logicalOperator = dagManager.getLogicalPlan().getOperatorMeta(operatorName);
+    Set<LogicalPlan.InputPortMeta> inputPorts;
+    Set<LogicalPlan.OutputPortMeta> outputPorts;
     if (logicalOperator == null) {
-      throw new NotFoundException();
+      ModuleMeta logicalModule = dagManager.getModuleMeta(operatorName);
+      if (logicalModule == null) {
+        throw new NotFoundException();
+      }
+      inputPorts = logicalModule.getInputStreams().keySet();
+      outputPorts = logicalModule.getOutputStreams().keySet();
+    } else {
+      inputPorts = logicalOperator.getInputStreams().keySet();
+      outputPorts = logicalOperator.getOutputStreams().keySet();
     }
+
+    JSONObject result = getPortsObjects(inputPorts, outputPorts);
+    return result;
+  }
+
+  private JSONObject getPortsObjects(Collection<LogicalPlan.InputPortMeta> inputs, Collection<LogicalPlan.OutputPortMeta> outputs)
+  {
     JSONObject result = new JSONObject();
     JSONArray ports = new JSONArray();
     try {
-      for (LogicalPlan.InputPortMeta inputPort : logicalOperator.getInputStreams().keySet()) {
+      for (LogicalPlan.InputPortMeta inputPort : inputs) {
         JSONObject port = new JSONObject();
         port.put("name", inputPort.getPortName());
         port.put("type", "input");
         ports.put(port);
       }
-      for (LogicalPlan.OutputPortMeta outputPort : logicalOperator.getOutputStreams().keySet()) {
+      for (LogicalPlan.OutputPortMeta outputPort : outputs) {
         JSONObject port = new JSONObject();
         port.put("name", outputPort.getPortName());
         port.put("type", "output");
         ports.put(port);
       }
       result.put("ports", ports);
-    }
-    catch (JSONException ex) {
+    } catch (JSONException ex) {
       throw new RuntimeException(ex);
     }
     return result;
   }
 
+  private JSONObject getPortObject(Collection<LogicalPlan.InputPortMeta> inputs, Collection<LogicalPlan.OutputPortMeta> outputs,
+                                   String portName) throws JSONException
+  {
+    for (LogicalPlan.InputPortMeta inputPort : inputs) {
+      if (inputPort.getPortName().equals(portName)) {
+        JSONObject port = new JSONObject();
+        port.put("name", inputPort.getPortName());
+        port.put("type", "input");
+        return port;
+      }
+    }
+    for (LogicalPlan.OutputPortMeta outputPort : outputs) {
+      if (outputPort.getPortName().equals(portName)) {
+        JSONObject port = new JSONObject();
+        port.put("name", outputPort.getPortName());
+        port.put("type", "output");
+        return port;
+      }
+    }
+    return null;
+  }
+
+
   @GET
   @Path(PATH_LOGICAL_PLAN_OPERATORS + "/{operatorName}/ports/{portName}")
   @Produces(MediaType.APPLICATION_JSON)
   public JSONObject getPort(@PathParam("operatorName") String operatorName, @PathParam("portName") String portName)
   {
     OperatorMeta logicalOperator = dagManager.getLogicalPlan().getOperatorMeta(operatorName);
+    Set<LogicalPlan.InputPortMeta> inputPorts;
+    Set<LogicalPlan.OutputPortMeta> outputPorts;
     if (logicalOperator == null) {
-      throw new NotFoundException();
+      ModuleMeta logicalModule = dagManager.getModuleMeta(operatorName);
+      if (logicalModule == null) {
+        throw new NotFoundException();
+      }
+      inputPorts = logicalModule.getInputStreams().keySet();
+      outputPorts = logicalModule.getOutputStreams().keySet();
+    } else {
+      inputPorts = logicalOperator.getInputStreams().keySet();
+      outputPorts = logicalOperator.getOutputStreams().keySet();
     }
+
     try {
-      for (LogicalPlan.InputPortMeta inputPort : logicalOperator.getInputStreams().keySet()) {
-        if (portName.equals(portName)) {
-          JSONObject port = new JSONObject();
-          port.put("name", inputPort.getPortName());
-          port.put("type", "input");
-          return port;
-        }
-      }
-      for (LogicalPlan.OutputPortMeta outputPort : logicalOperator.getOutputStreams().keySet()) {
-        if (portName.equals(portName)) {
-          JSONObject port = new JSONObject();
-          port.put("name", outputPort.getPortName());
-          port.put("type", "output");
-          return port;
-        }
+      JSONObject resp = getPortObject(inputPorts, outputPorts, portName);
+      if (resp != null) {
+        return resp;
       }
-    }
-    catch (JSONException ex) {
+    } catch (JSONException ex) {
       throw new RuntimeException(ex);
     }
     throw new NotFoundException();
@@ -711,31 +746,41 @@ public class StramWebServices
   {
     init();
     OperatorMeta logicalOperator = dagManager.getLogicalPlan().getOperatorMeta(operatorName);
+    BeanMap operatorProperties = null;
     if (logicalOperator == null) {
-      throw new NotFoundException();
+      ModuleMeta logicalModule = dagManager.getModuleMeta(operatorName);
+      if (logicalModule == null) {
+        throw new NotFoundException();
+      }
+      operatorProperties = LogicalPlanConfiguration.getObjectProperties(logicalModule.getModule());
+    } else {
+      operatorProperties = LogicalPlanConfiguration.getObjectProperties(logicalOperator.getOperator());
     }
 
-    BeanMap operatorProperties = LogicalPlanConfiguration.getObjectProperties(logicalOperator.getOperator());
+    Map<String, Object> m = getPropertiesAsMap(propertyName, operatorProperties);
+    return new JSONObject(objectMapper.writeValueAsString(m));
+  }
+
+  private Map<String, Object> getPropertiesAsMap(@QueryParam("propertyName") String propertyName, BeanMap operatorProperties)
+  {
     Map<String, Object> m = new HashMap<String, Object>();
     @SuppressWarnings("rawtypes")
     Iterator entryIterator = operatorProperties.entryIterator();
     while (entryIterator.hasNext()) {
       try {
         @SuppressWarnings("unchecked")
-        Map.Entry<String, Object> entry = (Map.Entry<String, Object>)entryIterator.next();
+        Entry<String, Object> entry = (Entry<String, Object>)entryIterator.next();
         if (propertyName == null) {
           m.put(entry.getKey(), entry.getValue());
-        }
-        else if (propertyName.equals(entry.getKey())) {
+        } else if (propertyName.equals(entry.getKey())) {
           m.put(entry.getKey(), entry.getValue());
           break;
         }
-      }
-      catch (Exception ex) {
+      } catch (Exception ex) {
         LOG.warn("Caught exception", ex);
       }
     }
-    return new JSONObject(objectMapper.writeValueAsString(m));
+    return m;
   }
 
   @GET
@@ -765,10 +810,10 @@ public class StramWebServices
   @GET
   @Path(PATH_LOGICAL_PLAN)
   @Produces(MediaType.APPLICATION_JSON)
-  public JSONObject getLogicalPlan() throws JSONException, IOException
+  public JSONObject getLogicalPlan(@QueryParam("includeModules") String includeModules) throws JSONException, IOException
   {
-    LogicalPlan lp = dagManager.getLogicalPlan();
-    return new JSONObject(objectMapper.writeValueAsString(LogicalPlanSerializer.convertToMap(lp)));
+    return new JSONObject(objectMapper.writeValueAsString(LogicalPlanSerializer.convertToMap(
+        dagManager.getLogicalPlan(), includeModules != null)));
   }
 
   @POST // not supported by WebAppProxyServlet, can only be called directly


[34/50] incubator-apex-core git commit: APEXCORE-303 Added unit test

Posted by th...@apache.org.
APEXCORE-303 Added unit test


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

Branch: refs/heads/master
Commit: 42ac8ae05b1b4fb2cffc09dcc9dcb17e2c2665c7
Parents: ec0c88b
Author: David Yan <da...@datatorrent.com>
Authored: Fri Jan 8 16:44:08 2016 -0800
Committer: David Yan <da...@datatorrent.com>
Committed: Fri Jan 8 16:44:08 2016 -0800

----------------------------------------------------------------------
 .../stram/client/StramClientUtilsTest.java           | 15 +++++++++++++++
 1 file changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42ac8ae0/engine/src/test/java/com/datatorrent/stram/client/StramClientUtilsTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/client/StramClientUtilsTest.java b/engine/src/test/java/com/datatorrent/stram/client/StramClientUtilsTest.java
index 51193ef..bb67020 100644
--- a/engine/src/test/java/com/datatorrent/stram/client/StramClientUtilsTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/client/StramClientUtilsTest.java
@@ -76,6 +76,21 @@ public class StramClientUtilsTest
     Assert.assertEquals("1111 123 xxx foobar yyy", prop.get("var.result"));
   }
 
+  @Test
+  public void testEvalConfiguration() throws Exception
+  {
+    Configuration conf = new Configuration();
+    conf.set("a.b.c", "123");
+    conf.set("x.y.z", "foobar");
+    conf.set("sub.result", "1111 ${a.b.c} xxx ${x.y.z} yyy");
+    conf.set("script.result", "1111 {% (_prop[\"a.b.c\"] * _prop[\"a.b.c\"]).toFixed(0) %} xxx");
+
+    StramClientUtils.evalConfiguration(conf);
+
+    Assert.assertEquals("1111 123 xxx foobar yyy", conf.get("sub.result"));
+    Assert.assertEquals("1111 15129 xxx", conf.get("script.result"));
+  }
+
   private String getHostString(String host) throws UnknownHostException
   {
     InetAddress address = InetAddress.getByName(host);


[38/50] incubator-apex-core git commit: Merge branch 'APEXCORE-303' of https://github.com/davidyan74/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'APEXCORE-303' of https://github.com/davidyan74/incubator-apex-core into devel-3


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

Branch: refs/heads/master
Commit: ff198ae9705aa1897b161ce95370e923ac1323ea
Parents: 59770d0 42ac8ae
Author: Gaurav Gupta <ga...@apache.org>
Authored: Thu Jan 14 16:03:04 2016 -0800
Committer: Gaurav Gupta <ga...@apache.org>
Committed: Thu Jan 14 16:03:04 2016 -0800

----------------------------------------------------------------------
 .../datatorrent/stram/client/StramAppLauncher.java   |  1 +
 .../datatorrent/stram/client/StramClientUtils.java   | 13 ++++++++++++-
 .../stram/client/StramClientUtilsTest.java           | 15 +++++++++++++++
 3 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------