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 2015/11/03 03:27:02 UTC

[01/50] [abbrv] incubator-apex-core git commit: APEX-92 #Comment #resolve Fix for adding failed nodes to blacklist, after failure is observed for the same node more than MAX number of tries

Repository: incubator-apex-core
Updated Branches:
  refs/heads/master 832901c26 -> 712138ac4


APEX-92 #Comment #resolve
Fix for adding failed nodes to blacklist, after failure is observed for the same node more than MAX number of tries


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/d8e1e74d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/d8e1e74d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/d8e1e74d

Branch: refs/heads/master
Commit: d8e1e74da00619a3c0e021ecd167de57fcc0e262
Parents: 90bda5e
Author: ishark <is...@datatorrent.com>
Authored: Wed Sep 23 16:17:04 2015 -0700
Committer: ishark <is...@datatorrent.com>
Committed: Fri Sep 25 16:09:54 2015 -0700

----------------------------------------------------------------------
 engine/pom.xml                                  |  2 +-
 .../stram/StreamingAppMasterService.java        | 83 ++++++++++++++------
 2 files changed, 62 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d8e1e74d/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 32f2001..15e0565 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>2248</maxAllowedViolations>
+          <maxAllowedViolations>2238</maxAllowedViolations>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d8e1e74d/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 bfeedbd..5d84e10 100644
--- a/engine/src/main/java/com/datatorrent/stram/StreamingAppMasterService.java
+++ b/engine/src/main/java/com/datatorrent/stram/StreamingAppMasterService.java
@@ -32,9 +32,6 @@ import javax.xml.bind.annotation.XmlElement;
 import com.google.common.collect.Maps;
 
 import org.apache.commons.io.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.tuple.MutablePair;
 import org.apache.hadoop.conf.Configuration;
@@ -63,13 +60,15 @@ import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebApps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.datatorrent.api.Attribute;
 import com.datatorrent.api.AutoMetric;
 import com.datatorrent.api.Context.DAGContext;
 import com.datatorrent.api.DAG;
 import com.datatorrent.api.StringCodec;
-
+import com.datatorrent.common.util.Pair;
 import com.datatorrent.stram.StreamingContainerManager.ContainerResource;
 import com.datatorrent.stram.api.AppDataSource;
 import com.datatorrent.stram.api.BaseContext;
@@ -103,11 +102,13 @@ public class StreamingAppMasterService extends CompositeService
   private static final long DELEGATION_TOKEN_RENEW_INTERVAL = Long.MAX_VALUE / 2;
   private static final long DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL = 24 * 60 * 60 * 1000;
   private static final int NUMBER_MISSED_HEARTBEATS = 30;
+  private static final int MAX_CONTAINER_FAILURES_PER_NODE = 3;
+  private static final long BLACKLIST_REMOVAL_TIME = 60 * 60 * 1000;
   private AMRMClient<ContainerRequest> amRmClient;
   private NMClientAsync nmClient;
   private LogicalPlan dag;
   // Application Attempt Id ( combination of attemptId and fail count )
-  final private ApplicationAttemptId appAttemptID;
+  private final ApplicationAttemptId appAttemptID;
   // Hostname of the container
   private final String appMasterHostname = "";
   // Tracking url to which app master publishes info for clients to monitor
@@ -118,6 +119,8 @@ public class StreamingAppMasterService extends CompositeService
   private final AtomicInteger numCompletedContainers = new AtomicInteger();
   // Containers that the RM has allocated to us
   private final ConcurrentMap<String, AllocatedContainer> allocatedContainers = Maps.newConcurrentMap();
+  private final ConcurrentMap<String, AtomicInteger> failedContainersMap = Maps.newConcurrentMap();
+  private final Queue<Pair<Long, List<String>>> blacklistedNodesQueueWithTimeStamp = new ConcurrentLinkedQueue<Pair<Long, List<String>>>();
   // Count of failed containers
   private final AtomicInteger numFailedContainers = new AtomicInteger();
   private final ConcurrentLinkedQueue<Runnable> pendingTasks = new ConcurrentLinkedQueue<Runnable>();
@@ -279,8 +282,7 @@ public class StreamingAppMasterService extends CompositeService
         if (c.getExternalId() == null || c.getState() == PTContainer.State.KILLED) {
           if (c.getRequiredVCores() == 0) {
             result++;
-          }
-          else {
+          } else {
             result += c.getRequiredVCores();
           }
         }
@@ -475,15 +477,12 @@ public class StreamingAppMasterService extends CompositeService
           LOG.info("System CWD content: " + line);
         }
         LOG.info("Dumping files in local dir: end");
-      }
-      finally {
+      } finally {
         buf.close();
       }
-    }
-    catch (IOException e) {
+    } catch (IOException e) {
       LOG.debug("Exception", e);
-    }
-    catch (InterruptedException e) {
+    } catch (InterruptedException e) {
       LOG.info("Interrupted", e);
     }
 
@@ -492,11 +491,9 @@ public class StreamingAppMasterService extends CompositeService
     try {
       // find a better way of logging this using the logger.
       Configuration.dumpConfiguration(getConfig(), new PrintWriter(System.out));
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       LOG.error("Error dumping configuration.", e);
     }
-
   }
 
   @Override
@@ -507,8 +504,7 @@ public class StreamingAppMasterService extends CompositeService
     FileInputStream fis = new FileInputStream("./" + LogicalPlan.SER_FILE_NAME);
     try {
       this.dag = LogicalPlan.read(fis);
-    }
-    finally {
+    } finally {
       fis.close();
     }
     // "debug" simply dumps all data using LOG.info
@@ -651,6 +647,9 @@ public class StreamingAppMasterService extends CompositeService
     int minVcores = conf.getInt("yarn.scheduler.minimum-allocation-vcores", 0);
     LOG.info("Max mem {}m, Min mem {}m, Max vcores {} and Min vcores {} capabililty of resources in this cluster ", maxMem, minMem, maxVcores, minVcores);
 
+    int maxConsecutiveContainerFailures = conf.getInt("MAX_CONSECUTIVE_CONTAINER_FAILURES", MAX_CONTAINER_FAILURES_PER_NODE);
+    long blacklistRemovalTime = conf.getLong("BLACKLIST_REMOVAL_TIME", BLACKLIST_REMOVAL_TIME);
+
     // for locality relaxation fall back
     Map<StreamingContainerAgent.ContainerStartRequest, MutablePair<Integer, ContainerRequest>> requestedResources = Maps.newHashMap();
 
@@ -692,11 +691,9 @@ public class StreamingAppMasterService extends CompositeService
         return;
       }
       resourceRequestor.updateNodeReports(clientRMService.getNodeReports());
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       throw new RuntimeException("Failed to retrieve cluster nodes report.", e);
-    }
-    finally {
+    } finally {
       clientRMService.stop();
     }
 
@@ -781,6 +778,23 @@ public class StreamingAppMasterService extends CompositeService
         }
       }
 
+     /* Remove nodes from blacklist after timeout */
+      long currentTime = System.currentTimeMillis();
+      List<String> blacklistRemovals = new ArrayList<String>();
+      for (Iterator<Pair<Long, List<String>>> it = blacklistedNodesQueueWithTimeStamp.iterator(); it.hasNext();) {
+        Pair<Long, List<String>> entry = it.next();
+        Long timeDiff = currentTime - entry.getFirst();
+        if (timeDiff > blacklistRemovalTime) {
+          blacklistRemovals.addAll(entry.getSecond());
+          it.remove();
+        } else {
+          break;
+        }
+      }
+      if (!blacklistRemovals.isEmpty()) {
+        amRmClient.updateBlacklist(null, blacklistRemovals);
+      }
+
       numTotalContainers += containerRequests.size();
       numRequestedContainers += containerRequests.size();
       AllocateResponse amResp = sendContainerAskToRM(containerRequests, removedContainerRequests, releasedContainers);
@@ -868,6 +882,7 @@ public class StreamingAppMasterService extends CompositeService
       // Check the completed containers
       List<ContainerStatus> completedContainers = amResp.getCompletedContainersStatuses();
       // LOG.debug("Got response from RM for container ask, completedCnt=" + completedContainers.size());
+      List<String> blacklistAdditions = new ArrayList<String>();
       for (ContainerStatus containerStatus : completedContainers) {
         LOG.info("Completed containerId=" + containerStatus.getContainerId() + ", state=" + containerStatus.getState() + ", exitStatus=" + containerStatus.getExitStatus() + ", diagnostics=" + containerStatus.getDiagnostics());
 
@@ -883,6 +898,20 @@ public class StreamingAppMasterService extends CompositeService
         if (0 != exitStatus) {
           if (allocatedContainer != null) {
             numFailedContainers.incrementAndGet();
+            if (exitStatus != 1) {
+              // If container failure due to framework
+              String hostname = allocatedContainer.container.getNodeId().getHost();
+              int failedTimes = 1;
+              AtomicInteger failed = failedContainersMap.putIfAbsent(hostname, new AtomicInteger(1));
+              if (failed != null) {
+                failedTimes = failed.incrementAndGet();
+              }
+              if (failedTimes >= maxConsecutiveContainerFailures) {
+                // Blacklist the node
+                LOG.info("Node {} failed {} times consecutively, marking the node blacklisted", hostname, failedTimes);
+                blacklistAdditions.add(hostname);
+              }
+            }
           }
 //          if (exitStatus == 1) {
 //            // non-recoverable StreamingContainer failure
@@ -902,6 +931,12 @@ public class StreamingAppMasterService extends CompositeService
           // container completed successfully
           numCompletedContainers.incrementAndGet();
           LOG.info("Container completed successfully." + ", containerId=" + containerStatus.getContainerId());
+          // Reset counter for node failure, if exists
+          String hostname = allocatedContainer.container.getNodeId().getHost();
+          AtomicInteger failedTimes = failedContainersMap.get(hostname);
+          if(failedTimes != null) {
+            failedTimes.set(0);
+          }
         }
 
         String containerIdStr = containerStatus.getContainerId().toString();
@@ -913,6 +948,10 @@ public class StreamingAppMasterService extends CompositeService
         dnmgr.recordEventAsync(ev);
       }
 
+      if (!blacklistAdditions.isEmpty()) {
+        amRmClient.updateBlacklist(blacklistAdditions, null);
+        blacklistedNodesQueueWithTimeStamp.add(new Pair<Long, List<String>>(System.currentTimeMillis(), blacklistAdditions));
+      }
       if (dnmgr.forcedShutdown) {
         LOG.info("Forced shutdown due to {}", dnmgr.shutdownDiagnosticsMessage);
         finalStatus = FinalApplicationStatus.FAILED;


[33/50] [abbrv] incubator-apex-core git commit: Add since tags for release 3.2.0

Posted by th...@apache.org.
Add since tags for release 3.2.0


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/42927bc5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/42927bc5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/42927bc5

Branch: refs/heads/master
Commit: 42927bc5c00d8c1a2f75f046434a2fc69594da2e
Parents: 5a90bb7
Author: Thomas Weise <th...@datatorrent.com>
Authored: Tue Oct 20 18:11:36 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Tue Oct 20 18:30:48 2015 -0700

----------------------------------------------------------------------
 api/src/main/java/com/datatorrent/api/annotation/Name.java     | 5 +++++
 .../main/java/com/datatorrent/bufferserver/util/VarInt.java    | 5 +++++
 .../com/datatorrent/common/metric/SingleMetricAggregator.java  | 5 +++++
 .../com/datatorrent/common/metric/sum/DoubleSumAggregator.java | 5 +++++
 .../com/datatorrent/common/metric/sum/LongSumAggregator.java   | 5 +++++
 .../src/main/java/com/datatorrent/stram/api/AppDataSource.java | 5 +++++
 engine/src/main/java/com/datatorrent/stram/engine/Slider.java  | 5 +++++
 .../datatorrent/stram/plan/logical/DefaultKryoStreamCodec.java | 2 ++
 .../stram/plan/logical/StreamCodecWrapperForPersistance.java   | 6 +++++-
 .../stram/stream/PartitionAwareSinkForPersistence.java         | 6 +++++-
 .../main/java/com/datatorrent/stram/webapp/ContainerInfo.java  | 5 +++++
 .../datatorrent/stram/webapp/asm/CompactAnnotationNode.java    | 5 +++++
 .../com/datatorrent/stram/webapp/asm/CompactFieldNode.java     | 5 +++++
 .../datatorrent/stram/webapp/asm/FieldSignatureVisitor.java    | 5 +++++
 14 files changed, 67 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/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 a93f59c..2d03a92 100644
--- a/api/src/main/java/com/datatorrent/api/annotation/Name.java
+++ b/api/src/main/java/com/datatorrent/api/annotation/Name.java
@@ -20,6 +20,11 @@ package com.datatorrent.api.annotation;
 
 import java.lang.annotation.*;
 
+/**
+ * <p>Name class.</p>
+ *
+ * @since 3.2.0
+ */
 @Documented
 @Target(ElementType.TYPE)
 @Retention(RetentionPolicy.RUNTIME)

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/bufferserver/src/main/java/com/datatorrent/bufferserver/util/VarInt.java
----------------------------------------------------------------------
diff --git a/bufferserver/src/main/java/com/datatorrent/bufferserver/util/VarInt.java b/bufferserver/src/main/java/com/datatorrent/bufferserver/util/VarInt.java
index 0c54429..6f12cc4 100644
--- a/bufferserver/src/main/java/com/datatorrent/bufferserver/util/VarInt.java
+++ b/bufferserver/src/main/java/com/datatorrent/bufferserver/util/VarInt.java
@@ -17,6 +17,11 @@
  * under the License.
  */
 package com.datatorrent.bufferserver.util;
+/**
+ * <p>VarInt class.</p>
+ *
+ * @since 3.2.0
+ */
 public class VarInt extends com.datatorrent.netlet.util.VarInt
 {
   public static void read(SerializedData current)

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/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 468e5c6..9b568d2 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,11 @@
 package com.datatorrent.common.metric;
 
 import java.util.Collection;
+/**
+ * <p>SingleMetricAggregator interface.</p>
+ *
+ * @since 3.2.0
+ */
 public interface SingleMetricAggregator
 {
   Object aggregate(Collection<Object> metricValues);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/common/src/main/java/com/datatorrent/common/metric/sum/DoubleSumAggregator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/metric/sum/DoubleSumAggregator.java b/common/src/main/java/com/datatorrent/common/metric/sum/DoubleSumAggregator.java
index ee6df6b..70e6c61 100644
--- a/common/src/main/java/com/datatorrent/common/metric/sum/DoubleSumAggregator.java
+++ b/common/src/main/java/com/datatorrent/common/metric/sum/DoubleSumAggregator.java
@@ -25,6 +25,11 @@ import com.datatorrent.api.annotation.Name;
 
 import com.datatorrent.common.metric.SingleMetricAggregator;
 
+/**
+ * <p>DoubleSumAggregator class.</p>
+ *
+ * @since 3.2.0
+ */
 @Name("sum")
 public class DoubleSumAggregator implements SingleMetricAggregator, Serializable
 {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/common/src/main/java/com/datatorrent/common/metric/sum/LongSumAggregator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/metric/sum/LongSumAggregator.java b/common/src/main/java/com/datatorrent/common/metric/sum/LongSumAggregator.java
index e1cb869..ad88dda 100644
--- a/common/src/main/java/com/datatorrent/common/metric/sum/LongSumAggregator.java
+++ b/common/src/main/java/com/datatorrent/common/metric/sum/LongSumAggregator.java
@@ -25,6 +25,11 @@ import com.datatorrent.api.annotation.Name;
 
 import com.datatorrent.common.metric.SingleMetricAggregator;
 
+/**
+ * <p>LongSumAggregator class.</p>
+ *
+ * @since 3.2.0
+ */
 @Name("sum")
 public class LongSumAggregator implements SingleMetricAggregator, Serializable
 {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/engine/src/main/java/com/datatorrent/stram/api/AppDataSource.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/api/AppDataSource.java b/engine/src/main/java/com/datatorrent/stram/api/AppDataSource.java
index 1246721..e6a4606 100644
--- a/engine/src/main/java/com/datatorrent/stram/api/AppDataSource.java
+++ b/engine/src/main/java/com/datatorrent/stram/api/AppDataSource.java
@@ -21,6 +21,11 @@ package com.datatorrent.stram.api;
 import java.util.Map;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
+/**
+ * <p>AppDataSource class.</p>
+ *
+ * @since 3.2.0
+ */
 @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
 public class AppDataSource
 {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/engine/src/main/java/com/datatorrent/stram/engine/Slider.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/engine/Slider.java b/engine/src/main/java/com/datatorrent/stram/engine/Slider.java
index 27b77e8..d6c1b5c 100644
--- a/engine/src/main/java/com/datatorrent/stram/engine/Slider.java
+++ b/engine/src/main/java/com/datatorrent/stram/engine/Slider.java
@@ -26,6 +26,11 @@ import java.util.List;
 import com.datatorrent.api.*;
 import com.datatorrent.api.Context.OperatorContext;
 import com.datatorrent.api.Operator.Unifier;
+/**
+ * <p>Slider class.</p>
+ *
+ * @since 3.2.0
+ */
 public class Slider implements Unifier<Object>, Operator.IdleTimeHandler, Operator.ActivationListener<OperatorContext>, StatsListener, Serializable, Operator.CheckpointListener
 {
   private List<List<Object>> cache;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/engine/src/main/java/com/datatorrent/stram/plan/logical/DefaultKryoStreamCodec.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/DefaultKryoStreamCodec.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/DefaultKryoStreamCodec.java
index 2dcb998..23009e3 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/DefaultKryoStreamCodec.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/DefaultKryoStreamCodec.java
@@ -36,6 +36,8 @@ import com.datatorrent.netlet.util.Slice;
 /**
  * This codec is used for serializing the objects of class which are Kryo
  * serializable. Used for stream codec wrapper used for persistence
+ *
+ * @since 3.2.0
  */
 public class DefaultKryoStreamCodec<T> extends SerializableObject implements StreamCodec<T>
 {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/engine/src/main/java/com/datatorrent/stram/plan/logical/StreamCodecWrapperForPersistance.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/plan/logical/StreamCodecWrapperForPersistance.java b/engine/src/main/java/com/datatorrent/stram/plan/logical/StreamCodecWrapperForPersistance.java
index 17be686..c6ac5e7 100644
--- a/engine/src/main/java/com/datatorrent/stram/plan/logical/StreamCodecWrapperForPersistance.java
+++ b/engine/src/main/java/com/datatorrent/stram/plan/logical/StreamCodecWrapperForPersistance.java
@@ -29,7 +29,11 @@ import com.datatorrent.api.Partitioner.PartitionKeys;
 import com.datatorrent.api.StreamCodec;
 import com.datatorrent.netlet.util.Slice;
 import com.datatorrent.stram.plan.logical.LogicalPlan.InputPortMeta;
-
+/**
+ * <p>StreamCodecWrapperForPersistance class.</p>
+ *
+ * @since 3.2.0
+ */
 public class StreamCodecWrapperForPersistance<T> implements StreamCodec<T>, Serializable
 {
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/engine/src/main/java/com/datatorrent/stram/stream/PartitionAwareSinkForPersistence.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/stream/PartitionAwareSinkForPersistence.java b/engine/src/main/java/com/datatorrent/stram/stream/PartitionAwareSinkForPersistence.java
index e5a2b77..b6fdbe9 100644
--- a/engine/src/main/java/com/datatorrent/stram/stream/PartitionAwareSinkForPersistence.java
+++ b/engine/src/main/java/com/datatorrent/stram/stream/PartitionAwareSinkForPersistence.java
@@ -23,7 +23,11 @@ import java.util.Set;
 
 import com.datatorrent.api.Sink;
 import com.datatorrent.stram.plan.logical.StreamCodecWrapperForPersistance;
-
+/**
+ * <p>PartitionAwareSinkForPersistence class.</p>
+ *
+ * @since 3.2.0
+ */
 public class PartitionAwareSinkForPersistence extends PartitionAwareSink<Object>
 {
   StreamCodecWrapperForPersistance<Object> serdeForPersistence;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/engine/src/main/java/com/datatorrent/stram/webapp/ContainerInfo.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/webapp/ContainerInfo.java b/engine/src/main/java/com/datatorrent/stram/webapp/ContainerInfo.java
index b3e15c3..06300ce 100644
--- a/engine/src/main/java/com/datatorrent/stram/webapp/ContainerInfo.java
+++ b/engine/src/main/java/com/datatorrent/stram/webapp/ContainerInfo.java
@@ -24,6 +24,11 @@ import javax.xml.bind.annotation.XmlRootElement;
 
 import com.datatorrent.api.annotation.RecordField;
 
+/**
+ * <p>ContainerInfo class.</p>
+ *
+ * @since 3.2.0
+ */
 @XmlRootElement(name = "container")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class ContainerInfo {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactAnnotationNode.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactAnnotationNode.java b/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactAnnotationNode.java
index 99ddd75..7860378 100644
--- a/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactAnnotationNode.java
+++ b/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactAnnotationNode.java
@@ -19,6 +19,11 @@
 package com.datatorrent.stram.webapp.asm;
 
 import java.util.Map;
+/**
+ * <p>CompactAnnotationNode class.</p>
+ *
+ * @since 3.2.0
+ */
 public class CompactAnnotationNode {
 
   private String desc;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactFieldNode.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactFieldNode.java b/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactFieldNode.java
index bd5991e..778fc41 100644
--- a/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactFieldNode.java
+++ b/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactFieldNode.java
@@ -23,6 +23,11 @@ import java.util.List;
 
 import org.apache.xbean.asm5.tree.AnnotationNode;
 import org.apache.xbean.asm5.tree.FieldNode;
+/**
+ * <p>CompactFieldNode class.</p>
+ *
+ * @since 3.2.0
+ */
 public class CompactFieldNode {
   private String name;
   private String desc;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/42927bc5/engine/src/main/java/com/datatorrent/stram/webapp/asm/FieldSignatureVisitor.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/webapp/asm/FieldSignatureVisitor.java b/engine/src/main/java/com/datatorrent/stram/webapp/asm/FieldSignatureVisitor.java
index c58fe7b..24b71d9 100644
--- a/engine/src/main/java/com/datatorrent/stram/webapp/asm/FieldSignatureVisitor.java
+++ b/engine/src/main/java/com/datatorrent/stram/webapp/asm/FieldSignatureVisitor.java
@@ -20,6 +20,11 @@ package com.datatorrent.stram.webapp.asm;
 
 
 import org.apache.xbean.asm5.signature.SignatureVisitor;
+/**
+ * <p>FieldSignatureVisitor class.</p>
+ *
+ * @since 3.2.0
+ */
 public class FieldSignatureVisitor extends BaseSignatureVisitor{
 
   private Type fieldType;


[38/50] [abbrv] incubator-apex-core git commit: Fix version lookup.

Posted by th...@apache.org.
Fix version lookup.


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/0aaec0c5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/0aaec0c5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/0aaec0c5

Branch: refs/heads/master
Commit: 0aaec0c56db8c635b167d4cb4bd1f8b206b649d2
Parents: e562da3
Author: Thomas Weise <th...@datatorrent.com>
Authored: Wed Oct 21 12:55:58 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Wed Oct 21 12:55:58 2015 -0700

----------------------------------------------------------------------
 engine/src/main/java/com/datatorrent/stram/util/VersionInfo.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/0aaec0c5/engine/src/main/java/com/datatorrent/stram/util/VersionInfo.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/util/VersionInfo.java b/engine/src/main/java/com/datatorrent/stram/util/VersionInfo.java
index cd1bc3b..1321491 100644
--- a/engine/src/main/java/com/datatorrent/stram/util/VersionInfo.java
+++ b/engine/src/main/java/com/datatorrent/stram/util/VersionInfo.java
@@ -57,7 +57,7 @@ public class VersionInfo {
         }
       }
 
-      Enumeration<URL> resources = VersionInfo.class.getClassLoader().getResources("META-INF/maven/com.datatorrent/dt-engine/pom.properties");
+      Enumeration<URL> resources = VersionInfo.class.getClassLoader().getResources("META-INF/maven/org.apache.apex/apex-engine/pom.properties");
       while (resources.hasMoreElements()) {
         Properties pomInfo = new Properties();
         pomInfo.load(resources.nextElement().openStream());


[20/50] [abbrv] incubator-apex-core git commit: Merge branch 'apex-116' of https://github.com/PramodSSImmaneni/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'apex-116' 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/2bd6a8db
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/2bd6a8db
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/2bd6a8db

Branch: refs/heads/master
Commit: 2bd6a8db567964037dad0c5b8fcbe2bdcec88fb7
Parents: 88ee4ad 45e891c
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Sun Oct 4 10:29:25 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Sun Oct 4 10:29:25 2015 -0700

----------------------------------------------------------------------
 .../stram/client/StramClientUtils.java          | 28 +++++++++++++-------
 .../com/datatorrent/stram/util/ConfigUtils.java |  1 +
 2 files changed, 20 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/2bd6a8db/engine/src/main/java/com/datatorrent/stram/client/StramClientUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/2bd6a8db/engine/src/main/java/com/datatorrent/stram/util/ConfigUtils.java
----------------------------------------------------------------------


[08/50] [abbrv] incubator-apex-core git commit: Merge branch 'APEX-159' of https://github.com/vrozov/incubator-apex-core into APEX-159

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

Conflicts:
	engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java


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/8e49cfb1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/8e49cfb1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/8e49cfb1

Branch: refs/heads/master
Commit: 8e49cfb1a9f1ccd818016ad6f169ebfde0256775
Parents: fb53705 0fc22a0
Author: Thomas Weise <th...@datatorrent.com>
Authored: Sun Sep 27 17:28:45 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sun Sep 27 17:28:45 2015 -0700

----------------------------------------------------------------------
 .../stram/LaunchContainerRunnable.java          | 27 +++++++-------------
 .../java/com/datatorrent/stram/StramClient.java | 10 +-------
 2 files changed, 10 insertions(+), 27 deletions(-)
----------------------------------------------------------------------



[45/50] [abbrv] incubator-apex-core git commit: Merge branch 'APEX-217' of https://github.com/vrozov/incubator-apex-core into release-3.2

Posted by th...@apache.org.
Merge branch 'APEX-217' of https://github.com/vrozov/incubator-apex-core into release-3.2


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/8ef3f6e5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/8ef3f6e5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/8ef3f6e5

Branch: refs/heads/master
Commit: 8ef3f6e514f0fd84dbace789cae9ef01d9270cb9
Parents: d9b2af0 eb8cc74
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Oct 22 21:50:13 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Oct 22 21:50:13 2015 -0700

----------------------------------------------------------------------
 NOTICE                                                            | 1 +
 apex-app-archetype/src/main/appended-resources/META-INF/NOTICE.vm | 3 +++
 .../src/main/appended-resources/META-INF/NOTICE.vm                | 3 +++
 api/src/main/appended-resources/META-INF/NOTICE.vm                | 3 +++
 bufferserver/src/main/appended-resources/META-INF/NOTICE.vm       | 3 +++
 common/src/main/appended-resources/META-INF/NOTICE.vm             | 3 +++
 engine/src/main/appended-resources/META-INF/NOTICE.vm             | 3 +++
 pom.xml                                                           | 3 +++
 8 files changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/8ef3f6e5/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index ee77ae6,f20e85f..5be2636
--- a/pom.xml
+++ b/pom.xml
@@@ -358,9 -360,9 +360,10 @@@
          <distMgmtReleaseId>apache.staging.https</distMgmtReleaseId>
          <distMgmtReleaseUrl>https://repository.apache.org/service/local/staging/deploy/maven2</distMgmtReleaseUrl>
          <distMgmtDevUrl>file://${project.build.directory}/mvn-repo</distMgmtDevUrl>
 +        <sourceReleaseAssemblyDescriptor>source-release-zip-tar</sourceReleaseAssemblyDescriptor>
          <package.prefix>/opt/datatorrent</package.prefix>
          <package.groupname>dtorrent</package.groupname>
+         <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>
        <distributionManagement>
          <repository>


[37/50] [abbrv] incubator-apex-core git commit: Merge branch 'release-3.2' of github.com:vrozov/incubator-apex-core into release-3.2

Posted by th...@apache.org.
Merge branch 'release-3.2' of github.com:vrozov/incubator-apex-core into release-3.2


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/e562da31
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/e562da31
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/e562da31

Branch: refs/heads/master
Commit: e562da313c59e65c76c8509671fd4df2f95b6805
Parents: badcf8a 4c6d3f5
Author: Pramod Immaneni <pr...@datatorrent.com>
Authored: Wed Oct 21 06:20:37 2015 -0700
Committer: Pramod Immaneni <pr...@datatorrent.com>
Committed: Wed Oct 21 06:20:37 2015 -0700

----------------------------------------------------------------------
 bufferserver/pom.xml                            |   2 +-
 .../bufferserver/internal/DataList.java         | 248 ++++++++++++-------
 .../bufferserver/internal/DataListener.java     |   2 +-
 .../bufferserver/internal/FastDataList.java     |  11 +-
 .../bufferserver/internal/LogicalNode.java      |  37 ++-
 .../datatorrent/bufferserver/server/Server.java | 108 ++++----
 6 files changed, 232 insertions(+), 176 deletions(-)
----------------------------------------------------------------------



[28/50] [abbrv] incubator-apex-core git commit: SPOI-6505 #resolve Added check for presence of @OutputPortFieldAnnotation.

Posted by th...@apache.org.
SPOI-6505 #resolve Added check for presence of @OutputPortFieldAnnotation.


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/dc40027c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/dc40027c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/dc40027c

Branch: refs/heads/master
Commit: dc40027c0b7c6f11393efaa25f361fe2ae27f372
Parents: 83724f6
Author: ishark <is...@datatorrent.com>
Authored: Wed Oct 14 17:33:22 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sat Oct 17 20:08:56 2015 -0700

----------------------------------------------------------------------
 .../com/datatorrent/stram/webapp/asm/CompactUtil.java | 14 +++++++++-----
 1 file changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/dc40027c/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactUtil.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactUtil.java b/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactUtil.java
index b0611b3..0b55578 100644
--- a/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactUtil.java
+++ b/engine/src/main/java/com/datatorrent/stram/webapp/asm/CompactUtil.java
@@ -153,12 +153,16 @@ public class CompactUtil
         if (annotation.desc.contains("InputPortFieldAnnotation")
             || annotation.desc.contains("OutputPortFieldAnnotation")) {
           List<Object> annotationValues = annotation.values;
-          int index = 0;
-          while (index < annotationValues.size()) {
-            annotationMap.put((String) annotationValues.get(index++), annotationValues.get(index++));
+          if (annotationValues != null) {
+            int index = 0;
+            while (index <= annotationValues.size() - 2) {
+              String key = (String)annotationValues.get(index++);
+              Object value = annotationValues.get(index++);
+              annotationMap.put(key, value);
+            }
+            node.setAnnotations(annotationMap);
+            annotations.add(node);
           }
-          node.setAnnotations(annotationMap);
-          annotations.add(node);
         }
       }
       cfn.setVisibleAnnotations(annotations);


[23/50] [abbrv] incubator-apex-core git commit: APEX-178 #resolve #comment added checkstyle rule that fails when wildcards imports are added

Posted by th...@apache.org.
APEX-178 #resolve #comment added checkstyle rule that fails when wildcards imports are added


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/b986f70e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/b986f70e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/b986f70e

Branch: refs/heads/master
Commit: b986f70e45e6ab091fff6f0f9cae625eccccd8b7
Parents: 8104e25
Author: Chandni Singh <cs...@apache.org>
Authored: Sun Oct 4 22:35:34 2015 -0700
Committer: Chandni Singh <cs...@apache.org>
Committed: Sun Oct 4 22:35:34 2015 -0700

----------------------------------------------------------------------
 apex_checks.xml      | 2 +-
 api/pom.xml          | 2 +-
 bufferserver/pom.xml | 2 +-
 common/pom.xml       | 2 +-
 engine/pom.xml       | 2 +-
 5 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b986f70e/apex_checks.xml
----------------------------------------------------------------------
diff --git a/apex_checks.xml b/apex_checks.xml
index 7ba9ea4..3bdc27e 100644
--- a/apex_checks.xml
+++ b/apex_checks.xml
@@ -141,7 +141,7 @@
       <property name="option" value="bottom"/>
       <property name="sortStaticImportsAlphabetically" value="true"/>
     </module>
-
+    <module name="AvoidStarImport"/>
     <module name="RedundantImport"/>
     <module name="UnusedImports"/>
     <module name="MethodParamPad"/>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b986f70e/api/pom.xml
----------------------------------------------------------------------
diff --git a/api/pom.xml b/api/pom.xml
index 067c40a..8c330dc 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -89,7 +89,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>42</maxAllowedViolations>
+          <maxAllowedViolations>50</maxAllowedViolations>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b986f70e/bufferserver/pom.xml
----------------------------------------------------------------------
diff --git a/bufferserver/pom.xml b/bufferserver/pom.xml
index 773add8..6317470 100644
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@ -51,7 +51,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>121</maxAllowedViolations>
+          <maxAllowedViolations>124</maxAllowedViolations>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b986f70e/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 12a1d30..520d09a 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -74,7 +74,7 @@
       <groupId>org.apache.maven.plugins</groupId>
       <artifactId>maven-checkstyle-plugin</artifactId>
       <configuration>
-        <maxAllowedViolations>76</maxAllowedViolations>
+        <maxAllowedViolations>87</maxAllowedViolations>
       </configuration>
     </plugin>
   </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b986f70e/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index ae334f2..3efd33f 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>2209</maxAllowedViolations>
+          <maxAllowedViolations>2322</maxAllowedViolations>
         </configuration>
       </plugin>
     </plugins>


[44/50] [abbrv] incubator-apex-core git commit: APEX-217 #resolve added NOTICE.vm that generates original copyright in NOTICE. The workaround suggested in https://issues.apache.org/jira/browse/MASFRES-5.

Posted by th...@apache.org.
APEX-217 #resolve added NOTICE.vm that generates original copyright in NOTICE. The workaround suggested in https://issues.apache.org/jira/browse/MASFRES-5.


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/eb8cc746
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/eb8cc746
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/eb8cc746

Branch: refs/heads/master
Commit: eb8cc746937bf4562450cfea0f5195150592dd01
Parents: ce44d23
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Thu Oct 22 21:17:51 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Thu Oct 22 21:26:30 2015 -0700

----------------------------------------------------------------------
 NOTICE                                                            | 1 +
 apex-app-archetype/src/main/appended-resources/META-INF/NOTICE.vm | 3 +++
 .../src/main/appended-resources/META-INF/NOTICE.vm                | 3 +++
 api/src/main/appended-resources/META-INF/NOTICE.vm                | 3 +++
 bufferserver/src/main/appended-resources/META-INF/NOTICE.vm       | 3 +++
 common/src/main/appended-resources/META-INF/NOTICE.vm             | 3 +++
 engine/src/main/appended-resources/META-INF/NOTICE.vm             | 3 +++
 pom.xml                                                           | 3 +++
 8 files changed, 22 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/eb8cc746/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index eef5e86..1dfedde 100644
--- a/NOTICE
+++ b/NOTICE
@@ -4,6 +4,7 @@ Copyright (c) 2015 The Apache Software Foundation
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
 
+
 The initial developer of the original code is
 DataTorrent, Inc. (http://www.datatorrent.com)
 Copyright (c) 2012 - 2015. All Rights Reserved.

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/eb8cc746/apex-app-archetype/src/main/appended-resources/META-INF/NOTICE.vm
----------------------------------------------------------------------
diff --git a/apex-app-archetype/src/main/appended-resources/META-INF/NOTICE.vm b/apex-app-archetype/src/main/appended-resources/META-INF/NOTICE.vm
new file mode 100644
index 0000000..7edf492
--- /dev/null
+++ b/apex-app-archetype/src/main/appended-resources/META-INF/NOTICE.vm
@@ -0,0 +1,3 @@
+#if($project.properties.postNoticeText)
+$project.properties.postNoticeText
+#end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/eb8cc746/apex-conf-archetype/src/main/appended-resources/META-INF/NOTICE.vm
----------------------------------------------------------------------
diff --git a/apex-conf-archetype/src/main/appended-resources/META-INF/NOTICE.vm b/apex-conf-archetype/src/main/appended-resources/META-INF/NOTICE.vm
new file mode 100644
index 0000000..7edf492
--- /dev/null
+++ b/apex-conf-archetype/src/main/appended-resources/META-INF/NOTICE.vm
@@ -0,0 +1,3 @@
+#if($project.properties.postNoticeText)
+$project.properties.postNoticeText
+#end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/eb8cc746/api/src/main/appended-resources/META-INF/NOTICE.vm
----------------------------------------------------------------------
diff --git a/api/src/main/appended-resources/META-INF/NOTICE.vm b/api/src/main/appended-resources/META-INF/NOTICE.vm
new file mode 100644
index 0000000..7edf492
--- /dev/null
+++ b/api/src/main/appended-resources/META-INF/NOTICE.vm
@@ -0,0 +1,3 @@
+#if($project.properties.postNoticeText)
+$project.properties.postNoticeText
+#end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/eb8cc746/bufferserver/src/main/appended-resources/META-INF/NOTICE.vm
----------------------------------------------------------------------
diff --git a/bufferserver/src/main/appended-resources/META-INF/NOTICE.vm b/bufferserver/src/main/appended-resources/META-INF/NOTICE.vm
new file mode 100644
index 0000000..7edf492
--- /dev/null
+++ b/bufferserver/src/main/appended-resources/META-INF/NOTICE.vm
@@ -0,0 +1,3 @@
+#if($project.properties.postNoticeText)
+$project.properties.postNoticeText
+#end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/eb8cc746/common/src/main/appended-resources/META-INF/NOTICE.vm
----------------------------------------------------------------------
diff --git a/common/src/main/appended-resources/META-INF/NOTICE.vm b/common/src/main/appended-resources/META-INF/NOTICE.vm
new file mode 100644
index 0000000..7edf492
--- /dev/null
+++ b/common/src/main/appended-resources/META-INF/NOTICE.vm
@@ -0,0 +1,3 @@
+#if($project.properties.postNoticeText)
+$project.properties.postNoticeText
+#end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/eb8cc746/engine/src/main/appended-resources/META-INF/NOTICE.vm
----------------------------------------------------------------------
diff --git a/engine/src/main/appended-resources/META-INF/NOTICE.vm b/engine/src/main/appended-resources/META-INF/NOTICE.vm
new file mode 100644
index 0000000..7edf492
--- /dev/null
+++ b/engine/src/main/appended-resources/META-INF/NOTICE.vm
@@ -0,0 +1,3 @@
+#if($project.properties.postNoticeText)
+$project.properties.postNoticeText
+#end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/eb8cc746/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1fdabcf..f20e85f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -109,6 +109,7 @@
             <exclude>DISCLAIMER</exclude>
             <exclude>LICENSE</exclude>
             <exclude>NOTICE</exclude>
+            <exclude>**/NOTICE.vm</exclude>
             <exclude>**/*.md</exclude>
             <exclude>**/*.txt</exclude>
             <exclude>**/*.importorder</exclude>
@@ -191,6 +192,7 @@
             <exclude>**/*.md</exclude>
             <exclude>**/*.txt</exclude>
             <exclude>**/*.importorder</exclude>
+            <exclude>**/NOTICE.vm</exclude>
           </excludes>
         </configuration>
       </plugin>
@@ -360,6 +362,7 @@
         <distMgmtDevUrl>file://${project.build.directory}/mvn-repo</distMgmtDevUrl>
         <package.prefix>/opt/datatorrent</package.prefix>
         <package.groupname>dtorrent</package.groupname>
+        <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>
       <distributionManagement>
         <repository>


[26/50] [abbrv] incubator-apex-core git commit: APEX-190 Replace Maven group and artifact ID, configure ASF repositories, use apache-release profile.

Posted by th...@apache.org.
APEX-190 Replace Maven group and artifact ID, configure ASF repositories, use apache-release profile.


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/51ce8f95
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/51ce8f95
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/51ce8f95

Branch: refs/heads/master
Commit: 51ce8f95d784a07d183cc54f32033c5115f72023
Parents: 4007ec5
Author: Thomas Weise <th...@datatorrent.com>
Authored: Tue Oct 13 17:15:27 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Tue Oct 13 17:15:27 2015 -0700

----------------------------------------------------------------------
 apex-app-archetype/README.md                    |  2 +-
 apex-app-archetype/pom.xml                      |  8 +--
 .../main/resources/archetype-resources/pom.xml  | 25 ++++---
 apex-conf-archetype/README.md                   |  2 +-
 apex-conf-archetype/pom.xml                     |  8 +--
 api/pom.xml                                     | 12 ++--
 bufferserver/pom.xml                            | 14 ++--
 common/pom.xml                                  | 12 ++--
 engine/pom.xml                                  | 12 ++--
 engine/src/main/scripts/dtcli                   |  2 +-
 .../stram/client/AppPackageTest.java            |  2 +-
 .../resources/testAppPackage/mydtapp/pom.xml    |  6 +-
 pom.xml                                         | 73 ++++++--------------
 13 files changed, 77 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/apex-app-archetype/README.md
----------------------------------------------------------------------
diff --git a/apex-app-archetype/README.md b/apex-app-archetype/README.md
index 775f026..3ca588f 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=com.datatorrent -DarchetypeArtifactId=apex-app-archetype -DarchetypeVersion=3.0.0 -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.2.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/51ce8f95/apex-app-archetype/pom.xml
----------------------------------------------------------------------
diff --git a/apex-app-archetype/pom.xml b/apex-app-archetype/pom.xml
index 7c9f332..f88dabf 100644
--- a/apex-app-archetype/pom.xml
+++ b/apex-app-archetype/pom.xml
@@ -23,15 +23,15 @@
   <modelVersion>4.0.0</modelVersion>
 
   <parent>
-    <groupId>com.datatorrent</groupId>
-    <artifactId>dt-framework</artifactId>
-    <version>3.2.0-SNAPSHOT</version>
+    <groupId>org.apache.apex</groupId>
+    <artifactId>apex</artifactId>
+    <version>3.2.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>apex-app-archetype</artifactId>
   <packaging>maven-archetype</packaging>
 
-  <name>Apex Application Maven archetype</name>
+  <name>Apache Apex Application Maven Archetype</name>
 
   <build>
     <extensions>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/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 33d1d1b..d3d0353 100644
--- a/apex-app-archetype/src/main/resources/archetype-resources/pom.xml
+++ b/apex-app-archetype/src/main/resources/archetype-resources/pom.xml
@@ -13,7 +13,7 @@
 
   <properties>
     <!-- change this if you desire to use a different version of DataTorrent -->
-    <datatorrent.version>${archetypeVersion}</datatorrent.version>
+    <apex.version>${archetypeVersion}</apex.version>
     <datatorrent.apppackage.classpath>lib/*.jar</datatorrent.apppackage.classpath>
   </properties>
 
@@ -99,7 +99,7 @@
                <archive>
                  <manifestEntries>
                    <Class-Path>${datatorrent.apppackage.classpath}</Class-Path>
-                   <DT-Engine-Version>${datatorrent.version}</DT-Engine-Version>
+                   <DT-Engine-Version>${apex.version}</DT-Engine-Version>
                    <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>
@@ -256,7 +256,14 @@
     <dependency>
       <groupId>com.datatorrent</groupId>
       <artifactId>malhar-library</artifactId>
-      <version>${datatorrent.version}</version>
+      <version>3.1.1</version>
+      <!-- remove when switching to malhar version that depends on apex-common as shown below -->
+      <exclusions>
+        <exclusion>
+          <groupId>com.datatorrent</groupId>
+          <artifactId>dt-common</artifactId>
+        </exclusion>
+      </exclusions>
       <!-- 
            If you know that your application does not need transitive dependencies pulled in by malhar-library,
            uncomment the following to reduce the size of your app package.
@@ -271,9 +278,9 @@
       -->
     </dependency>
     <dependency>
-      <groupId>com.datatorrent</groupId>
-      <artifactId>dt-common</artifactId>
-      <version>${datatorrent.version}</version>
+      <groupId>org.apache.apex</groupId>
+      <artifactId>apex-common</artifactId>
+      <version>${apex.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
@@ -283,9 +290,9 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>com.datatorrent</groupId>
-      <artifactId>dt-engine</artifactId>
-      <version>${datatorrent.version}</version>
+      <groupId>org.apache.apex</groupId>
+      <artifactId>apex-engine</artifactId>
+      <version>${apex.version}</version>
       <scope>test</scope>
     </dependency>
   </dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/apex-conf-archetype/README.md
----------------------------------------------------------------------
diff --git a/apex-conf-archetype/README.md b/apex-conf-archetype/README.md
index d54876c..2bcd18a 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=com.datatorrent -DarchetypeArtifactId=apex-conf-archetype -DarchetypeVersion=3.0.0 -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.2.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/51ce8f95/apex-conf-archetype/pom.xml
----------------------------------------------------------------------
diff --git a/apex-conf-archetype/pom.xml b/apex-conf-archetype/pom.xml
index e5b4d3e..7caf93f 100644
--- a/apex-conf-archetype/pom.xml
+++ b/apex-conf-archetype/pom.xml
@@ -23,15 +23,15 @@
   <modelVersion>4.0.0</modelVersion>
 
   <parent>
-    <groupId>com.datatorrent</groupId>
-    <artifactId>dt-framework</artifactId>
-    <version>3.2.0-SNAPSHOT</version>
+    <groupId>org.apache.apex</groupId>
+    <artifactId>apex</artifactId>
+    <version>3.2.0-incubating-SNAPSHOT</version>
   </parent>
 
   <artifactId>apex-conf-archetype</artifactId>
   <packaging>maven-archetype</packaging>
 
-  <name>Apex App Configuration Maven archetype</name>
+  <name>Apache Apex App Configuration Maven Archetype</name>
 
   <build>
     <extensions>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/api/pom.xml
----------------------------------------------------------------------
diff --git a/api/pom.xml b/api/pom.xml
index 8c330dc..a44d8a6 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -24,13 +24,13 @@
   <modelVersion>4.0.0</modelVersion>
 
   <parent>
-    <groupId>com.datatorrent</groupId>
-    <artifactId>dt-framework</artifactId>
-    <version>3.2.0-SNAPSHOT</version>
+    <groupId>org.apache.apex</groupId>
+    <artifactId>apex</artifactId>
+    <version>3.2.0-incubating-SNAPSHOT</version>
   </parent>
 
-  <artifactId>dt-api</artifactId>
-  <name>DataTorrent API</name>
+  <artifactId>apex-api</artifactId>
+  <name>Apache Apex API</name>
   <packaging>jar</packaging>
 
   <repositories>
@@ -139,7 +139,7 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>${project.groupId}</groupId>
+      <groupId>com.datatorrent</groupId>
       <artifactId>netlet</artifactId>
       <version>1.2.0</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/bufferserver/pom.xml
----------------------------------------------------------------------
diff --git a/bufferserver/pom.xml b/bufferserver/pom.xml
index 6317470..1346ba7 100644
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@ -23,15 +23,15 @@
   <modelVersion>4.0.0</modelVersion>
 
   <parent>
-    <groupId>com.datatorrent</groupId>
-    <artifactId>dt-framework</artifactId>
-    <version>3.2.0-SNAPSHOT</version>
+    <groupId>org.apache.apex</groupId>
+    <artifactId>apex</artifactId>
+    <version>3.2.0-incubating-SNAPSHOT</version>
   </parent>
 
-  <artifactId>dt-bufferserver</artifactId>
+  <artifactId>apex-bufferserver</artifactId>
   <description>Interface for nodes from 2 different containers to talk to each other.</description>
 
-  <name>Buffer Server</name>
+  <name>Apache Apex Buffer Server</name>
 
   <build>
     <plugins>
@@ -65,8 +65,8 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>com.datatorrent</groupId>
-      <artifactId>dt-common</artifactId>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>apex-common</artifactId>
       <version>${project.version}</version>
       <type>jar</type>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 520d09a..6ee0305 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -23,13 +23,13 @@
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>com.datatorrent</groupId>
-    <artifactId>dt-framework</artifactId>
-    <version>3.2.0-SNAPSHOT</version>
+    <groupId>org.apache.apex</groupId>
+    <artifactId>apex</artifactId>
+    <version>3.2.0-incubating-SNAPSHOT</version>
   </parent>
 
-  <artifactId>dt-common</artifactId>
-  <name>Base Library</name>
+  <artifactId>apex-common</artifactId>
+  <name>Apache Apex Common Library</name>
   <packaging>jar</packaging>
 
   <build>
@@ -83,7 +83,7 @@
   <dependencies>
     <dependency>
       <groupId>${project.groupId}</groupId>
-      <artifactId>dt-api</artifactId>
+      <artifactId>apex-api</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 3efd33f..5d30a8e 100644
--- a/engine/pom.xml
+++ b/engine/pom.xml
@@ -23,15 +23,15 @@
   <modelVersion>4.0.0</modelVersion>
 
   <parent>
-    <groupId>com.datatorrent</groupId>
-    <artifactId>dt-framework</artifactId>
-    <version>3.2.0-SNAPSHOT</version>
+    <groupId>org.apache.apex</groupId>
+    <artifactId>apex</artifactId>
+    <version>3.2.0-incubating-SNAPSHOT</version>
   </parent>
 
-  <artifactId>dt-engine</artifactId>
+  <artifactId>apex-engine</artifactId>
   <packaging>jar</packaging>
 
-  <name>Streaming Application Manager</name>
+  <name>Apache Apex Stream Processing Engine</name>
 
   <build>
     <finalName>${project.artifactId}</finalName>
@@ -165,7 +165,7 @@
     </dependency>
     <dependency>
       <groupId>${project.groupId}</groupId>
-      <artifactId>dt-bufferserver</artifactId>
+      <artifactId>apex-bufferserver</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/engine/src/main/scripts/dtcli
----------------------------------------------------------------------
diff --git a/engine/src/main/scripts/dtcli b/engine/src/main/scripts/dtcli
index d7f9d76..e254215 100755
--- a/engine/src/main/scripts/dtcli
+++ b/engine/src/main/scripts/dtcli
@@ -67,7 +67,7 @@ fi
 
 if [ -f "$MVN_GENERATED_PATH" ]; then
   # development launch mode
-  DT_CORE_JAR="$BUILD_DIR/dt-engine.jar"
+  DT_CORE_JAR="$BUILD_DIR/apex-engine.jar"
   if [ ! -f "$DT_CORE_JAR" ]; then
     echoerr "Error: Cannot find $DT_CORE_JAR";
     exit 1;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/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 1224115..8fb4b1b 100644
--- a/engine/src/test/java/com/datatorrent/stram/client/AppPackageTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/client/AppPackageTest.java
@@ -84,7 +84,7 @@ public class AppPackageTest
   {
     Assert.assertEquals("mydtapp", json.getString("appPackageName"));
     Assert.assertEquals("1.0-SNAPSHOT", json.getString("appPackageVersion"));
-    Assert.assertEquals("3.0.0", json.getString("dtEngineVersion"));
+    Assert.assertEquals("3.2.0-incubating-SNAPSHOT", json.getString("dtEngineVersion"));
     Assert.assertEquals("lib/*.jar", json.getJSONArray("classPath").getString(0));
 
     JSONObject application = json.getJSONArray("applications").getJSONObject(0);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/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 abbca75..3b1e799 100644
--- a/engine/src/test/resources/testAppPackage/mydtapp/pom.xml
+++ b/engine/src/test/resources/testAppPackage/mydtapp/pom.xml
@@ -33,7 +33,7 @@
 
   <properties>
     <!-- change this if you desire to use a different version of DataTorrent -->
-    <datatorrent.version>3.0.0</datatorrent.version>
+    <datatorrent.version>3.2.0-incubating-SNAPSHOT</datatorrent.version>
     <datatorrent.apppackage.classpath>lib/*.jar</datatorrent.apppackage.classpath>
   </properties>
 
@@ -184,8 +184,8 @@
   <dependencies>
     <!-- add your dependencies here -->
     <dependency>
-      <groupId>com.datatorrent</groupId>
-      <artifactId>dt-common</artifactId>
+      <groupId>org.apache.apex</groupId>
+      <artifactId>apex-common</artifactId>
       <version>${datatorrent.version}</version>
       <scope>provided</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/51ce8f95/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2c14ce8..a7ccc90 100644
--- a/pom.xml
+++ b/pom.xml
@@ -22,9 +22,15 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
 
-  <groupId>com.datatorrent</groupId>
-  <artifactId>dt-framework</artifactId>
-  <version>3.2.0-SNAPSHOT</version>
+  <parent>
+    <groupId>org.apache</groupId>
+    <artifactId>apache</artifactId>
+    <version>16</version>
+  </parent>
+
+  <groupId>org.apache.apex</groupId>
+  <artifactId>apex</artifactId>
+  <version>3.2.0-incubating-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <name>Apache Apex Core</name>
@@ -43,6 +49,7 @@
   </licenses>
 
   <repositories>
+    <!-- added for semantic versioning check, won't be needed once we publish to central -->
     <repository>
       <snapshots>
         <enabled>false</enabled>
@@ -63,7 +70,7 @@
     <jersey.version>1.9</jersey.version>
     <!-- do not change jetty version as later versions have problems with DefaultServlet -->
     <jetty.version>8.1.10.v20130312</jetty.version>
-    <maven.deploy.repo.classifier></maven.deploy.repo.classifier>
+    <maven.deploy.repo.classifier/>
     <license.skip>true</license.skip>
     <checkstyle.console>false</checkstyle.console>
   </properties>
@@ -340,30 +347,29 @@
   </build>
   <profiles>
     <profile>
-      <id>release</id>
+      <id>apache-release</id>
       <modules>
       </modules>
       <properties>
         <maven.compiler.debug>false</maven.compiler.debug>
         <maven.compiler.optimize>true</maven.compiler.optimize>
         <maven.deploy.skip>false</maven.deploy.skip>
-        <!-- always generate on release build -->
-        <generate-revision-info-phase>generate-resources</generate-revision-info-phase>
-        <distMgmtDevUrl>internal.repo::default::file://${project.build.directory}/mvn-repo</distMgmtDevUrl>
-        <distMgmtUrl>dav:http://www.datatorrent.com:8081/nexus/content/repositories</distMgmtUrl>
+        <distMgmtSnapshotsId>apache.snapshots.https</distMgmtSnapshotsId>
+        <distMgmtSnapshotsUrl>https://repository.apache.org/content/repositories/snapshots</distMgmtSnapshotsUrl>
+        <distMgmtReleaseId>apache.staging.https</distMgmtReleaseId>
+        <distMgmtReleaseUrl>https://repository.apache.org/service/local/staging/deploy/maven2</distMgmtReleaseUrl>
+        <distMgmtDevUrl>file://${project.build.directory}/mvn-repo</distMgmtDevUrl>
         <package.prefix>/opt/datatorrent</package.prefix>
         <package.groupname>dtorrent</package.groupname>
       </properties>
       <distributionManagement>
         <repository>
-          <id>releases</id>
-          <name>DataTorrent Repository</name>
-          <url>${distMgmtUrl}/${maven.deploy.repo.classifier}releases/</url>
+          <id>${distMgmtReleaseId}</id>
+          <url>${distMgmtReleaseUrl}</url>
         </repository>
         <snapshotRepository>
-          <id>snapshots</id>
-          <name>DataTorrent Snapshots</name>
-          <url>${distMgmtUrl}/${maven.deploy.repo.classifier}snapshots/</url>
+          <id>${distMgmtSnapshotsId}</id>
+          <url>${distMgmtSnapshotsUrl}</url>
         </snapshotRepository>
       </distributionManagement>
       <build>
@@ -381,43 +387,6 @@
               </execution>
             </executions>
           </plugin>
-          <plugin>
-            <artifactId>maven-site-plugin</artifactId>
-            <version>3.2</version>
-          </plugin>
-          <plugin>
-            <artifactId>maven-deploy-plugin</artifactId>
-            <version>2.7</version>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-javadoc-plugin</artifactId>
-            <executions>
-              <execution>
-                <!-- build javadoc jars per jar for publishing to maven -->
-                <id>module-javadocs</id>
-                <phase>prepare-package</phase>
-                <goals>
-                  <goal>jar</goal>
-                </goals>
-                <configuration>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-source-plugin</artifactId>
-            <version>2.2.1</version>
-            <executions>
-              <execution>
-                <id>attach-sources</id>
-                <goals>
-                  <goal>jar-no-fork</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
         </plugins>
       </build>
     </profile>


[05/50] [abbrv] incubator-apex-core git commit: APEX-149 #resolve Fixed the property name used to lookup RM webapp address in non-HA mode

Posted by th...@apache.org.
APEX-149 #resolve Fixed the property name used to lookup RM webapp address in non-HA mode


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/0a89c83c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/0a89c83c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/0a89c83c

Branch: refs/heads/master
Commit: 0a89c83c48c207cd282f264a8bf515621768eceb
Parents: 0a85586
Author: Pramod Immaneni <pr...@datatorrent.com>
Authored: Sat Sep 26 10:58:59 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sat Sep 26 11:02:17 2015 -0700

----------------------------------------------------------------------
 .../stram/security/StramWSFilter.java           | 47 +++++++++++++++-----
 .../security/StramWSFilterInitializer.java      | 19 ++++++--
 2 files changed, 50 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/0a89c83c/engine/src/main/java/com/datatorrent/stram/security/StramWSFilter.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/security/StramWSFilter.java b/engine/src/main/java/com/datatorrent/stram/security/StramWSFilter.java
index 8be7fed..061bdc7 100644
--- a/engine/src/main/java/com/datatorrent/stram/security/StramWSFilter.java
+++ b/engine/src/main/java/com/datatorrent/stram/security/StramWSFilter.java
@@ -37,6 +37,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
 
 import com.datatorrent.stram.webapp.WebServices;
@@ -102,6 +103,7 @@ public class StramWSFilter implements Filter
         proxyAddresses = new HashSet<String>();
         for (String proxyHost : proxyHosts) {
           try {
+            logger.debug("resolving proxy hostname {}", proxyHost);
             for (InetAddress add : InetAddress.getAllByName(proxyHost)) {
               logger.debug("proxy address is: {}", add.getHostAddress());
               proxyAddresses.add(add.getHostAddress());
@@ -133,9 +135,8 @@ public class StramWSFilter implements Filter
 
     HttpServletRequest httpReq = (HttpServletRequest)req;
     HttpServletResponse httpResp = (HttpServletResponse)resp;
-    logger.debug("Remote address for request is: {}", httpReq.getRemoteAddr());
+    String remoteAddr = httpReq.getRemoteAddr();
     String requestURI = httpReq.getRequestURI();
-    logger.debug("Request path {}", requestURI);
     boolean authenticate = true;
     String user = null;
     if(getProxyAddresses().contains(httpReq.getRemoteAddr())) {
@@ -149,9 +150,11 @@ public class StramWSFilter implements Filter
       }
       if (requestURI.equals(WebServices.PATH) && (user != null)) {
         String token = createClientToken(user, httpReq.getLocalAddr());
-        logger.debug("Create token {}", token);
+        logger.debug("{}: creating token {}", remoteAddr, token);
         Cookie cookie = new Cookie(CLIENT_COOKIE, token);
         httpResp.addCookie(cookie);
+      } else {
+        logger.info("{}: proxy access to URI {} by user {}, no cookie created", remoteAddr, requestURI, user);
       }
       authenticate = false;
     }
@@ -167,19 +170,24 @@ public class StramWSFilter implements Filter
       }
       boolean valid = false;
       if (cookie != null) {
-        logger.debug("Verifying token {}", cookie.getValue());
-        user = verifyClientToken(cookie.getValue());
-        valid = true;
-        logger.debug("Token valid");
+        user = verifyClientToken(cookie.getValue(), remoteAddr);
+        if (user != null) {
+          valid = true;
+        } else {
+          logger.debug("{}: invalid cookie {}", remoteAddr, cookie.getValue());
+        }
+      } else {
+        logger.debug("{}: cookie not found {}", remoteAddr, CLIENT_COOKIE);
       }
       if (!valid) {
+        logger.debug("{}: auth failure", remoteAddr);
         httpResp.sendError(HttpServletResponse.SC_UNAUTHORIZED);
         return;
       }
     }
 
     if(user == null) {
-      logger.debug("Could not find {} cookie, so user will not be set", WEBAPP_PROXY_USER);
+      logger.debug("{}: could not find user, so user principal will not be set", remoteAddr);
       chain.doFilter(req, resp);
     } else {
       final StramWSPrincipal principal = new StramWSPrincipal(user);
@@ -199,16 +207,31 @@ public class StramWSFilter implements Filter
     return token.encodeToUrlString();
   }
 
-  private String verifyClientToken(String tokenstr) throws IOException
+  private String verifyClientToken(String tokenstr, String cid) throws IOException
   {
     Token<StramDelegationTokenIdentifier> token = new Token<StramDelegationTokenIdentifier>();
-    token.decodeFromUrlString(tokenstr);
+    try {
+      token.decodeFromUrlString(tokenstr);
+    } catch (IOException e) {
+      logger.debug("{}: error decoding token: {}", cid, e.getMessage());
+      return null;
+    }
     byte[] identifier = token.getIdentifier();
     byte[] password = token.getPassword();
     StramDelegationTokenIdentifier tokenIdentifier = new StramDelegationTokenIdentifier();
     DataInputStream input = new DataInputStream(new ByteArrayInputStream(identifier));
-    tokenIdentifier.readFields(input);
-    tokenManager.verifyToken(tokenIdentifier, password);
+    try {
+      tokenIdentifier.readFields(input);
+    } catch (IOException e) {
+      logger.debug("{}: error decoding identifier: {}", cid, e.getMessage());
+      return null;
+    }
+    try {
+      tokenManager.verifyToken(tokenIdentifier, password);
+    } catch (SecretManager.InvalidToken e) {
+      logger.debug("{}: invalid token {}: {}", cid, tokenIdentifier, e.getMessage());
+      return null;
+    }
     return tokenIdentifier.getOwner().toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/0a89c83c/engine/src/main/java/com/datatorrent/stram/security/StramWSFilterInitializer.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/security/StramWSFilterInitializer.java b/engine/src/main/java/com/datatorrent/stram/security/StramWSFilterInitializer.java
index f4f8d22..a2b2821 100644
--- a/engine/src/main/java/com/datatorrent/stram/security/StramWSFilterInitializer.java
+++ b/engine/src/main/java/com/datatorrent/stram/security/StramWSFilterInitializer.java
@@ -33,6 +33,9 @@ import org.apache.hadoop.http.FilterInitializer;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import com.datatorrent.stram.util.ConfigUtils;
 
 /**
@@ -43,12 +46,15 @@ import com.datatorrent.stram.util.ConfigUtils;
  */
 public class StramWSFilterInitializer extends FilterInitializer
 {
+  private static final Logger logger = LoggerFactory.getLogger(StramWSFilterInitializer.class);
+
   private static final String FILTER_NAME = "AM_PROXY_FILTER";
   private static final String FILTER_CLASS = StramWSFilter.class.getCanonicalName();
 
   @Override
   public void initFilter(FilterContainer container, Configuration conf)
   {
+    logger.debug("Conf {}", conf);
     Map<String, String> params = new HashMap<String, String>();
     Collection<String> proxies = new ArrayList<String>();
     if (ConfigUtils.isRMHAEnabled(conf)) {
@@ -80,6 +86,8 @@ public class StramWSFilterInitializer extends FilterInitializer
   public String getProxyHostAndPort(Configuration conf)
   {
     String addr = conf.get(YarnConfiguration.PROXY_ADDRESS);
+    logger.info("proxy address setting {}", addr);
+    logger.debug("proxy setting sources {}", conf.getPropertySources(YarnConfiguration.PROXY_ADDRESS));
     if (addr == null || addr.isEmpty()) {
       addr = getResolvedRMWebAppURLWithoutScheme(conf, null);
     }
@@ -96,27 +104,29 @@ public class StramWSFilterInitializer extends FilterInitializer
     boolean sslEnabled = conf.getBoolean(
             CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_KEY,
             CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_DEFAULT);
-    return getResolvedRMWebAppURLWithoutScheme(conf, sslEnabled, (rmId != null) ? "." + rmId : null);
+    return getResolvedRMWebAppURLWithoutScheme(conf, sslEnabled, (rmId != null) ? "." + rmId : "");
   }
 
   /*
     From org.apache.hadoop.yarn.webapp.util.WebAppUtils
     Modified for HA support
   */
-  public String getResolvedRMWebAppURLWithoutScheme(Configuration conf, boolean sslEnabled, String rmId)
+  public String getResolvedRMWebAppURLWithoutScheme(Configuration conf, boolean sslEnabled, String rmPrpKey)
   {
     InetSocketAddress address = null;
     if (sslEnabled) {
       address =
-              conf.getSocketAddr(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS + rmId,
+              conf.getSocketAddr(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS + rmPrpKey,
                       YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_ADDRESS,
                       YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT);
     } else {
       address =
-              conf.getSocketAddr(YarnConfiguration.RM_WEBAPP_ADDRESS + rmId,
+              conf.getSocketAddr(YarnConfiguration.RM_WEBAPP_ADDRESS + rmPrpKey,
                       YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS,
                       YarnConfiguration.DEFAULT_RM_WEBAPP_PORT);
     }
+    logger.info("rm webapp address setting {}", address);
+    logger.debug("rm setting sources {}", conf.getPropertySources(YarnConfiguration.RM_WEBAPP_ADDRESS));
     address = NetUtils.getConnectAddress(address);
     StringBuffer sb = new StringBuffer();
     InetAddress resolved = address.getAddress();
@@ -133,6 +143,7 @@ public class StramWSFilterInitializer extends FilterInitializer
       sb.append(address.getHostName());
     }
     sb.append(":").append(address.getPort());
+    logger.info("rm webapp resolved hostname {}", sb.toString());
     return sb.toString();
   }
 


[18/50] [abbrv] incubator-apex-core git commit: Correct ownership related info in pom.xml

Posted by th...@apache.org.
Correct ownership related info in pom.xml


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/0aa70063
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/0aa70063
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/0aa70063

Branch: refs/heads/master
Commit: 0aa700634e0a1d487ee90f7545d4195d934ba841
Parents: 88ee4ad
Author: Thomas Weise <th...@datatorrent.com>
Authored: Sat Oct 3 17:52:48 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sat Oct 3 17:52:48 2015 -0700

----------------------------------------------------------------------
 pom.xml | 15 +++++++--------
 1 file changed, 7 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/0aa70063/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0086730..8f68dcb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -27,12 +27,12 @@
   <version>3.2.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
-  <name>Realtime Stream Processing Framework</name>
-  <url>http://www.datatorrent.com</url>
+  <name>Apache Apex Core</name>
+  <url>http://apex.apache.org</url>
 
   <organization>
-    <name>DataTorrent, Inc.</name>
-    <url>http://www.datatorrent.com/</url>
+    <name>Apache Software Foundation</name>
+    <url>http://www.apache.org</url>
   </organization>
 
   <licenses>
@@ -47,7 +47,7 @@
       <snapshots>
         <enabled>false</enabled>
       </snapshots>
-      <id>Datatorrent-Releases</id>
+      <id>datatorrent</id>
       <name>DataTorrent Release Repository</name>
       <url>https://www.datatorrent.com/maven/content/repositories/releases/</url>
     </repository>
@@ -55,7 +55,6 @@
 
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <netbeans.hint.license>datatorrent-inc</netbeans.hint.license>
     <maven.compiler.optimize>false</maven.compiler.optimize>
     <maven.build.timestamp.format>yyyyMMdd</maven.build.timestamp.format>
     <hadoop.version>2.2.0</hadoop.version>
@@ -96,8 +95,8 @@
         <configuration>
           <header>license.txt</header>
           <properties>
-            <owner>DataTorrent, Inc.</owner>
-            <email>apex-dev@googlegroups.com</email>
+            <owner>Apache Software Foundation</owner>
+            <email>dev@apex.incubator.apache.org</email>
           </properties>
           <excludes>
             <exclude>license.txt</exclude>


[41/50] [abbrv] incubator-apex-core git commit: APEX-218 Modify project names for NOTICE.

Posted by th...@apache.org.
APEX-218 Modify project names for NOTICE.


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/b47ab81b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/b47ab81b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/b47ab81b

Branch: refs/heads/master
Commit: b47ab81b0412dd358e1d10513fc96639db3ec29c
Parents: e3e8cf8
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Oct 22 20:35:32 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Oct 22 20:41:36 2015 -0700

----------------------------------------------------------------------
 apex-app-archetype/pom.xml  | 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                     | 4 ++--
 7 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b47ab81b/apex-app-archetype/pom.xml
----------------------------------------------------------------------
diff --git a/apex-app-archetype/pom.xml b/apex-app-archetype/pom.xml
index f88dabf..8138e2e 100644
--- a/apex-app-archetype/pom.xml
+++ b/apex-app-archetype/pom.xml
@@ -31,7 +31,7 @@
   <artifactId>apex-app-archetype</artifactId>
   <packaging>maven-archetype</packaging>
 
-  <name>Apache Apex Application Maven Archetype</name>
+  <name>Apache Apex (incubating) Application Maven Archetype</name>
 
   <build>
     <extensions>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b47ab81b/apex-conf-archetype/pom.xml
----------------------------------------------------------------------
diff --git a/apex-conf-archetype/pom.xml b/apex-conf-archetype/pom.xml
index 7caf93f..bd0d959 100644
--- a/apex-conf-archetype/pom.xml
+++ b/apex-conf-archetype/pom.xml
@@ -31,7 +31,7 @@
   <artifactId>apex-conf-archetype</artifactId>
   <packaging>maven-archetype</packaging>
 
-  <name>Apache Apex App Configuration Maven Archetype</name>
+  <name>Apache Apex (incubating) App Configuration Maven Archetype</name>
 
   <build>
     <extensions>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b47ab81b/api/pom.xml
----------------------------------------------------------------------
diff --git a/api/pom.xml b/api/pom.xml
index a44d8a6..f410f8d 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -30,7 +30,7 @@
   </parent>
 
   <artifactId>apex-api</artifactId>
-  <name>Apache Apex API</name>
+  <name>Apache Apex (incubating) API</name>
   <packaging>jar</packaging>
 
   <repositories>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b47ab81b/bufferserver/pom.xml
----------------------------------------------------------------------
diff --git a/bufferserver/pom.xml b/bufferserver/pom.xml
index 6b9a848..5bcddc1 100644
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@ -31,7 +31,7 @@
   <artifactId>apex-bufferserver</artifactId>
   <description>Interface for nodes from 2 different containers to talk to each other.</description>
 
-  <name>Apache Apex Buffer Server</name>
+  <name>Apache Apex (incubating) Buffer Server</name>
 
   <build>
     <plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b47ab81b/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 6ee0305..7dd504a 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -29,7 +29,7 @@
   </parent>
 
   <artifactId>apex-common</artifactId>
-  <name>Apache Apex Common Library</name>
+  <name>Apache Apex (incubating) Common Library</name>
   <packaging>jar</packaging>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b47ab81b/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 5d30a8e..874d332 100644
--- a/engine/pom.xml
+++ b/engine/pom.xml
@@ -31,7 +31,7 @@
   <artifactId>apex-engine</artifactId>
   <packaging>jar</packaging>
 
-  <name>Apache Apex Stream Processing Engine</name>
+  <name>Apache Apex (incubating) Stream Processing Engine</name>
 
   <build>
     <finalName>${project.artifactId}</finalName>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b47ab81b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1fdabcf..a5679d5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -33,8 +33,8 @@
   <version>3.2.0-incubating-SNAPSHOT</version>
   <packaging>pom</packaging>
 
-  <name>Apache Apex</name>
-  <url>http://apex.apache.org</url>
+  <name>Apache Apex (incubating)</name>
+  <url>http://apex.incubator.apache.org</url>
 
   <organization>
     <name>Apache Software Foundation</name>


[35/50] [abbrv] incubator-apex-core git commit: Fix EOL

Posted by th...@apache.org.
Fix EOL


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/b527b301
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/b527b301
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/b527b301

Branch: refs/heads/master
Commit: b527b301a29b05a747f6ce1b43f852c8a4d894f5
Parents: 979a0ef
Author: MalharJenkins <je...@datatorrent.com>
Authored: Tue Oct 20 19:54:58 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Tue Oct 20 19:54:58 2015 -0700

----------------------------------------------------------------------
 .../datatorrent/common/util/package-info.java   |  38 +++---
 .../com/datatorrent/stram/cli/package-info.java |  66 ++++-----
 .../datatorrent/stram/engine/package-info.java  |  94 ++++++-------
 .../com/datatorrent/stram/package-info.java     |  54 ++++----
 .../datatorrent/stram/stream/package-info.java  | 136 +++++++++----------
 .../datatorrent/stram/util/package-info.java    |  64 ++++-----
 .../datatorrent/stram/webapp/package-info.java  |  76 +++++------
 7 files changed, 264 insertions(+), 264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b527b301/common/src/main/java/com/datatorrent/common/util/package-info.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/com/datatorrent/common/util/package-info.java b/common/src/main/java/com/datatorrent/common/util/package-info.java
index 41134d0..5bfc38f 100644
--- a/common/src/main/java/com/datatorrent/common/util/package-info.java
+++ b/common/src/main/java/com/datatorrent/common/util/package-info.java
@@ -1,22 +1,22 @@
-/**
- * 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.
- */
+/**
+ * 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.
+ */
 /**
  * Shared utilities
  */
-package com.datatorrent.common.util;
+package com.datatorrent.common.util;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b527b301/engine/src/main/java/com/datatorrent/stram/cli/package-info.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/cli/package-info.java b/engine/src/main/java/com/datatorrent/stram/cli/package-info.java
index 116366b..05b45e2 100644
--- a/engine/src/main/java/com/datatorrent/stram/cli/package-info.java
+++ b/engine/src/main/java/com/datatorrent/stram/cli/package-info.java
@@ -1,33 +1,33 @@
-/**
- * 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.
- */
-/**
- *
- * <b>com.datatorrent.stram.cli</b> package deals with code for command line interface for the streaming platform<p>
- * <br>
- * The cli code wraps the webservices layer (<b>com.datatorrent.stream.webapp</b>) and thus accesses the dag
- * via one single point. All webservice calls for streaming data go through to the stram.<br>
- * <br>
- * The com.datatorrent.stram.cli package consists of<br>
- * <b>{@link com.datatorrent.stram.cli.DTCli}</b>: Provides command line interface for a streaming application on hadoop (yarn)<br>
- * <br>
- *
- */
-package com.datatorrent.stram.cli;
-
-
+/**
+ * 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.
+ */
+/**
+ *
+ * <b>com.datatorrent.stram.cli</b> package deals with code for command line interface for the streaming platform<p>
+ * <br>
+ * The cli code wraps the webservices layer (<b>com.datatorrent.stream.webapp</b>) and thus accesses the dag
+ * via one single point. All webservice calls for streaming data go through to the stram.<br>
+ * <br>
+ * The com.datatorrent.stram.cli package consists of<br>
+ * <b>{@link com.datatorrent.stram.cli.DTCli}</b>: Provides command line interface for a streaming application on hadoop (yarn)<br>
+ * <br>
+ *
+ */
+package com.datatorrent.stram.cli;
+
+

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b527b301/engine/src/main/java/com/datatorrent/stram/engine/package-info.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/engine/package-info.java b/engine/src/main/java/com/datatorrent/stram/engine/package-info.java
index e7c1d6e..fdad92f 100644
--- a/engine/src/main/java/com/datatorrent/stram/engine/package-info.java
+++ b/engine/src/main/java/com/datatorrent/stram/engine/package-info.java
@@ -1,47 +1,47 @@
-/**
- * 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.
- */
-/**
- * Execution layer for data operators, tuple processing, serialization, streams, window boundaries etc.<p>
- * <br>
- * <br>
- * The streaming platform would take care of the following for you<br>(TBD, add "How it will")<br>
- * - Standard partitioning (round robin, sticky key). You can add  your own partitioning.<br>
- * - End of window statistics collection in terms of number of tuples, bandwidth, I/O etc<br>
- * - Ensuring the the emitted tuple reaches the downstream operators<br>
- * - Queueing tuples and retaining them till all future downstream operators have consumed it<br>
- * <br>
- * <br>
- * There are pre-defined library operators that you can use: see ...<br>
- * Examples are in the following packages<br>
-    <ul>
-      <li><a href="../../../datatorrent/lib/algo/package-summary.html">Algorithmic</a>: A set of algorithmic operators</li>
-      <li><a href="../../../datatorrent/lib/io/package-summary.html">Input-Output</a>: A set of operators for Input-Output from Hadoop. Consists of adapters to various message buses</li>
-      <li><a href="../../../datatorrent/lib/logs/package-summary.html">Log Collection</a>: A set of operators for log collection</li>
-      <li><a href="../../../datatorrent/lib/math/package-summary.html">Arithmetic</a>: A set of arithmetic operators</li>
-      <li><a href="../../../datatorrent/lib/stream/package-summary.html">Stream</a>: A set of operators for stream operations</li>
-      <li><a href="../../../datatorrent/lib/testbench/package-summary.html">Testbench</a>: A set of operators for testing your dag, operators </li>
-      <li><a href="../../../datatorrent/lib/util/package-summary.html">Utilities</a>: A set of utility classes</li>
-    </ul>
- * <br>
- *
- */
-
-package com.datatorrent.stram.engine;
-
-
+/**
+ * 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.
+ */
+/**
+ * Execution layer for data operators, tuple processing, serialization, streams, window boundaries etc.<p>
+ * <br>
+ * <br>
+ * The streaming platform would take care of the following for you<br>(TBD, add "How it will")<br>
+ * - Standard partitioning (round robin, sticky key). You can add  your own partitioning.<br>
+ * - End of window statistics collection in terms of number of tuples, bandwidth, I/O etc<br>
+ * - Ensuring the the emitted tuple reaches the downstream operators<br>
+ * - Queueing tuples and retaining them till all future downstream operators have consumed it<br>
+ * <br>
+ * <br>
+ * There are pre-defined library operators that you can use: see ...<br>
+ * Examples are in the following packages<br>
+    <ul>
+      <li><a href="../../../datatorrent/lib/algo/package-summary.html">Algorithmic</a>: A set of algorithmic operators</li>
+      <li><a href="../../../datatorrent/lib/io/package-summary.html">Input-Output</a>: A set of operators for Input-Output from Hadoop. Consists of adapters to various message buses</li>
+      <li><a href="../../../datatorrent/lib/logs/package-summary.html">Log Collection</a>: A set of operators for log collection</li>
+      <li><a href="../../../datatorrent/lib/math/package-summary.html">Arithmetic</a>: A set of arithmetic operators</li>
+      <li><a href="../../../datatorrent/lib/stream/package-summary.html">Stream</a>: A set of operators for stream operations</li>
+      <li><a href="../../../datatorrent/lib/testbench/package-summary.html">Testbench</a>: A set of operators for testing your dag, operators </li>
+      <li><a href="../../../datatorrent/lib/util/package-summary.html">Utilities</a>: A set of utility classes</li>
+    </ul>
+ * <br>
+ *
+ */
+
+package com.datatorrent.stram.engine;
+
+

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b527b301/engine/src/main/java/com/datatorrent/stram/package-info.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/package-info.java b/engine/src/main/java/com/datatorrent/stram/package-info.java
index 3e23940..eb337d6 100644
--- a/engine/src/main/java/com/datatorrent/stram/package-info.java
+++ b/engine/src/main/java/com/datatorrent/stram/package-info.java
@@ -1,27 +1,27 @@
-/**
- * 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.
- */
-/**
- * <b>com.datatorrent.stram</b> package contains all code for streaming application master<p>
- * <br>
- * The application master is also called <b>STRAM</b><br>
- * (TBD - To explain all functionality)<br>
- *
- */
-
-package com.datatorrent.stram;
+/**
+ * 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.
+ */
+/**
+ * <b>com.datatorrent.stram</b> package contains all code for streaming application master<p>
+ * <br>
+ * The application master is also called <b>STRAM</b><br>
+ * (TBD - To explain all functionality)<br>
+ *
+ */
+
+package com.datatorrent.stram;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b527b301/engine/src/main/java/com/datatorrent/stram/stream/package-info.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/stream/package-info.java b/engine/src/main/java/com/datatorrent/stram/stream/package-info.java
index e83d345..58fd326 100644
--- a/engine/src/main/java/com/datatorrent/stram/stream/package-info.java
+++ b/engine/src/main/java/com/datatorrent/stram/stream/package-info.java
@@ -1,68 +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.
- */
-/**
- *
- * <b>com.datatorrent.stream</b> package contains all code related to various implementations of Stream interface<p>
- * <br>
- * A stream is a logical unit of a dag that defines the connection between
- * a node and list of listener operators. Stream has the following properties in Malhar's streaming platform<br>
- * - One writer node<br>
- * - Any number of listener operators<br>
- * - Context as defined by the properties specified in the dag<br>
- * A stream definition in the dag is a logical definition. Multiple logical listerner operators means that the emitted tuple
- * would reach each of them. Partitioning is done when a single
- * logical listener node partitions into multilpe physical operators. This may happen due to initial user
- * specification, or dynamic run time constraint enforcement. In such a scenerio the logical stream gets partitioned
- * into physical streams. Each physical stream would retain the
- * characteristics of the logical node (one writer, multiple readers, and context).<br>
- * <br>
- * The streams included in com.datatorrent.stream include<br>
- * <br>
- * <br><b>Buffer Server Streams</b><br>
- * <b>{@link com.datatorrent.stream.BufferServerInputStream}</b>: extends {@link com.datatorrent.stream.SocketInputStream},
- * takes data from buffer server into the node. Every logical stream will have at least two such
- * objects ({@link com.datatorrent.stream.BufferServerInputStream}
- *  and {@link com.datatorrent.stream.BufferServerOutputStream}). If the logical stream gets partitioned
- * into physical streams then each of these physical streams will have these objects. Inlined version of
- *  a logical stream does not go through the buffer server and hence would not have
- * {@link com.datatorrent.stream.BufferServerInputStream} and {@link com.datatorrent.stream.BufferServerOutputStream} objects<br>
- * <b>{@link com.datatorrent.stream.BufferServerOutputStream}</b>: extends {@link com.datatorrent.stream.SocketOutputStream}
- * and in conjunction with {@link com.datatorrent.stream.BufferServerInputStream} forms a complete stream
- * in a node->buffer server->node path<br>
- * <br><b>Inline Stream (Within a Hadoop Container)</b><br>
- * <b>{@link com.datatorrent.stream.InlineStream}</b>: Streams data between two operators in inline mode. This implementation of
- * {@link com.datatorrent.engine.Stream} and {{@link com.datatorrent.api.Sink}
- * interface does not have connection to BufferServer and cannot be persisted.<br>
- *
- * <b>{@link com.datatorrent.stream.MuxStream}</b>: <br>
- * <b>{@link com.datatorrent.stream.PartitionAwareSink}</b>: <br>
- *
- * <br><b>Socket Interface Streams</b><br>
- * <b>{@link com.datatorrent.stream.SocketInputStream}</b>: Implements {@link com.datatorrent.engine.Stream} interface and provides
- * basic stream connection for a node to read from a socket. Users can use this class if they want to directly connect to
- * a outside socket<br>
- * <b>{@link com.datatorrent.stream.SocketOutputStream}</b>: Implements {@link com.datatorrent.engine.Stream} interface and provides
- * basic stream connection for a node to write to a socket. Most likely users would not use it to write to a socket by themselves.
- *   Would be used in adapters and via {@link com.datatorrent.stream.BufferServerOutputStream}<br>
- * <br>
- *
- */
-
-package com.datatorrent.stram.stream;
-
+/**
+ * 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.
+ */
+/**
+ *
+ * <b>com.datatorrent.stream</b> package contains all code related to various implementations of Stream interface<p>
+ * <br>
+ * A stream is a logical unit of a dag that defines the connection between
+ * a node and list of listener operators. Stream has the following properties in Malhar's streaming platform<br>
+ * - One writer node<br>
+ * - Any number of listener operators<br>
+ * - Context as defined by the properties specified in the dag<br>
+ * A stream definition in the dag is a logical definition. Multiple logical listerner operators means that the emitted tuple
+ * would reach each of them. Partitioning is done when a single
+ * logical listener node partitions into multilpe physical operators. This may happen due to initial user
+ * specification, or dynamic run time constraint enforcement. In such a scenerio the logical stream gets partitioned
+ * into physical streams. Each physical stream would retain the
+ * characteristics of the logical node (one writer, multiple readers, and context).<br>
+ * <br>
+ * The streams included in com.datatorrent.stream include<br>
+ * <br>
+ * <br><b>Buffer Server Streams</b><br>
+ * <b>{@link com.datatorrent.stream.BufferServerInputStream}</b>: extends {@link com.datatorrent.stream.SocketInputStream},
+ * takes data from buffer server into the node. Every logical stream will have at least two such
+ * objects ({@link com.datatorrent.stream.BufferServerInputStream}
+ *  and {@link com.datatorrent.stream.BufferServerOutputStream}). If the logical stream gets partitioned
+ * into physical streams then each of these physical streams will have these objects. Inlined version of
+ *  a logical stream does not go through the buffer server and hence would not have
+ * {@link com.datatorrent.stream.BufferServerInputStream} and {@link com.datatorrent.stream.BufferServerOutputStream} objects<br>
+ * <b>{@link com.datatorrent.stream.BufferServerOutputStream}</b>: extends {@link com.datatorrent.stream.SocketOutputStream}
+ * and in conjunction with {@link com.datatorrent.stream.BufferServerInputStream} forms a complete stream
+ * in a node->buffer server->node path<br>
+ * <br><b>Inline Stream (Within a Hadoop Container)</b><br>
+ * <b>{@link com.datatorrent.stream.InlineStream}</b>: Streams data between two operators in inline mode. This implementation of
+ * {@link com.datatorrent.engine.Stream} and {{@link com.datatorrent.api.Sink}
+ * interface does not have connection to BufferServer and cannot be persisted.<br>
+ *
+ * <b>{@link com.datatorrent.stream.MuxStream}</b>: <br>
+ * <b>{@link com.datatorrent.stream.PartitionAwareSink}</b>: <br>
+ *
+ * <br><b>Socket Interface Streams</b><br>
+ * <b>{@link com.datatorrent.stream.SocketInputStream}</b>: Implements {@link com.datatorrent.engine.Stream} interface and provides
+ * basic stream connection for a node to read from a socket. Users can use this class if they want to directly connect to
+ * a outside socket<br>
+ * <b>{@link com.datatorrent.stream.SocketOutputStream}</b>: Implements {@link com.datatorrent.engine.Stream} interface and provides
+ * basic stream connection for a node to write to a socket. Most likely users would not use it to write to a socket by themselves.
+ *   Would be used in adapters and via {@link com.datatorrent.stream.BufferServerOutputStream}<br>
+ * <br>
+ *
+ */
+
+package com.datatorrent.stram.stream;
+

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b527b301/engine/src/main/java/com/datatorrent/stram/util/package-info.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/util/package-info.java b/engine/src/main/java/com/datatorrent/stram/util/package-info.java
index a5e1d50..200bf98 100644
--- a/engine/src/main/java/com/datatorrent/stram/util/package-info.java
+++ b/engine/src/main/java/com/datatorrent/stram/util/package-info.java
@@ -1,32 +1,32 @@
-/**
- * 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.
- */
-/**
- * 
- * <b>com.datatorrent.util</b> package consists of utilities classes that Malhar provides<p>
- * <br>
- * The current utilities include<br>
- * <b>{@link com.datatorrent.util.CircularBuffer}</b>: Takes a class T and provides a circular buffer<br>
- * <b>{@link com.datatorrent.util.ScheduledExecutorService}</b>: <br>
- * <b>{@link com.datatorrent.util.ScheduledThreadPoolExecutor}</b>: <br>
- * <b>{@link com.datatorrent.util.StablePriorityQueue}</b>: Implements a priority queue (Queue<E>) and is mainly used to queue tuples<br> 
- * <br>
- */
-
-package com.datatorrent.stram.util;
-
+/**
+ * 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.
+ */
+/**
+ * 
+ * <b>com.datatorrent.util</b> package consists of utilities classes that Malhar provides<p>
+ * <br>
+ * The current utilities include<br>
+ * <b>{@link com.datatorrent.util.CircularBuffer}</b>: Takes a class T and provides a circular buffer<br>
+ * <b>{@link com.datatorrent.util.ScheduledExecutorService}</b>: <br>
+ * <b>{@link com.datatorrent.util.ScheduledThreadPoolExecutor}</b>: <br>
+ * <b>{@link com.datatorrent.util.StablePriorityQueue}</b>: Implements a priority queue (Queue<E>) and is mainly used to queue tuples<br> 
+ * <br>
+ */
+
+package com.datatorrent.stram.util;
+

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b527b301/engine/src/main/java/com/datatorrent/stram/webapp/package-info.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/webapp/package-info.java b/engine/src/main/java/com/datatorrent/stram/webapp/package-info.java
index a3d18bf..947f262 100644
--- a/engine/src/main/java/com/datatorrent/stram/webapp/package-info.java
+++ b/engine/src/main/java/com/datatorrent/stram/webapp/package-info.java
@@ -1,38 +1,38 @@
-/**
- * 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.
- */
-/**
- *
- * <b>com.datatorrent.stram.webapp</b> package is the web service layer of Malhar's streaming platform<p>
- * <br>
- * The webservices layer interacts with the streaming application master (stram). The internals of Hadoop are taken care
- * off and an common set of functionality is provided for all streaming related runtime data.<br>
- * <br>
- * The webservices layer consists of the following data:<br>
- * <b>{@link com.datatorrent.stram.webapp.AppInfo}</b>: Provides application level data like user, appId, elapsed time, etc.<br>
- * <b>{@link com.datatorrent.stram.webapp.OperatorInfo}</b>: Provides data on the operator. This includes throughput, container id etc.<br>
- * <b>{@link com.datatorrent.stram.webapp.OperatorsInfo}</b>: Provides data on all the operators of the data.<br>
- * <b>{@link com.datatorrent.stram.webapp.StramWebApp}</b>: TBD<br>
- * <b>{@link com.datatorrent.stram.webapp.StramWebServices}</b>: TBD<br>
- * <b>Access and Authoriation</b>: TBD<br>
- * <br>
- *
- */
-
-package com.datatorrent.stram.webapp;
-
+/**
+ * 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.
+ */
+/**
+ *
+ * <b>com.datatorrent.stram.webapp</b> package is the web service layer of Malhar's streaming platform<p>
+ * <br>
+ * The webservices layer interacts with the streaming application master (stram). The internals of Hadoop are taken care
+ * off and an common set of functionality is provided for all streaming related runtime data.<br>
+ * <br>
+ * The webservices layer consists of the following data:<br>
+ * <b>{@link com.datatorrent.stram.webapp.AppInfo}</b>: Provides application level data like user, appId, elapsed time, etc.<br>
+ * <b>{@link com.datatorrent.stram.webapp.OperatorInfo}</b>: Provides data on the operator. This includes throughput, container id etc.<br>
+ * <b>{@link com.datatorrent.stram.webapp.OperatorsInfo}</b>: Provides data on all the operators of the data.<br>
+ * <b>{@link com.datatorrent.stram.webapp.StramWebApp}</b>: TBD<br>
+ * <b>{@link com.datatorrent.stram.webapp.StramWebServices}</b>: TBD<br>
+ * <b>Access and Authoriation</b>: TBD<br>
+ * <br>
+ *
+ */
+
+package com.datatorrent.stram.webapp;
+


[29/50] [abbrv] incubator-apex-core git commit: APEX-197 #resolve #comment making the types of files excluded consistent

Posted by th...@apache.org.
APEX-197 #resolve #comment making the types of files excluded consistent


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/8f3c9ea4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/8f3c9ea4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/8f3c9ea4

Branch: refs/heads/master
Commit: 8f3c9ea46beedea6992ea57a1e60ed94d4f82594
Parents: dc40027
Author: Chandni Singh <cs...@apache.org>
Authored: Wed Oct 14 16:48:11 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sat Oct 17 23:13:53 2015 -0700

----------------------------------------------------------------------
 pom.xml | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/8f3c9ea4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a7ccc90..6aad8c7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -109,11 +109,10 @@
             <exclude>DISCLAIMER</exclude>
             <exclude>LICENSE</exclude>
             <exclude>NOTICE</exclude>
-            <exclude>license.txt</exclude>
             <exclude>**/*.md</exclude>
             <exclude>**/*.txt</exclude>
+            <exclude>**/*.importorder</exclude>
             <exclude>**/archetype-resources/**</exclude>
-            <exclude>src/test/resources/projects/basic/goal.txt</exclude>
           </excludes>
           <mapping>
             <dtcli>SCRIPT_STYLE</dtcli>
@@ -187,11 +186,11 @@
             <exclude>.idea/**</exclude>
             <exclude>**/src/test/resources/**/MANIFEST.MF</exclude>
             <exclude>**/src/test/resources/**/*.json</exclude>
-            <exclude>**/src/test/resources/**/*.txt</exclude>
             <exclude>**/resources/META-INF/services/**</exclude>
-            <exclude>CHANGELOG.md</exclude>
             <exclude>**/archetype-resources/**</exclude>
-            <exclude>README.md</exclude>
+            <exclude>**/*.md</exclude>
+            <exclude>**/*.txt</exclude>
+            <exclude>**/*.importorder</exclude>
           </excludes>
         </configuration>
       </plugin>


[19/50] [abbrv] incubator-apex-core git commit: Using YarnConfiguration getSocketAddr to get the individual rm addresses in HA mode as it will lookup all the necessary configuration to return the address namely yarn.resourcemanager.address followed by ya

Posted by th...@apache.org.
Using YarnConfiguration getSocketAddr to get the individual rm addresses in HA mode as it will lookup all the necessary configuration to return the address namely yarn.resourcemanager.address followed by yarn.resourcemanager.hostname


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/45e891c7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/45e891c7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/45e891c7

Branch: refs/heads/master
Commit: 45e891c7cd7f83a34a4eefc0de10306ba139d2da
Parents: bf72215
Author: Pramod Immaneni <pr...@datatorrent.com>
Authored: Fri Sep 25 01:39:48 2015 -0700
Committer: Pramod Immaneni <pr...@datatorrent.com>
Committed: Sun Oct 4 09:54:50 2015 -0700

----------------------------------------------------------------------
 .../stram/client/StramClientUtils.java          | 28 +++++++++++++-------
 .../com/datatorrent/stram/util/ConfigUtils.java |  1 +
 2 files changed, 20 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/45e891c7/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 d596a73..45d2feb 100644
--- a/engine/src/main/java/com/datatorrent/stram/client/StramClientUtils.java
+++ b/engine/src/main/java/com/datatorrent/stram/client/StramClientUtils.java
@@ -22,10 +22,6 @@ import java.util.*;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
-
 import org.mozilla.javascript.Scriptable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,7 +34,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -58,6 +53,10 @@ import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.log4j.DTLoggerFactory;
 
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
 import com.datatorrent.api.StreamingApplication;
 
 import com.datatorrent.stram.StramClient;
@@ -246,10 +245,7 @@ public class StramClientUtils
       for (String rmId : ConfigUtils.getRMHAIds(conf)) {
         LOG.info("Yarn Resource Manager id: {}", rmId);
         // Set RM_ID to get the corresponding RM_ADDRESS
-        services.add(SecurityUtil.buildTokenService(NetUtils.createSocketAddr(
-                conf.get(RM_HOSTNAME_PREFIX + rmId),
-                YarnConfiguration.DEFAULT_RM_PORT,
-                RM_HOSTNAME_PREFIX + rmId)).toString());
+        services.add(SecurityUtil.buildTokenService(getRMHAAddress(rmId)).toString());
       }
       Text rmTokenService = new Text(Joiner.on(',').join(services));
 
@@ -284,6 +280,20 @@ public class StramClientUtils
       credentials.addToken(token.getService(), token);
     }
 
+    public InetSocketAddress getRMHAAddress(String rmId)
+    {
+      YarnConfiguration yarnConf;
+      if (conf instanceof YarnConfiguration) {
+        yarnConf = (YarnConfiguration)conf;
+      } else {
+        yarnConf = new YarnConfiguration(conf);
+      }
+      yarnConf.set(ConfigUtils.RM_HA_ID, rmId);
+      InetSocketAddress socketAddr = yarnConf.getSocketAddr(YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS, YarnConfiguration.DEFAULT_RM_PORT);
+      yarnConf.unset(ConfigUtils.RM_HA_ID);
+      return socketAddr;
+    }
+
   }
 
   private static final Logger LOG = LoggerFactory.getLogger(StramClientUtils.class);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/45e891c7/engine/src/main/java/com/datatorrent/stram/util/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/util/ConfigUtils.java b/engine/src/main/java/com/datatorrent/stram/util/ConfigUtils.java
index 481815f..68fe27b 100644
--- a/engine/src/main/java/com/datatorrent/stram/util/ConfigUtils.java
+++ b/engine/src/main/java/com/datatorrent/stram/util/ConfigUtils.java
@@ -39,6 +39,7 @@ public class ConfigUtils
   private static final String RM_HA_PREFIX = YarnConfiguration.RM_PREFIX + "ha.";
   public static final String RM_HA_ENABLED = RM_HA_PREFIX + "enabled";
   public static final String RM_HA_IDS = RM_HA_PREFIX + "rm-ids";
+  public static final String RM_HA_ID = RM_HA_PREFIX + "id";
   public static final boolean DEFAULT_RM_HA_ENABLED = false;
 
   private static String yarnLogDir;


[07/50] [abbrv] incubator-apex-core git commit: APEX-159 #resolve Fix application path in mini cluster test.

Posted by th...@apache.org.
APEX-159 #resolve Fix application path in mini cluster 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/fb53705d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/fb53705d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/fb53705d

Branch: refs/heads/master
Commit: fb53705d928f18087675a1bfb8b45c9580129682
Parents: 0a89c83
Author: Thomas Weise <th...@datatorrent.com>
Authored: Sun Sep 27 16:08:00 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sun Sep 27 16:08:00 2015 -0700

----------------------------------------------------------------------
 .../java/com/datatorrent/stram/StramMiniClusterTest.java | 11 ++---------
 1 file changed, 2 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/fb53705d/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java b/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java
index 493156b..f0fd325 100644
--- a/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java
@@ -51,7 +51,6 @@ import com.sun.jersey.api.client.WebResource;
 import com.datatorrent.api.*;
 import com.datatorrent.api.Context.OperatorContext;
 
-import com.datatorrent.common.util.AsyncFSStorageAgent;
 import com.datatorrent.stram.client.StramClientUtils;
 import com.datatorrent.stram.client.StramClientUtils.YarnClientHelper;
 import com.datatorrent.stram.engine.GenericTestOperator;
@@ -206,9 +205,6 @@ public class StramMiniClusterTest
     LogicalPlanConfiguration tb = new LogicalPlanConfiguration(conf);
     tb.addFromProperties(dagProps, null);
     LogicalPlan dag = createDAG(tb);
-    AsyncFSStorageAgent agent = new AsyncFSStorageAgent(testMeta.dir, null);
-    agent.setSyncCheckpoint(true);
-    dag.setAttribute(OperatorContext.STORAGE_AGENT, agent);
     Configuration yarnConf = new Configuration(yarnCluster.getConfig());
     StramClient client = new StramClient(yarnConf, dag);
     try {
@@ -231,7 +227,7 @@ public class StramMiniClusterTest
   private LogicalPlan createDAG(LogicalPlanConfiguration lpc) throws Exception
   {
     LogicalPlan dag = new LogicalPlan();
-    dag.setAttribute(LogicalPlan.APPLICATION_PATH, "file:" + System.getProperty("user.dir") + "/" + testMeta.dir);
+    dag.setAttribute(LogicalPlan.APPLICATION_PATH, new File(testMeta.dir).toURI().toString());
     lpc.prepareDAG(dag,null,"testApp");
     dag.validate();
     Assert.assertEquals("", Integer.valueOf(128), dag.getValue(DAG.MASTER_MEMORY_MB));
@@ -364,10 +360,7 @@ public class StramMiniClusterTest
   {
 
     LogicalPlan dag = new LogicalPlan();
-    dag.setAttribute(LogicalPlan.APPLICATION_PATH, testMeta.dir);
-    AsyncFSStorageAgent agent = new AsyncFSStorageAgent(testMeta.dir, null);
-    agent.setSyncCheckpoint(true);
-    dag.setAttribute(OperatorContext.STORAGE_AGENT, agent);
+    dag.setAttribute(LogicalPlan.APPLICATION_PATH, new File(testMeta.dir).toURI().toString());
     FailingOperator badOperator = dag.addOperator("badOperator", FailingOperator.class);
     dag.getContextAttributes(badOperator).put(OperatorContext.RECOVERY_ATTEMPTS, 1);
 


[09/50] [abbrv] incubator-apex-core git commit: Merge branch 'APEX-158' of github.com:vrozov/incubator-apex-core into vrozov-APEX-158

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


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/4adac06d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/4adac06d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/4adac06d

Branch: refs/heads/master
Commit: 4adac06d3d79e3731a86a958194fefea9cf69a9f
Parents: 8e49cfb 9990e15
Author: Chetan Narsude <ch...@datatorrent.com>
Authored: Mon Sep 28 11:17:26 2015 -0700
Committer: Chetan Narsude <ch...@datatorrent.com>
Committed: Mon Sep 28 11:17:26 2015 -0700

----------------------------------------------------------------------
 bufferserver/pom.xml | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------



[27/50] [abbrv] incubator-apex-core git commit: Fix JavaDoc in AsyncFSStorageAgent.

Posted by th...@apache.org.
Fix JavaDoc in AsyncFSStorageAgent.


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/83724f6c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/83724f6c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/83724f6c

Branch: refs/heads/master
Commit: 83724f6c382d8721f6a9e7b03855de3f75e73db2
Parents: 51ce8f9
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Oct 15 11:02:56 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Oct 15 11:02:56 2015 -0700

----------------------------------------------------------------------
 .../com/datatorrent/common/util/AsyncFSStorageAgent.java  | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/83724f6c/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 0e23c2e..83bbdca 100644
--- a/common/src/main/java/com/datatorrent/common/util/AsyncFSStorageAgent.java
+++ b/common/src/main/java/com/datatorrent/common/util/AsyncFSStorageAgent.java
@@ -16,10 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/**
- * @since 3.1.0
- */
-
 package com.datatorrent.common.util;
 
 import java.io.*;
@@ -33,6 +29,12 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.datatorrent.netlet.util.DTThrowable;
+
+/**
+ * <p>AsyncFSStorageAgent class.</p>
+ *
+ * @since 3.1.0
+ */
 public class AsyncFSStorageAgent extends FSStorageAgent
 {
   private final transient Configuration conf;


[17/50] [abbrv] incubator-apex-core git commit: Fix license header related info in README.md, remove obsolete files.

Posted by th...@apache.org.
Fix license header related info in README.md, remove obsolete files.


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/ec03f12c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/ec03f12c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/ec03f12c

Branch: refs/heads/master
Commit: ec03f12cd0f1b9994a878237ca649d4c4a9c0423
Parents: 88ee4ad
Author: MalharJenkins <je...@datatorrent.com>
Authored: Sat Oct 3 15:49:20 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sat Oct 3 15:52:22 2015 -0700

----------------------------------------------------------------------
 README.md                                       |   23 +-
 engine/src/test/resources/README                |    9 -
 ...san_ RSA Public Key Cryptography in Java.pdf | 1932 ------------------
 3 files changed, 5 insertions(+), 1959 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ec03f12c/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index d48608b..baabcd2 100644
--- a/README.md
+++ b/README.md
@@ -23,24 +23,11 @@ You acknowledge that your submissions to this repository are made pursuant the t
   * Create a GitHub pull request from your repository, providing as many details about your changes as possible
   * After review and acceptance one of the committers will merge the pull request.
 
-When adding **new files**, please include the following Apache v2.0 license header at the top of the file, with the fields enclosed by brackets "[]" replaced with your own identifying information **(don't include the brackets!)**:
-```java
-/**
- * Copyright (C) [XXXX] [NAME OF COPYRIGHT OWNER]
- *
- * Licensed 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.
- */
-```
+When adding **new files**, please include the Apache v2.0 license header. From the top level directory:
+
+Run `mvn license:check -Dlicense.skip=false` to check correct header formatting.
+Run `mvn license:format -Dlicense.skip=false` to automatically add the header when missing.
+
 Thanks for contributing!
  
 ##Building Apex

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ec03f12c/engine/src/test/resources/README
----------------------------------------------------------------------
diff --git a/engine/src/test/resources/README b/engine/src/test/resources/README
deleted file mode 100644
index ef675c1..0000000
--- a/engine/src/test/resources/README
+++ /dev/null
@@ -1,9 +0,0 @@
-PKI in java is apparently straight forward. Please read the codeartisan*.pdf file in the same directory on
-complete approach. However I found that web page towards the end of my investigations. I followed the
-following steps for creating rsa.pub and rsa.prv files.
-
-$ ssh-keyget -t rsa -f ./id_rsa # just press enter when asked to enter passphrase and then repeat it again.
-$ openssl rsa -in id_rsa -pubout -outform DER -out rsa.pub # this gets you public key in X509 encoded format
-$ openssl pkcs8 -topk8 -inform PEM -outform DER -in id_rsa  -nocrypt > rsa.prv # this gets you private key in pkcs8 encoded format
-
-Original files and resultant files are stored in this directory.
\ No newline at end of file


[11/50] [abbrv] incubator-apex-core git commit: APEX-157 #comment #resolve Added changes for attribute serializable check in dag.validate

Posted by th...@apache.org.
APEX-157 #comment #resolve Added changes for attribute serializable check in dag.validate


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/b799bd2d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/b799bd2d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/b799bd2d

Branch: refs/heads/master
Commit: b799bd2d3edd7af3bdf4631129601e68ce349bf4
Parents: 4adac06
Author: ishark <is...@datatorrent.com>
Authored: Thu Sep 24 17:05:13 2015 -0700
Committer: ishark <is...@datatorrent.com>
Committed: Mon Sep 28 16:43:08 2015 -0700

----------------------------------------------------------------------
 .../stram/plan/logical/LogicalPlan.java         | 23 ++++++
 .../logical/LogicalPlanConfigurationTest.java   |  6 ++
 .../stram/plan/logical/LogicalPlanTest.java     | 83 +++++++++++++++++++-
 engine/src/test/resources/testTopology.json     |  2 +-
 4 files changed, 112 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b799bd2d/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 f068884..6405644 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
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 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.Operator.InputPort;
 import com.datatorrent.api.Operator.OutputPort;
@@ -1312,6 +1313,8 @@ public class LogicalPlan implements Serializable, DAG
             Validation.buildDefaultValidatorFactory();
     Validator validator = factory.getValidator();
 
+    checkAttributeValueSerializable(this.getAttributes(), DAG.class.getName());
+
     // clear oioRoot values in all operators
     for (OperatorMeta n: operators.values()) {
       n.oioRoot = null;
@@ -1336,6 +1339,8 @@ public class LogicalPlan implements Serializable, DAG
 
       OperatorMeta.PortMapping portMapping = n.getPortMapping();
 
+      checkAttributeValueSerializable(n.getAttributes(), n.getName());
+
       // Check operator annotation
       if (n.operatorAnnotation != null) {
         // Check if partition property of the operator is being honored
@@ -1368,6 +1373,7 @@ public class LogicalPlan implements Serializable, DAG
 
       // check that non-optional ports are connected
       for (InputPortMeta pm: portMapping.inPortMap.values()) {
+        checkAttributeValueSerializable(pm.getAttributes(), n.getName() + "." + pm.getPortName());
         StreamMeta sm = n.inputStreams.get(pm);
         if (sm == null) {
           if ((pm.portAnnotation == null || !pm.portAnnotation.optional()) && pm.classDeclaringHiddenPort == null) {
@@ -1397,6 +1403,7 @@ public class LogicalPlan implements Serializable, DAG
 
       boolean allPortsOptional = true;
       for (OutputPortMeta pm: portMapping.outPortMap.values()) {
+        checkAttributeValueSerializable(pm.getAttributes(), n.getName() + "." + pm.getPortName());
         if (!n.outputStreams.containsKey(pm)) {
           if ((pm.portAnnotation != null && !pm.portAnnotation.optional()) && pm.classDeclaringHiddenPort == null) {
             throw new ValidationException("Output port connection required: " + n.name + "." + pm.getPortName());
@@ -1458,6 +1465,22 @@ public class LogicalPlan implements Serializable, DAG
 
   }
 
+  private void checkAttributeValueSerializable(AttributeMap attributes, String context)
+  {
+    StringBuilder sb = new StringBuilder();
+    String delim = "";
+    // Check all attributes got operator are serializable
+    for (Entry<Attribute<?>, Object> entry : attributes.entrySet()) {
+      if (entry.getValue() != null && !(entry.getValue() instanceof Serializable)) {
+        sb.append(delim).append(entry.getKey().getSimpleName());
+        delim = ", ";
+      }
+    }
+    if (sb.length() > 0) {
+      throw new ValidationException("Attribute value(s) for " + sb.toString() + " in " + context + " are not serializable");
+    }
+  }
+
   /*
    * Validates OIO constraints for operators with more than one input streams
    * For a node to be OIO,

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b799bd2d/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanConfigurationTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanConfigurationTest.java b/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanConfigurationTest.java
index c4ad724..1d95afe 100644
--- a/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanConfigurationTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/plan/logical/LogicalPlanConfigurationTest.java
@@ -20,6 +20,7 @@ package com.datatorrent.stram.plan.logical;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.Serializable;
 import java.io.StringWriter;
 import java.lang.reflect.Field;
 
@@ -84,6 +85,11 @@ public class LogicalPlanConfigurationTest {
       return n;
   }
 
+  public static class TestStreamCodec<T> extends JsonStreamCodec<T> implements Serializable
+  {
+    private static final long serialVersionUID = 1L;
+  }
+
   /**
    * Test read from dt-site.xml in Hadoop configuration format.
    */

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b799bd2d/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 52c5f7d..a4ac488 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,9 +19,11 @@
 package com.datatorrent.stram.plan.logical;
 
 import com.datatorrent.common.util.BaseOperator;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.Serializable;
+import java.lang.reflect.Field;
 import java.util.*;
 
 import javax.validation.*;
@@ -41,6 +43,7 @@ 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;
@@ -683,7 +686,85 @@ public class LogicalPlanTest {
     Assert.assertNotNull("port object null", o1Clone.inport1);
   }
 
-  private static class TestStreamCodec implements StreamCodec<Object> {
+  @Test
+  public void testAttributeValuesSerializableCheck() throws NoSuchFieldException, SecurityException, IllegalArgumentException, IllegalAccessException
+  {
+    LogicalPlan dag = new LogicalPlan();
+    Attribute<Object> attr = new Attribute<Object>(new TestAttributeValue(), new Object2String());
+    Field nameField = Attribute.class.getDeclaredField("name");
+    nameField.setAccessible(true);
+    nameField.set(attr, "Test_Attribute");
+    nameField.setAccessible(false);
+
+    assertNotNull(attr);
+    // Dag attribute not serializable test
+    dag.setAttribute(attr, new TestAttributeValue());
+    try {
+      dag.validate();
+      Assert.fail("Setting not serializable attribute should throw exception");
+    } catch (ValidationException e) {
+      assertEquals("Validation Exception should match ", "Attribute value(s) for Test_Attribute in com.datatorrent.api.DAG are not serializable", e.getMessage());
+    }
+
+    // Operator attribute not serializable test
+    dag = new LogicalPlan();
+    TestGeneratorInputOperator operator = dag.addOperator("TestOperator", TestGeneratorInputOperator.class);
+    dag.setAttribute(operator, attr, new TestAttributeValue());
+    try {
+      dag.validate();
+      Assert.fail("Setting not serializable attribute should throw exception");
+    } catch (ValidationException e) {
+      assertEquals("Validation Exception should match ", "Attribute value(s) for Test_Attribute in TestOperator are not serializable", e.getMessage());
+    }
+
+    // Output Port attribute not serializable test
+    dag = new LogicalPlan();
+    operator = dag.addOperator("TestOperator", TestGeneratorInputOperator.class);
+    dag.setOutputPortAttribute(operator.outport, attr, new TestAttributeValue());
+    try {
+      dag.validate();
+      Assert.fail("Setting not serializable attribute should throw exception");
+    } catch (ValidationException e) {
+      assertEquals("Validation Exception should match ", "Attribute value(s) for Test_Attribute in TestOperator.outport are not serializable", e.getMessage());
+    }
+
+    // Input Port attribute not serializable test
+    dag = new LogicalPlan();
+    GenericTestOperator operator1 = dag.addOperator("TestOperator", GenericTestOperator.class);
+    dag.setInputPortAttribute(operator1.inport1, attr, new TestAttributeValue());
+    try {
+      dag.validate();
+      Assert.fail("Setting non serializable attribute should throw exception");
+    } catch (ValidationException e) {
+      assertEquals("Validation Exception should match ", "Attribute value(s) for Test_Attribute in TestOperator.inport1 are not serializable", e.getMessage());
+    }
+  }
+
+  private static class Object2String implements StringCodec<Object>
+  {
+
+    @Override
+    public Object fromString(String string)
+    {
+      // Stub method for testing - do nothing
+      return null;
+    }
+
+    @Override
+    public String toString(Object pojo)
+    {
+      // Stub method for testing - do nothing
+      return null;
+    }
+
+  }
+
+  private static class TestAttributeValue
+  {
+  }
+
+  private static class TestStreamCodec implements StreamCodec<Object>
+  {
     @Override
     public Object fromByteArray(Slice fragment)
     {

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/b799bd2d/engine/src/test/resources/testTopology.json
----------------------------------------------------------------------
diff --git a/engine/src/test/resources/testTopology.json b/engine/src/test/resources/testTopology.json
index 62c5262..45e1f0e 100644
--- a/engine/src/test/resources/testTopology.json
+++ b/engine/src/test/resources/testTopology.json
@@ -25,7 +25,7 @@
           "attributes": {
             "UNIFIER_LIMIT": 8,
             "STREAM_CODEC" : {
-              "com.datatorrent.common.codec.JsonStreamCodec" : {}
+              "com.datatorrent.stram.plan.logical.LogicalPlanConfigurationTest$TestStreamCodec" : {}
             }
           }
         }


[30/50] [abbrv] incubator-apex-core git commit: SPOI-5053 APEX-56 #resolve #comment Fixing removal of terminated operators from physical plan when downanStream operators are also completed till shutdown window Id Also fixed containers to be removed only

Posted by th...@apache.org.
SPOI-5053 APEX-56 #resolve #comment
Fixing removal of terminated operators from physical plan when downanStream operators are also completed till shutdown window Id
Also fixed containers to be removed only when operators are removed from physical 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/d2bf3e56
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/d2bf3e56
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/d2bf3e56

Branch: refs/heads/master
Commit: d2bf3e566e38ce518ad2e28fb260d0710c175251
Parents: 8f3c9ea
Author: ishark <is...@datatorrent.com>
Authored: Mon Oct 12 16:05:32 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sat Oct 17 23:37:35 2015 -0700

----------------------------------------------------------------------
 .../stram/StreamingContainerManager.java        | 35 +++++++++++---------
 .../stram/StreamingContainerManagerTest.java    | 11 ++----
 2 files changed, 22 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d2bf3e56/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 ed366db..3931fad 100644
--- a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
+++ b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
@@ -1040,10 +1040,7 @@ public class StreamingContainerManager implements PlanContext
         Iterator<Map.Entry<Long, Set<PTOperator>>> it = shutdownOperators.entrySet().iterator();
         while (it.hasNext()) {
           Map.Entry<Long, Set<PTOperator>> windowAndOpers = it.next();
-          if (windowAndOpers.getKey().longValue() > this.committedWindowId) {
-            // wait until window is committed
-            continue;
-          } else {
+          if (windowAndOpers.getKey().longValue() <= this.committedWindowId || checkDownStreamOperators(windowAndOpers)) {
             LOG.info("Removing inactive operators at window {} {}", Codec.getStringWindowId(windowAndOpers.getKey()), windowAndOpers.getValue());
             for (PTOperator oper : windowAndOpers.getValue()) {
               plan.removeTerminatedPartition(oper);
@@ -1070,8 +1067,7 @@ public class StreamingContainerManager implements PlanContext
       try {
         command.run();
         count++;
-      }
-      catch (Exception e) {
+      } catch (Exception e) {
         // TODO: handle error
         LOG.error("Failed to execute {}", command, e);
       }
@@ -1081,8 +1077,7 @@ public class StreamingContainerManager implements PlanContext
     if (count > 0) {
       try {
         checkpoint();
-      }
-      catch (Exception e) {
+      } catch (Exception e) {
         throw new RuntimeException("Failed to checkpoint state.", e);
       }
     }
@@ -1090,6 +1085,19 @@ public class StreamingContainerManager implements PlanContext
     return count;
   }
 
+  private boolean checkDownStreamOperators(Map.Entry<Long, Set<PTOperator>> windowAndOpers)
+  {
+    // Check if all downStream operators are at higher window Ids, then operator can be removed from dag
+    Set<PTOperator> downStreamOperators = getPhysicalPlan().getDependents(windowAndOpers.getValue());
+    for (PTOperator oper : downStreamOperators) {
+      long windowId = oper.stats.currentWindowId.get();
+      if (windowId < windowAndOpers.getKey().longValue()) {
+        return false;
+      }
+    }
+    return true;
+  }
+
   /**
    * Schedule container restart. Called by Stram after a container was terminated
    * and requires recovery (killed externally, or after heartbeat timeout). <br>
@@ -1495,8 +1503,6 @@ public class StreamingContainerManager implements PlanContext
     }
     Set<Integer> reportedOperators = Sets.newHashSetWithExpectedSize(sca.container.getOperators().size());
 
-    boolean containerIdle = true;
-
     for (OperatorHeartbeat shb : heartbeat.getContainerStats().operators) {
 
       long maxEndWindowTimestamp = 0;
@@ -1533,9 +1539,7 @@ public class StreamingContainerManager implements PlanContext
 
       oper.stats.lastHeartbeat = shb;
       List<ContainerStats.OperatorStats> statsList = shb.getOperatorStatsContainer();
-      if (!oper.stats.isIdle()) {
-        containerIdle = false;
-      }
+
       if (!statsList.isEmpty()) {
         long tuplesProcessed = 0;
         long tuplesEmitted = 0;
@@ -1743,11 +1747,10 @@ public class StreamingContainerManager implements PlanContext
 
     ContainerHeartbeatResponse rsp = getHeartbeatResponse(sca);
 
-    if (containerIdle && isApplicationIdle()) {
+    if (heartbeat.getContainerStats().operators.isEmpty() && isApplicationIdle()) {
       LOG.info("requesting idle shutdown for container {}", heartbeat.getContainerId());
       rsp.shutdown = true;
-    }
-    else {
+    } else {
       if (sca.shutdownRequested) {
         LOG.info("requesting shutdown for container {}", heartbeat.getContainerId());
         rsp.shutdown = true;

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d2bf3e56/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 710440d..2884323 100644
--- a/engine/src/test/java/com/datatorrent/stram/StreamingContainerManagerTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/StreamingContainerManagerTest.java
@@ -762,18 +762,13 @@ public class StreamingContainerManagerTest
     mc1.sendHeartbeat();
     scm.monitorHeartbeat();
 
+    Assert.assertEquals("committedWindowId", 1, scm.getCommittedWindowId());
+
     o2p1mos.currentWindowId(2).checkpointWindowId(2);
     mc2.sendHeartbeat();
     scm.monitorHeartbeat();
-    Assert.assertEquals("committedWindowId", 1, scm.getCommittedWindowId());
-    scm.monitorHeartbeat(); // committedWindowId updated in next cycle
-    Assert.assertEquals("committedWindowId", 2, scm.getCommittedWindowId());
-    Assert.assertEquals(1, o1p1.getContainer().getOperators().size());
-    Assert.assertEquals(1, o2p1.getContainer().getOperators().size());
-    Assert.assertEquals(2, physicalPlan.getContainers().size());
 
-    // call again as events are processed after committed window was updated
-    scm.processEvents();
+    // Operators are shutdown when both operators reach window Id 2
     Assert.assertEquals(0, o1p1.getContainer().getOperators().size());
     Assert.assertEquals(0, o2p1.getContainer().getOperators().size());
     Assert.assertEquals(0, physicalPlan.getContainers().size());


[14/50] [abbrv] incubator-apex-core git commit: Merge branch 'APEX-174' into devel-3

Posted by th...@apache.org.
Merge branch 'APEX-174' 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/9193ec81
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/9193ec81
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/9193ec81

Branch: refs/heads/master
Commit: 9193ec81ac2e41c06bc71e73643eca2bcc458bc8
Parents: cf97d0b dec2003
Author: Timothy Farkas <ti...@datatorrent.com>
Authored: Wed Sep 30 15:12:56 2015 -0700
Committer: Timothy Farkas <ti...@datatorrent.com>
Committed: Wed Sep 30 15:12:56 2015 -0700

----------------------------------------------------------------------
 apex_checks.xml                                            | 5 ++---
 bufferserver/pom.xml                                       | 2 +-
 common/pom.xml                                             | 2 +-
 engine/pom.xml                                             | 2 +-
 engine/src/main/java/org/apache/log4j/DTLoggerFactory.java | 4 ++--
 5 files changed, 7 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[47/50] [abbrv] incubator-apex-core git commit: APEX-219 Skip creation of DEPENDENCIES for the source archive.

Posted by th...@apache.org.
APEX-219 Skip creation of DEPENDENCIES for the source archive.


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/04c31c80
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/04c31c80
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/04c31c80

Branch: refs/heads/master
Commit: 04c31c80303992365a4fdf64cf7877f549bd2aa8
Parents: 9e985d4
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Oct 22 23:02:31 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Fri Oct 23 10:57:20 2015 -0700

----------------------------------------------------------------------
 pom.xml | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/04c31c80/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5be2636..a23782b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -393,6 +393,15 @@
         </plugins>
       </build>
     </profile>
-
+    <profile>
+      <activation>
+        <file>
+          <missing>${basedir}/src/main</missing>
+        </file>
+      </activation>
+      <properties>
+        <remoteresources.skip>true</remoteresources.skip>
+      </properties>
+    </profile>
   </profiles>
 </project>


[46/50] [abbrv] incubator-apex-core git commit: Add (incubating) to NOTICE

Posted by th...@apache.org.
Add (incubating) to NOTICE


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/9e985d49
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/9e985d49
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/9e985d49

Branch: refs/heads/master
Commit: 9e985d4928f5108c0903184ae61e532439a337f4
Parents: 8ef3f6e
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Oct 22 22:22:27 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Oct 22 22:22:27 2015 -0700

----------------------------------------------------------------------
 NOTICE | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


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


[12/50] [abbrv] incubator-apex-core git commit: Merge branch 'APEX-157' into devel-3

Posted by th...@apache.org.
Merge branch 'APEX-157' 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/cf97d0b7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/cf97d0b7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/cf97d0b7

Branch: refs/heads/master
Commit: cf97d0b7119edd1feea9ad5c9aac23aa9bf6732b
Parents: 4adac06 b799bd2
Author: Timothy Farkas <ti...@datatorrent.com>
Authored: Tue Sep 29 13:01:35 2015 -0700
Committer: Timothy Farkas <ti...@datatorrent.com>
Committed: Tue Sep 29 13:01:35 2015 -0700

----------------------------------------------------------------------
 .../stram/plan/logical/LogicalPlan.java         | 23 ++++++
 .../logical/LogicalPlanConfigurationTest.java   |  6 ++
 .../stram/plan/logical/LogicalPlanTest.java     | 83 +++++++++++++++++++-
 engine/src/test/resources/testTopology.json     |  2 +-
 4 files changed, 112 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[02/50] [abbrv] incubator-apex-core git commit: Merge branch 'APEX-142' of github.com:vrozov/incubator-apex-core into vrozov-APEX-142

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


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/e6263b5c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/e6263b5c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/e6263b5c

Branch: refs/heads/master
Commit: e6263b5cf0c79d8506dc2a396cce441dd23f84e1
Parents: 90bda5e 44d1bfc
Author: Chetan Narsude <ch...@datatorrent.com>
Authored: Fri Sep 25 16:45:46 2015 -0700
Committer: Chetan Narsude <ch...@datatorrent.com>
Committed: Fri Sep 25 16:58:15 2015 -0700

----------------------------------------------------------------------
 bufferserver/pom.xml                            | 64 +++++++-------------
 .../bufferserver/internal/DataList.java         |  6 +-
 .../bufferserver/client/SubscriberTest.java     |  6 +-
 .../bufferserver/server/ServerTest.java         | 12 ++--
 pom.xml                                         | 32 +++++-----
 5 files changed, 54 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/e6263b5c/bufferserver/pom.xml
----------------------------------------------------------------------
diff --cc bufferserver/pom.xml
index 2341550,c6f9eea..1c5027f
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@@ -33,6 -13,21 +33,28 @@@
  
    <name>Buffer Server</name>
  
+   <build>
+     <plugins>
+       <plugin>
+         <artifactId>maven-surefire-plugin</artifactId>
+         <dependencies>
+           <dependency>
+             <groupId>org.apache.maven.surefire</groupId>
+             <artifactId>surefire-testng</artifactId>
+             <version>2.14.1</version>
+           </dependency>
+         </dependencies>
+       </plugin>
++      <plugin>
++        <groupId>org.apache.maven.plugins</groupId>
++        <artifactId>maven-checkstyle-plugin</artifactId>
++        <configuration>
++          <maxAllowedViolations>123</maxAllowedViolations>
++        </configuration>
++      </plugin>
+     </plugins>
+   </build>
+ 
    <dependencies>
      <dependency>
        <groupId>org.testng</groupId>
@@@ -52,41 -42,27 +69,4 @@@
        <type>jar</type>
      </dependency>
    </dependencies>
- 
-   <build>
-     <plugins>
-       <plugin>
-         <groupId>org.apache.maven.plugins</groupId>
-         <artifactId>maven-checkstyle-plugin</artifactId>
-         <configuration>
-           <maxAllowedViolations>123</maxAllowedViolations>
-         </configuration>
-       </plugin>
-     </plugins>
-   </build>
--
--  <profiles>
--    <profile>
--      <id>netbeans-private-testng</id>
--      <activation>
--        <property>
--          <name>netbeans.testng.action</name>
--        </property>
--      </activation>
--      <build>
--        <plugins>
--          <plugin>
-             <groupId>org.apache.maven.plugins</groupId>
--            <artifactId>maven-surefire-plugin</artifactId>
-             <version>2.11</version>
--            <configuration>
--              <suiteXmlFiles>
--                <suiteXmlFile>target/nb-private/testng-suite.xml</suiteXmlFile>
--              </suiteXmlFiles>
--            </configuration>
--          </plugin>
--        </plugins>
--      </build>
--    </profile>
--  </profiles>
  </project>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/e6263b5c/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataList.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/e6263b5c/bufferserver/src/test/java/com/datatorrent/bufferserver/client/SubscriberTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/e6263b5c/bufferserver/src/test/java/com/datatorrent/bufferserver/server/ServerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/e6263b5c/pom.xml
----------------------------------------------------------------------


[48/50] [abbrv] incubator-apex-core git commit: Preparing to release 3.2.0-incubating-RC1

Posted by th...@apache.org.
Preparing to release 3.2.0-incubating-RC1


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/696965c2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/696965c2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/696965c2

Branch: refs/heads/master
Commit: 696965c2c85c48167f9a533e852fd1eda7c2bab9
Parents: 04c31c8
Author: Thomas Weise <th...@datatorrent.com>
Authored: Wed Oct 21 22:23:55 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Fri Oct 23 15:32:38 2015 -0700

----------------------------------------------------------------------
 CHANGELOG.md                                        | 16 +++++++++-------
 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 +-
 .../datatorrent/stram/client/AppPackageTest.java    |  2 +-
 .../test/resources/testAppPackage/mydtapp/pom.xml   |  2 +-
 pom.xml                                             |  2 +-
 12 files changed, 20 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 7670a49..693e643 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -2,19 +2,20 @@ Apex Changelog
 ========================================================================================================================
 
 
-Version 3.2.0
+Version 3.2.0-incubating - 2015-10-21
 ------------------------------------------------------------------------------------------------------------------------
 
 ### Bug
-* [APEX-88] - Stray directories getting created under working directory when running tests using StramLocalCluster
-* [APEX-89] - Javascript error when launching an app under CDH 5.2 (hadoop 2.5.0)
+* [APEX-56] - Controlled plan modification on operator shutdown
+* [APEX-88] - Stray directories under working directory when running tests using StramLocalCluster
+* [APEX-89] - Javascript error when launching app on CDH 5.2 (hadoop 2.5.0)
 * [APEX-93] - Persist operators need a re-deploy after a sink being persisted is dynamically partitioned
 * [APEX-96] - AsyncFSStorageAgent loses synccheckpoint flag value during serialization/deserialization
 * [APEX-97] - syncCheckpoint property on AsyncFSStorageAgent not working
 * [APEX-98] - WindowGenerator.getWindowMillis loses precisions
-* [APEX-100] - StreamingContainerManagerTest.testAppDataPush uses hardcoded 12345 port
+* [APEX-100] - StreamingContainerManagerTest.testAppDataPush uses hardcoded port
 * [APEX-101] - Negative Memory reported for Application Master
-* [APEX-102] - AppDataPush Agent Not Adding timeBuckets and dimensionsAggregators Correctly To The Schema
+* [APEX-102] - AppDataPushAgent Not Adding timeBuckets and dimensionsAggregators
 * [APEX-111] - dtcli: show-logical-plan with app package does not list the applications in the app package
 * [APEX-112] - Property change on logical operator converts from null to "null"(string)
 * [APEX-113] - Application Master not setting correct temp location
@@ -25,9 +26,10 @@ Version 3.2.0
 * [APEX-121] - Making sure that state is transferred to client for Statefull Codec
 * [APEX-126] - handleIdleTime Called Outside Of The Space Between beginWindow and endWindow
 * [APEX-149] - In secure mode non-HA setup STRAM web service calls are failing
-* [APEX-156] - checkstyle CustomImportOrder inconsistent behavior
 * [APEX-159] - StramMiniClusterTest.testOperatorFailureRecovery succeeds with unexpected error condition
-* [APEX-173] - StringCodec registered with BeanUtils is throwing null pointer
+* [APEX-184] - When There Are 8 Or More Subscribers Buffer Server Can Become Blocked
+* [APEX-198] - Unit tests created remnant directories outside of target directory
+* [APEX-212] - Null pointer exception after all physical operators are removed.
 
 ### Task
 * [APEX-16] - Configure Checkstyle plugin

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/apex-app-archetype/README.md
----------------------------------------------------------------------
diff --git a/apex-app-archetype/README.md b/apex-app-archetype/README.md
index 3ca588f..f1b62fe 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.2.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.2.0-incubating -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/696965c2/apex-app-archetype/pom.xml
----------------------------------------------------------------------
diff --git a/apex-app-archetype/pom.xml b/apex-app-archetype/pom.xml
index 8138e2e..fbe81ff 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.2.0-incubating-SNAPSHOT</version>
+    <version>3.2.0-incubating</version>
   </parent>
 
   <artifactId>apex-app-archetype</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/apex-conf-archetype/README.md
----------------------------------------------------------------------
diff --git a/apex-conf-archetype/README.md b/apex-conf-archetype/README.md
index 2bcd18a..85b79a8 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.2.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.2.0-incubating -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/696965c2/apex-conf-archetype/pom.xml
----------------------------------------------------------------------
diff --git a/apex-conf-archetype/pom.xml b/apex-conf-archetype/pom.xml
index bd0d959..547794d 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.2.0-incubating-SNAPSHOT</version>
+    <version>3.2.0-incubating</version>
   </parent>
 
   <artifactId>apex-conf-archetype</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/api/pom.xml
----------------------------------------------------------------------
diff --git a/api/pom.xml b/api/pom.xml
index f410f8d..8541a17 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.2.0-incubating-SNAPSHOT</version>
+    <version>3.2.0-incubating</version>
   </parent>
 
   <artifactId>apex-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/bufferserver/pom.xml
----------------------------------------------------------------------
diff --git a/bufferserver/pom.xml b/bufferserver/pom.xml
index 5bcddc1..4d6d7fa 100644
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.2.0-incubating-SNAPSHOT</version>
+    <version>3.2.0-incubating</version>
   </parent>
 
   <artifactId>apex-bufferserver</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 7dd504a..3dedae9 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.2.0-incubating-SNAPSHOT</version>
+    <version>3.2.0-incubating</version>
   </parent>
 
   <artifactId>apex-common</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 874d332..d4225b5 100644
--- a/engine/pom.xml
+++ b/engine/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.apex</groupId>
     <artifactId>apex</artifactId>
-    <version>3.2.0-incubating-SNAPSHOT</version>
+    <version>3.2.0-incubating</version>
   </parent>
 
   <artifactId>apex-engine</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/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 8fb4b1b..0d284cf 100644
--- a/engine/src/test/java/com/datatorrent/stram/client/AppPackageTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/client/AppPackageTest.java
@@ -84,7 +84,7 @@ public class AppPackageTest
   {
     Assert.assertEquals("mydtapp", json.getString("appPackageName"));
     Assert.assertEquals("1.0-SNAPSHOT", json.getString("appPackageVersion"));
-    Assert.assertEquals("3.2.0-incubating-SNAPSHOT", json.getString("dtEngineVersion"));
+    Assert.assertEquals("3.2.0-incubating", json.getString("dtEngineVersion"));
     Assert.assertEquals("lib/*.jar", json.getJSONArray("classPath").getString(0));
 
     JSONObject application = json.getJSONArray("applications").getJSONObject(0);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/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 3b1e799..c09f085 100644
--- a/engine/src/test/resources/testAppPackage/mydtapp/pom.xml
+++ b/engine/src/test/resources/testAppPackage/mydtapp/pom.xml
@@ -33,7 +33,7 @@
 
   <properties>
     <!-- change this if you desire to use a different version of DataTorrent -->
-    <datatorrent.version>3.2.0-incubating-SNAPSHOT</datatorrent.version>
+    <datatorrent.version>3.2.0-incubating</datatorrent.version>
     <datatorrent.apppackage.classpath>lib/*.jar</datatorrent.apppackage.classpath>
   </properties>
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/696965c2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a23782b..73ee47d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -30,7 +30,7 @@
 
   <groupId>org.apache.apex</groupId>
   <artifactId>apex</artifactId>
-  <version>3.2.0-incubating-SNAPSHOT</version>
+  <version>3.2.0-incubating</version>
   <packaging>pom</packaging>
 
   <name>Apache Apex (incubating)</name>


[49/50] [abbrv] incubator-apex-core git commit: Preparing to release 3.2.0-incubating-RC2

Posted by th...@apache.org.
Preparing to release 3.2.0-incubating-RC2


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/d61ca617
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/d61ca617
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/d61ca617

Branch: refs/heads/master
Commit: d61ca617fd44bf9d74800838341f92018f2c7d10
Parents: 696965c
Author: Thomas Weise <th...@datatorrent.com>
Authored: Fri Oct 23 15:37:46 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Fri Oct 23 15:37:46 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d61ca617/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 693e643..2d23dd7 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -2,7 +2,7 @@ Apex Changelog
 ========================================================================================================================
 
 
-Version 3.2.0-incubating - 2015-10-21
+Version 3.2.0-incubating - 2015-10-23
 ------------------------------------------------------------------------------------------------------------------------
 
 ### Bug


[42/50] [abbrv] incubator-apex-core git commit: Produce .zip and .tar.gz source release packages.

Posted by th...@apache.org.
Produce .zip and .tar.gz source release packages.


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/93ca5616
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/93ca5616
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/93ca5616

Branch: refs/heads/master
Commit: 93ca5616fc68ee7911492e39212926eecad5035e
Parents: b47ab81
Author: Thomas Weise <th...@datatorrent.com>
Authored: Wed Oct 21 21:32:59 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Oct 22 20:54:55 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/93ca5616/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a5679d5..ee77ae6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -358,6 +358,7 @@
         <distMgmtReleaseId>apache.staging.https</distMgmtReleaseId>
         <distMgmtReleaseUrl>https://repository.apache.org/service/local/staging/deploy/maven2</distMgmtReleaseUrl>
         <distMgmtDevUrl>file://${project.build.directory}/mvn-repo</distMgmtDevUrl>
+        <sourceReleaseAssemblyDescriptor>source-release-zip-tar</sourceReleaseAssemblyDescriptor>
         <package.prefix>/opt/datatorrent</package.prefix>
         <package.groupname>dtorrent</package.groupname>
       </properties>


[06/50] [abbrv] incubator-apex-core git commit: APEX-159 #resolve StramMiniClusterTest.testOperatorFailureRecovery succeeds with unexpected error condition

Posted by th...@apache.org.
APEX-159 #resolve StramMiniClusterTest.testOperatorFailureRecovery succeeds with unexpected error condition


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/0fc22a09
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/0fc22a09
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/0fc22a09

Branch: refs/heads/master
Commit: 0fc22a094d5ae67e249f8dbc728c5e6a32eecfbc
Parents: 0a85586
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Sat Sep 26 19:59:02 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Sat Sep 26 19:59:02 2015 -0700

----------------------------------------------------------------------
 .../stram/LaunchContainerRunnable.java          | 27 +++++++-------------
 .../java/com/datatorrent/stram/StramClient.java | 10 +-------
 .../datatorrent/stram/StramMiniClusterTest.java |  4 +--
 3 files changed, 12 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/0fc22a09/engine/src/main/java/com/datatorrent/stram/LaunchContainerRunnable.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/com/datatorrent/stram/LaunchContainerRunnable.java b/engine/src/main/java/com/datatorrent/stram/LaunchContainerRunnable.java
index 0a6c062..863808a 100644
--- a/engine/src/main/java/com/datatorrent/stram/LaunchContainerRunnable.java
+++ b/engine/src/main/java/com/datatorrent/stram/LaunchContainerRunnable.java
@@ -118,28 +118,19 @@ public class LaunchContainerRunnable implements Runnable
     LOG.info("CLASSPATH: {}", classPathEnv);
   }
 
+  public static void addFileToLocalResources(final String name, final FileStatus fileStatus, final LocalResourceType type, final Map<String, LocalResource> localResources)
+  {
+    final LocalResource localResource = LocalResource.newInstance(ConverterUtils.getYarnUrlFromPath(fileStatus.getPath()),
+            type, LocalResourceVisibility.APPLICATION, fileStatus.getLen(), fileStatus.getModificationTime());
+    localResources.put(name, localResource);
+  }
+
   public static void addFilesToLocalResources(LocalResourceType type, String commaSeparatedFileNames, Map<String, LocalResource> localResources, FileSystem fs) throws IOException
   {
     String[] files = StringUtils.splitByWholeSeparator(commaSeparatedFileNames, StramClient.LIB_JARS_SEP);
     for (String file : files) {
-      Path dst = new Path(file);
-      // Create a local resource to point to the destination jar path
-      FileStatus destStatus = fs.getFileStatus(dst);
-      LocalResource amJarRsrc = Records.newRecord(LocalResource.class);
-      // Set the type of resource - file or archive
-      amJarRsrc.setType(type);
-      // Set visibility of the resource
-      // Setting to most private option
-      amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION);
-      // Set the resource to be copied over
-      amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(dst));
-      // Set timestamp and length of file so that the framework
-      // can do basic sanity checks for the local resource
-      // after it has been copied over to ensure it is the same
-      // resource the client intended to use with the application
-      amJarRsrc.setTimestamp(destStatus.getModificationTime());
-      amJarRsrc.setSize(destStatus.getLen());
-      localResources.put(dst.getName(), amJarRsrc);
+      final Path dst = new Path(file);
+      addFileToLocalResources(dst.getName(), fs.getFileStatus(dst), type, localResources);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/0fc22a09/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 7edc628..7abfc82 100644
--- a/engine/src/main/java/com/datatorrent/stram/StramClient.java
+++ b/engine/src/main/java/com/datatorrent/stram/StramClient.java
@@ -497,15 +497,7 @@ public class StramClient
       outStream = fs.create(launchConfigDst, true);
       conf.writeXml(outStream);
       outStream.close();
-
-      FileStatus topologyFileStatus = fs.getFileStatus(cfgDst);
-      LocalResource topologyRsrc = Records.newRecord(LocalResource.class);
-      topologyRsrc.setType(LocalResourceType.FILE);
-      topologyRsrc.setVisibility(LocalResourceVisibility.APPLICATION);
-      topologyRsrc.setResource(ConverterUtils.getYarnUrlFromURI(cfgDst.toUri()));
-      topologyRsrc.setTimestamp(topologyFileStatus.getModificationTime());
-      topologyRsrc.setSize(topologyFileStatus.getLen());
-      localResources.put(LogicalPlan.SER_FILE_NAME, topologyRsrc);
+      LaunchContainerRunnable.addFileToLocalResources(LogicalPlan.SER_FILE_NAME, fs.getFileStatus(cfgDst), LocalResourceType.FILE, localResources);
 
       // Set local resource info into app master container launch context
       amContainer.setLocalResources(localResources);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/0fc22a09/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java b/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java
index 493156b..5d37f76 100644
--- a/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/StramMiniClusterTest.java
@@ -364,8 +364,8 @@ public class StramMiniClusterTest
   {
 
     LogicalPlan dag = new LogicalPlan();
-    dag.setAttribute(LogicalPlan.APPLICATION_PATH, testMeta.dir);
-    AsyncFSStorageAgent agent = new AsyncFSStorageAgent(testMeta.dir, null);
+    dag.setAttribute(LogicalPlan.APPLICATION_PATH, "file:" + System.getProperty("user.dir") + '/' + testMeta.dir);
+    AsyncFSStorageAgent agent = new AsyncFSStorageAgent(dag.getAttributes().get(LogicalPlan.APPLICATION_PATH), null);
     agent.setSyncCheckpoint(true);
     dag.setAttribute(OperatorContext.STORAGE_AGENT, agent);
     FailingOperator badOperator = dag.addOperator("badOperator", FailingOperator.class);


[43/50] [abbrv] incubator-apex-core git commit: Remove stray Copyright line.

Posted by th...@apache.org.
Remove stray Copyright line.


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/d9b2af07
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/d9b2af07
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/d9b2af07

Branch: refs/heads/master
Commit: d9b2af077b69bb07ac567856a232ea4bce3f21e5
Parents: 93ca561
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Oct 22 21:14:05 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Oct 22 21:14:05 2015 -0700

----------------------------------------------------------------------
 .../resources/archetype-resources/XmlJavadocCommentsExtractor.xsl  | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/d9b2af07/apex-app-archetype/src/main/resources/archetype-resources/XmlJavadocCommentsExtractor.xsl
----------------------------------------------------------------------
diff --git a/apex-app-archetype/src/main/resources/archetype-resources/XmlJavadocCommentsExtractor.xsl b/apex-app-archetype/src/main/resources/archetype-resources/XmlJavadocCommentsExtractor.xsl
index 975ef1f..08075a9 100644
--- a/apex-app-archetype/src/main/resources/archetype-resources/XmlJavadocCommentsExtractor.xsl
+++ b/apex-app-archetype/src/main/resources/archetype-resources/XmlJavadocCommentsExtractor.xsl
@@ -1,8 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <!--
 
-    Copyright (C) 2015 DataTorrent, Inc.
-
     Licensed 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


[22/50] [abbrv] incubator-apex-core git commit: Merge branch 'readme-cleanup' of https://github.com/tweise/incubator-apex-core into devel-3

Posted by th...@apache.org.
Merge branch 'readme-cleanup' 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/8104e25b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/8104e25b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/8104e25b

Branch: refs/heads/master
Commit: 8104e25b6e0111d839038ac0c30e25593f2cb013
Parents: 5324f10 ec03f12
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Sun Oct 4 17:08:53 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Sun Oct 4 17:08:53 2015 -0700

----------------------------------------------------------------------
 README.md                                       |   23 +-
 engine/src/test/resources/README                |    9 -
 ...san_ RSA Public Key Cryptography in Java.pdf | 1932 ------------------
 3 files changed, 5 insertions(+), 1959 deletions(-)
----------------------------------------------------------------------



[36/50] [abbrv] incubator-apex-core git commit: Merge branch 'release-3.2-EOL' of https://github.com/vrozov/incubator-apex-core into release-3.2

Posted by th...@apache.org.
Merge branch 'release-3.2-EOL' of https://github.com/vrozov/incubator-apex-core into release-3.2


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/badcf8ad
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/badcf8ad
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/badcf8ad

Branch: refs/heads/master
Commit: badcf8adc7a63a1e0f9188d6ca410116b3fd0a8c
Parents: 42927bc b527b30
Author: MalharJenkins <je...@datatorrent.com>
Authored: Tue Oct 20 20:23:55 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Tue Oct 20 20:25:39 2015 -0700

----------------------------------------------------------------------
 .../datatorrent/common/util/package-info.java   |  38 +++---
 .../com/datatorrent/stram/cli/package-info.java |  66 ++++-----
 .../datatorrent/stram/engine/package-info.java  |  94 ++++++-------
 .../com/datatorrent/stram/package-info.java     |  54 ++++----
 .../datatorrent/stram/stream/package-info.java  | 136 +++++++++----------
 .../datatorrent/stram/util/package-info.java    |  64 ++++-----
 .../datatorrent/stram/webapp/package-info.java  |  76 +++++------
 7 files changed, 264 insertions(+), 264 deletions(-)
----------------------------------------------------------------------



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

Posted by th...@apache.org.
Merge branch 'APEX-92' of https://github.com/ishark/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/0a85586d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/0a85586d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/0a85586d

Branch: refs/heads/master
Commit: 0a85586dfd194a0b77cb438e68a54c16e99d2cf7
Parents: e6263b5 d8e1e74
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Fri Sep 25 18:08:10 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Fri Sep 25 18:08:10 2015 -0700

----------------------------------------------------------------------
 engine/pom.xml                                  |  2 +-
 .../stram/StreamingAppMasterService.java        | 83 ++++++++++++++------
 2 files changed, 62 insertions(+), 23 deletions(-)
----------------------------------------------------------------------



[34/50] [abbrv] incubator-apex-core git commit: APEX-184 #resolve 1. In autoFlushExecutor don't exit run() until there is at least one listener that has more data to send. 2. Do not enable read in resumeReadIfSuspended when not able to switch to a new bu

Posted by th...@apache.org.
APEX-184 #resolve
1. In autoFlushExecutor don't exit run() until there is at least one listener that has more data to send.
2. Do not enable read in resumeReadIfSuspended when not able to switch to a new buffer.
3. Fix possible race condition in Block acquire.
4. Fix for incorrect counting of in memory block permits.
5. Fix check style violations.


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/4c6d3f5b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/4c6d3f5b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/4c6d3f5b

Branch: refs/heads/master
Commit: 4c6d3f5b34a163c74c8ff80763a57a62a7a297c2
Parents: 979a0ef
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Fri Oct 9 17:35:30 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Tue Oct 20 19:43:02 2015 -0700

----------------------------------------------------------------------
 bufferserver/pom.xml                            |   2 +-
 .../bufferserver/internal/DataList.java         | 248 ++++++++++++-------
 .../bufferserver/internal/DataListener.java     |   2 +-
 .../bufferserver/internal/FastDataList.java     |  11 +-
 .../bufferserver/internal/LogicalNode.java      |  37 ++-
 .../datatorrent/bufferserver/server/Server.java | 108 ++++----
 6 files changed, 232 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4c6d3f5b/bufferserver/pom.xml
----------------------------------------------------------------------
diff --git a/bufferserver/pom.xml b/bufferserver/pom.xml
index 1346ba7..6b9a848 100644
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@ -51,7 +51,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>124</maxAllowedViolations>
+          <maxAllowedViolations>60</maxAllowedViolations>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4c6d3f5b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataList.java
----------------------------------------------------------------------
diff --git a/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataList.java b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataList.java
index f5af2e5..1f6c273 100644
--- a/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataList.java
+++ b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataList.java
@@ -19,8 +19,13 @@
 package com.datatorrent.bufferserver.internal;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -40,7 +45,6 @@ import com.datatorrent.bufferserver.util.VarInt;
 import com.datatorrent.netlet.AbstractClient;
 import com.datatorrent.netlet.util.VarInt.MutableInt;
 
-import static com.google.common.collect.Lists.newArrayList;
 import static com.google.common.collect.Maps.newHashMap;
 import static com.google.common.collect.Sets.newHashSet;
 
@@ -66,9 +70,10 @@ public class DataList
   protected int size;
   protected int processingOffset;
   protected long baseSeconds;
-  private final List<AbstractClient> suspendedClients = newArrayList();
+  private final Set<AbstractClient> suspendedClients = newHashSet();
   private final AtomicInteger numberOfInMemBlockPermits;
   private MutableInt nextOffset = new MutableInt();
+  private Future<?> future;
 
   public DataList(final String identifier, final int blockSize, final int numberOfCacheBlocks)
   {
@@ -106,13 +111,20 @@ public class DataList
       for (Block temp = first; temp != null; temp = temp.next) {
         if (temp.starting_window >= longWindowId || temp.ending_window > longWindowId) {
           if (temp != last) {
+            last.refCount.decrementAndGet();
             last = temp;
             do {
               temp = temp.next;
               temp.discard(false);
-              if (temp.data != null) {
-                temp.data = null;
-                numberOfInMemBlockRewound++;
+              synchronized (temp) {
+                if (temp.refCount.get() != 0) {
+                  logger.debug("Discarded block {} has positive reference count. Listeners: {}", temp, all_listeners);
+                  throw new IllegalStateException("Discarded block " + temp + " has positive reference count!");
+                }
+                if (temp.data != null) {
+                  temp.data = null;
+                  numberOfInMemBlockRewound++;
+                }
               }
             } while (temp.next != null);
             last.next = null;
@@ -148,8 +160,13 @@ public class DataList
         Block temp = first;
         while (temp != last) {
           temp.discard(false);
-          temp.data = null;
-          temp = temp.next;
+          synchronized (temp) {
+            if (temp.refCount.get() != 0) {
+              throw new IllegalStateException("Discarded block " + temp + " not zero reference count!");
+            }
+            temp.data = null;
+            temp = temp.next;
+          }
         }
       }
       first = last;
@@ -173,9 +190,15 @@ public class DataList
           break;
         }
         temp.discard(false);
-        if (temp.data != null) {
-          temp.data = null;
-          numberOfInMemBlockPurged++;
+        synchronized (temp) {
+          if (temp.refCount.get() != 0) {
+            logger.debug("Discarded block {} has positive reference count. Listeners: {}", temp, all_listeners);
+            throw new IllegalStateException("Discarded block " + temp + " has positive reference count!");
+          }
+          if (temp.data != null) {
+            temp.data = null;
+            numberOfInMemBlockPurged++;
+          }
         }
       }
     }
@@ -202,21 +225,15 @@ public class DataList
     do {
       while (size == 0) {
         size = VarInt.read(last.data, processingOffset, writeOffset, nextOffset);
-        switch (nextOffset.integer) {
-          case -5:
-            throw new RuntimeException("problemo!");
-
-          case -4:
-          case -3:
-          case -2:
-          case -1:
-          case 0:
-            if (writeOffset == last.data.length) {
-              nextOffset.integer = 0;
-              processingOffset = 0;
-              size = 0;
-            }
-            break flush;
+        if (nextOffset.integer > -5 && nextOffset.integer < 1) {
+          if (writeOffset == last.data.length) {
+            nextOffset.integer = 0;
+            processingOffset = 0;
+            size = 0;
+          }
+          break flush;
+        } else if (nextOffset.integer == -5) {
+          throw new RuntimeException("problemo!");
         }
       }
 
@@ -240,6 +257,9 @@ public class DataList
             Tuple rwt = Tuple.getTuple(last.data, processingOffset, size);
             baseSeconds = (long)rwt.getBaseSeconds() << 32;
             break;
+
+          default:
+            break;
         }
         processingOffset += size;
         size = 0;
@@ -255,17 +275,28 @@ public class DataList
 
     last.writingOffset = writeOffset;
 
-    autoFlushExecutor.submit(new Runnable()
-    {
-      @Override
-      public void run()
+    notifyListeners();
+
+  }
+
+  public void notifyListeners()
+  {
+    if (future == null || future.isDone() || future.isCancelled()) {
+      future = autoFlushExecutor.submit(new Runnable()
       {
-        for (DataListener dl : all_listeners) {
-          dl.addedData();
+        @Override
+        public void run()
+        {
+          boolean atLeastOneListenerHasDataToSend;
+          do {
+            atLeastOneListenerHasDataToSend = false;
+            for (DataListener dl : all_listeners) {
+              atLeastOneListenerHasDataToSend |= dl.addedData();
+            }
+          } while (atLeastOneListenerHasDataToSend);
         }
-      }
-
-    });
+      });
+    }
   }
 
   public void setAutoFlushExecutor(final ExecutorService es)
@@ -381,7 +412,7 @@ public class DataList
   public boolean suspendRead(final AbstractClient client)
   {
     synchronized (suspendedClients) {
-      return client.suspendReadIfResumed() && suspendedClients.add(client);
+      return suspendedClients.add(client) && client.suspendReadIfResumed();
     }
   }
 
@@ -395,6 +426,8 @@ public class DataList
         }
         suspendedClients.clear();
       }
+    } else {
+      logger.debug("Keeping clients: {} suspended, numberOfInMemBlockPermits={}, Listeners: {}", suspendedClients, numberOfInMemBlockPermits, all_listeners);
     }
     return resumedSuspendedClients;
   }
@@ -409,7 +442,7 @@ public class DataList
     return new byte[blockSize];
   }
 
-  public void addBuffer(byte[] array)
+  public synchronized void addBuffer(byte[] array)
   {
     final int numberOfInMemBlockPermits = this.numberOfInMemBlockPermits.decrementAndGet();
     if (numberOfInMemBlockPermits < 0) {
@@ -468,8 +501,7 @@ public class DataList
         oldestBlockIndex = index;
         oldestReadOffset = entry.getValue().getReadOffset();
         status.slowestConsumer = entry.getKey();
-      }
-      else if (index == oldestBlockIndex && entry.getValue().getReadOffset() < oldestReadOffset) {
+      } else if (index == oldestBlockIndex && entry.getValue().getReadOffset() < oldestReadOffset) {
         oldestReadOffset = entry.getValue().getReadOffset();
         status.slowestConsumer = entry.getKey();
       }
@@ -481,8 +513,7 @@ public class DataList
       status.numBytesAllocated += b.data.length;
       if (oldestBlockIndex == i) {
         status.numBytesWaiting += b.writingOffset - oldestReadOffset;
-      }
-      else if (oldestBlockIndex < i) {
+      } else if (oldestBlockIndex < i) {
         status.numBytesWaiting += b.writingOffset - b.readingOffset;
       }
       b = b.next;
@@ -508,7 +539,7 @@ public class DataList
     /**
      * actual data - stored as length followed by actual data.
      */
-    byte data[];
+    byte[] data;
     /**
      * readingOffset is the offset of the first valid byte in the array.
      */
@@ -536,8 +567,8 @@ public class DataList
     /**
      * how count of references to this block.
      */
-    AtomicInteger refCount;
-    Future future;
+    private final AtomicInteger refCount;
+    private Future<?> future;
 
     public Block(String id, int size)
     {
@@ -566,8 +597,7 @@ public class DataList
         if (current.offset + current.length > writingOffset) {
           current.length = 0;
         }
-      }
-      else {
+      } else {
         current.length = 0;
       }
     }
@@ -581,7 +611,7 @@ public class DataList
           SerializedData sd = dli.next();
           switch (sd.buffer[sd.dataOffset]) {
             case MessageType.RESET_WINDOW_VALUE:
-              ResetWindowTuple rwt = (ResetWindowTuple) Tuple.getTuple(sd.buffer, sd.dataOffset, sd.length - sd.dataOffset + sd.offset);
+              ResetWindowTuple rwt = (ResetWindowTuple)Tuple.getTuple(sd.buffer, sd.dataOffset, sd.length - sd.dataOffset + sd.offset);
               bs = (long)rwt.getBaseSeconds() << 32;
               if (bs > windowId) {
                 writingOffset = sd.offset;
@@ -590,12 +620,15 @@ public class DataList
               break;
 
             case MessageType.BEGIN_WINDOW_VALUE:
-              BeginWindowTuple bwt = (BeginWindowTuple) Tuple.getTuple(sd.buffer, sd.dataOffset, sd.length - sd.dataOffset + sd.offset);
+              BeginWindowTuple bwt = (BeginWindowTuple)Tuple.getTuple(sd.buffer, sd.dataOffset, sd.length - sd.dataOffset + sd.offset);
               if ((bs | bwt.getWindowId()) >= windowId) {
                 writingOffset = sd.offset;
                 break done;
               }
               break;
+
+            default:
+              break;
           }
         }
       }
@@ -628,19 +661,19 @@ public class DataList
           SerializedData sd = dli.next();
           switch (sd.buffer[sd.dataOffset]) {
             case MessageType.RESET_WINDOW_VALUE:
-              ResetWindowTuple rwt = (ResetWindowTuple) Tuple.getTuple(sd.buffer, sd.dataOffset, sd.length - sd.dataOffset + sd.offset);
+              ResetWindowTuple rwt = (ResetWindowTuple)Tuple.getTuple(sd.buffer, sd.dataOffset, sd.length - sd.dataOffset + sd.offset);
               bs = (long)rwt.getBaseSeconds() << 32;
               lastReset = sd;
               break;
 
             case MessageType.BEGIN_WINDOW_VALUE:
-              BeginWindowTuple bwt = (BeginWindowTuple) Tuple.getTuple(sd.buffer, sd.dataOffset, sd.length - sd.dataOffset + sd.offset);
+              BeginWindowTuple bwt = (BeginWindowTuple)Tuple.getTuple(sd.buffer, sd.dataOffset, sd.length - sd.dataOffset + sd.offset);
               if ((bs | bwt.getWindowId()) > longWindowId) {
                 found = true;
                 if (lastReset != null) {
-                /*
-                 * Restore the last Reset tuple if there was any and adjust the writingOffset to the beginning of the reset tuple.
-                 */
+                  /*
+                   * Restore the last Reset tuple if there was any and adjust the writingOffset to the beginning of the reset tuple.
+                   */
                   if (sd.offset >= lastReset.length) {
                     sd.offset -= lastReset.length;
                     if (!(sd.buffer == lastReset.buffer && sd.offset == lastReset.offset)) {
@@ -655,6 +688,10 @@ public class DataList
 
                 break done;
               }
+              break;
+
+            default:
+              break;
           }
         }
       }
@@ -671,8 +708,7 @@ public class DataList
           System.arraycopy(lastReset.buffer, lastReset.offset, this.data, this.readingOffset, lastReset.length);
           this.starting_window = this.ending_window = bs;
           //logger.debug("=20140220= reassign the windowids {}", this);
-        }
-        else {
+        } else {
           this.readingOffset = this.writingOffset;
           this.starting_window = this.ending_window = longWindowId;
           //logger.debug("=20140220= avoid the windowids {}", this);
@@ -692,8 +728,7 @@ public class DataList
           sd.offset = 0;
           sd.dataOffset = VarInt.write(sd.length - i, sd.buffer, sd.offset, i);
           sd.buffer[sd.dataOffset] = MessageType.NO_MESSAGE_VALUE;
-        }
-        else {
+        } else {
           logger.warn("Unhandled condition while purging the data purge to offset {}", sd.offset);
         }
 
@@ -710,15 +745,17 @@ public class DataList
         {
           byte[] data = storage.retrieve(identifier, uniqueIdentifier);
           synchronized (Block.this) {
-            Block.this.data = data;
-            readingOffset = 0;
-            writingOffset = data.length;
-            if (refCount.get() > 1) {
+            if (Block.this.data == null) {
+              Block.this.data = data;
+              readingOffset = 0;
+              writingOffset = data.length;
               Block.this.notifyAll();
-            }
-            int numberOfInMemBlockPermits = DataList.this.numberOfInMemBlockPermits.decrementAndGet();
-            if (numberOfInMemBlockPermits < 0) {
-              logger.warn("Exceeded allowed memory block allocation by {}", -numberOfInMemBlockPermits);
+              int numberOfInMemBlockPermits = DataList.this.numberOfInMemBlockPermits.decrementAndGet();
+              if (numberOfInMemBlockPermits < 0) {
+                logger.warn("Exceeded allowed memory block allocation by {}", -numberOfInMemBlockPermits);
+              }
+            } else {
+              logger.debug("Block {} was already loaded into memory", Block.this);
             }
           }
         }
@@ -727,20 +764,34 @@ public class DataList
 
     protected void acquire(boolean wait)
     {
-      if (refCount.getAndIncrement() == 0 && storage != null && data == null) {
+      int refCount = this.refCount.getAndIncrement();
+      synchronized (Block.this) {
+        if (data != null) {
+          return;
+        }
+      }
+      if (refCount == 0 && storage != null) {
         final Runnable retriever = getRetriever();
+        if (future != null && future.cancel(false)) {
+          logger.debug("Block {} future is cancelled", this);
+        }
         if (wait) {
+          future = null;
           retriever.run();
         } else {
           future = storageExecutor.submit(retriever);
         }
-      } else if (wait && data == null) {
+      } else if (wait) {
         try {
           synchronized (Block.this) {
-            wait();
+            if (future == null) {
+              throw new IllegalStateException("No task is scheduled to retrieve block " + Block.this);
+            }
+            while (data == null) {
+              wait();
+            }
           }
-        }
-        catch (InterruptedException ex) {
+        } catch (InterruptedException ex) {
           throw new RuntimeException("Interrupted while waiting for data to be loaded!", ex);
         }
       }
@@ -758,15 +809,16 @@ public class DataList
           }
           if (uniqueIdentifier == 0) {
             logger.warn("Storage returned unexpectedly, please check the status of the spool directory!");
-          }
-          else {
-            //logger.debug("Spooled {} to disk", Block.this);
+          } else {
+            int numberOfInMemBlockPermits = DataList.this.numberOfInMemBlockPermits.get();
             synchronized (Block.this) {
-              if (refCount.get() == 0) {
+              if (refCount.get() == 0 && Block.this.data != null) {
                 Block.this.data = null;
+                numberOfInMemBlockPermits = DataList.this.numberOfInMemBlockPermits.incrementAndGet();
+              } else {
+                logger.debug("Keeping Block {} unchanged", Block.this);
               }
             }
-            int numberOfInMemBlockPermits = DataList.this.numberOfInMemBlockPermits.incrementAndGet();
             assert numberOfInMemBlockPermits < MAX_COUNT_OF_INMEM_BLOCKS : "Number of in memory block permits " + numberOfInMemBlockPermits + " exceeded configured maximum " + MAX_COUNT_OF_INMEM_BLOCKS + '.';
             resumeSuspendedClients(numberOfInMemBlockPermits);
           }
@@ -780,10 +832,17 @@ public class DataList
       if (refCount == 0 && storage != null) {
         assert (next != null);
         final Runnable storer = getStorer(data, readingOffset, writingOffset, storage);
-        if (wait && numberOfInMemBlockPermits.get() == 0) {
+        if (future != null && future.cancel(false)) {
+          logger.debug("Block {} future is cancelled", this);
+        }
+        final int numberOfInMemBlockPermits = DataList.this.numberOfInMemBlockPermits.get();
+        if (wait && numberOfInMemBlockPermits == 0) {
+          future = null;
           storer.run();
-        } else if (numberOfInMemBlockPermits.get() < MAX_COUNT_OF_INMEM_BLOCKS/2) {
+        } else if (numberOfInMemBlockPermits < MAX_COUNT_OF_INMEM_BLOCKS / 2) {
           future = storageExecutor.submit(storer);
+        } else {
+          future = null;
         }
       } else {
         logger.debug("Holding {} in memory due to {} references.", this, refCount);
@@ -809,11 +868,12 @@ public class DataList
     protected void discard(final boolean wait)
     {
       if (storage != null) {
-        if (future != null) {
-          future.cancel(false);
-        }
         final Runnable discarder = getDiscarder();
+        if (future != null && future.cancel(false)) {
+          logger.debug("Block {} future is cancelled", this);
+        }
         if (wait) {
+          future = null;
           discarder.run();
         } else {
           future = storageExecutor.submit(discarder);
@@ -828,7 +888,7 @@ public class DataList
              + ", readingOffset=" + readingOffset + ", writingOffset=" + writingOffset
              + ", starting_window=" + Codec.getStringWindowId(starting_window) + ", ending_window=" + Codec.getStringWindowId(ending_window)
              + ", refCount=" + refCount.get() + ", uniqueIdentifier=" + uniqueIdentifier + ", next=" + (next == null ? "null" : next.identifier)
-             + '}';
+             + ", future=" + (future == null ? "null" : future.isDone() ? "Done" : future.isCancelled() ? "Cancelled" : future) + '}';
     }
 
   }
@@ -895,19 +955,13 @@ public class DataList
     {
       while (size == 0) {
         size = VarInt.read(buffer, readOffset, da.writingOffset, nextOffset);
-        switch (nextOffset.integer) {
-          case -5:
-            throw new RuntimeException("problemo!");
-
-          case -4:
-          case -3:
-          case -2:
-          case -1:
-          case 0:
-            if (da.writingOffset == buffer.length && switchToNextBlock()) {
-              continue;
-            }
-            return false;
+        if (nextOffset.integer > -5 && nextOffset.integer < 1) {
+          if (da.writingOffset == buffer.length && switchToNextBlock()) {
+            continue;
+          }
+          return false;
+        } else if (size == -5) {
+          throw new RuntimeException("problemo!");
         }
       }
 
@@ -965,6 +1019,12 @@ public class DataList
       size = 0;
     }
 
+    @Override
+    public String toString()
+    {
+      return getClass().getName() + '@' + Integer.toHexString(hashCode()) + "{da=" + da + '}';
+    }
+
   }
 
   private static final Logger logger = LoggerFactory.getLogger(DataList.class);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4c6d3f5b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataListener.java
----------------------------------------------------------------------
diff --git a/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataListener.java b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataListener.java
index fd9cebc..4add008 100644
--- a/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataListener.java
+++ b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/DataListener.java
@@ -36,7 +36,7 @@ public interface DataListener
 
   /**
    */
-  public void addedData();
+  public boolean addedData();
 
   /**
    *

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4c6d3f5b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/FastDataList.java
----------------------------------------------------------------------
diff --git a/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/FastDataList.java b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/FastDataList.java
index 939d0c1..6ba7b64 100644
--- a/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/FastDataList.java
+++ b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/FastDataList.java
@@ -99,17 +99,8 @@ public class FastDataList extends DataList
 
     last.writingOffset = writeOffset;
 
-    autoFlushExecutor.submit(new Runnable()
-    {
-      @Override
-      public void run()
-      {
-        for (DataListener dl : all_listeners) {
-          dl.addedData();
-        }
-      }
+    notifyListeners();
 
-    });
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4c6d3f5b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/LogicalNode.java
----------------------------------------------------------------------
diff --git a/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/LogicalNode.java b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/LogicalNode.java
index 40a8207..f867d69 100644
--- a/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/LogicalNode.java
+++ b/bufferserver/src/main/java/com/datatorrent/bufferserver/internal/LogicalNode.java
@@ -73,8 +73,7 @@ public class LogicalNode implements DataListener
 
     if (iterator instanceof DataListIterator) {
       this.iterator = (DataListIterator)iterator;
-    }
-    else {
+    } else {
       throw new IllegalArgumentException("iterator does not belong to DataListIterator class");
     }
 
@@ -195,12 +194,12 @@ public class LogicalNode implements DataListener
             case MessageType.BEGIN_WINDOW_VALUE:
               tuple = Tuple.getTuple(data.buffer, data.dataOffset, data.length - data.dataOffset + data.offset);
               logger.debug("{}->{} condition {} =? {}",
-                           new Object[] {
-                upstream,
-                group,
-                Codec.getStringWindowId(baseSeconds | tuple.getWindowId()),
-                Codec.getStringWindowId(skipWindowId)
-              });
+                  new Object[] {
+                      upstream,
+                      group,
+                      Codec.getStringWindowId(baseSeconds | tuple.getWindowId()),
+                      Codec.getStringWindowId(skipWindowId)
+                  });
               if ((baseSeconds | tuple.getWindowId()) > skipWindowId) {
                 logger.debug("caught up {}->{} skipping {} payload tuples", upstream, group, skippedPayloadTuples);
                 ready = GiveAll.getInstance().distribute(physicalNodes, data);
@@ -219,8 +218,7 @@ public class LogicalNode implements DataListener
               logger.debug("Message {} was not distributed to {}", MessageType.valueOf(data.buffer[data.dataOffset]), physicalNodes);
           }
         }
-      }
-      catch (InterruptedException ie) {
+      } catch (InterruptedException ie) {
         throw new RuntimeException(ie);
       }
 
@@ -232,9 +230,8 @@ public class LogicalNode implements DataListener
     logger.debug("Exiting catch up because caughtup = {}", caughtup);
   }
 
-  @SuppressWarnings("fallthrough")
   @Override
-  public void addedData()
+  public boolean addedData()
   {
     if (isReady()) {
       if (caughtup) {
@@ -257,6 +254,8 @@ public class LogicalNode implements DataListener
                 case MessageType.RESET_WINDOW_VALUE:
                   Tuple resetWindow = Tuple.getTuple(data.buffer, data.dataOffset, data.length - data.dataOffset + data.offset);
                   baseSeconds = (long)resetWindow.getBaseSeconds() << 32;
+                  ready = GiveAll.getInstance().distribute(physicalNodes, data);
+                  break;
 
                 default:
                   //logger.debug("sending data of type {}", MessageType.valueOf(data.buffer[data.dataOffset]));
@@ -264,8 +263,7 @@ public class LogicalNode implements DataListener
                   break;
               }
             }
-          }
-          else {
+          } else {
             while (ready && iterator.hasNext()) {
               SerializedData data = iterator.next();
               switch (data.buffer[data.dataOffset]) {
@@ -287,6 +285,8 @@ public class LogicalNode implements DataListener
                 case MessageType.RESET_WINDOW_VALUE:
                   tuple = Tuple.getTuple(data.buffer, data.dataOffset, data.length - data.dataOffset + data.offset);
                   baseSeconds = (long)tuple.getBaseSeconds() << 32;
+                  ready = GiveAll.getInstance().distribute(physicalNodes, data);
+                  break;
 
                 default:
                   ready = GiveAll.getInstance().distribute(physicalNodes, data);
@@ -294,15 +294,14 @@ public class LogicalNode implements DataListener
               }
             }
           }
-        }
-        catch (InterruptedException ie) {
+        } catch (InterruptedException ie) {
           throw new RuntimeException(ie);
         }
-      }
-      else {
+      } else {
         catchUp();
       }
     }
+    return !ready;
   }
 
   /**
@@ -345,7 +344,7 @@ public class LogicalNode implements DataListener
   @Override
   public String toString()
   {
-    return "LogicalNode{" + "upstream=" + upstream + ", group=" + group + ", partitions=" + partitions + '}';
+    return "LogicalNode{" + "upstream=" + upstream + ", group=" + group + ", partitions=" + partitions + ", iterator=" + iterator + '}';
   }
 
   private static final Logger logger = LoggerFactory.getLogger(LogicalNode.class);

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4c6d3f5b/bufferserver/src/main/java/com/datatorrent/bufferserver/server/Server.java
----------------------------------------------------------------------
diff --git a/bufferserver/src/main/java/com/datatorrent/bufferserver/server/Server.java b/bufferserver/src/main/java/com/datatorrent/bufferserver/server/Server.java
index 9f31e02..cd45738 100644
--- a/bufferserver/src/main/java/com/datatorrent/bufferserver/server/Server.java
+++ b/bufferserver/src/main/java/com/datatorrent/bufferserver/server/Server.java
@@ -29,7 +29,13 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map.Entry;
-import java.util.concurrent.*;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -37,7 +43,12 @@ import org.slf4j.LoggerFactory;
 import com.datatorrent.bufferserver.internal.DataList;
 import com.datatorrent.bufferserver.internal.FastDataList;
 import com.datatorrent.bufferserver.internal.LogicalNode;
-import com.datatorrent.bufferserver.packet.*;
+import com.datatorrent.bufferserver.packet.PayloadTuple;
+import com.datatorrent.bufferserver.packet.PublishRequestTuple;
+import com.datatorrent.bufferserver.packet.PurgeRequestTuple;
+import com.datatorrent.bufferserver.packet.ResetRequestTuple;
+import com.datatorrent.bufferserver.packet.SubscribeRequestTuple;
+import com.datatorrent.bufferserver.packet.Tuple;
 import com.datatorrent.bufferserver.storage.Storage;
 import com.datatorrent.common.util.NameableThreadFactory;
 import com.datatorrent.netlet.AbstractLengthPrependerClient;
@@ -100,16 +111,15 @@ public class Server implements ServerListener
   @Override
   public void unregistered(SelectionKey key)
   {
-        serverHelperExecutor.shutdown();
-        storageHelperExecutor.shutdown();
-        try {
-          serverHelperExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
-        }
-        catch (InterruptedException ex) {
-          logger.debug("Executor Termination", ex);
-        }
-        logger.info("Server stopped listening at {}", address);
-      }
+    serverHelperExecutor.shutdown();
+    storageHelperExecutor.shutdown();
+    try {
+      serverHelperExecutor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException ex) {
+      logger.debug("Executor Termination", ex);
+    }
+    logger.info("Server stopped listening at {}", address);
+  }
 
   public synchronized InetSocketAddress run(EventLoop eventloop)
   {
@@ -117,8 +127,7 @@ public class Server implements ServerListener
     while (address == null) {
       try {
         wait(20);
-      }
-      catch (InterruptedException ex) {
+      } catch (InterruptedException ex) {
         throw new RuntimeException(ex);
       }
     }
@@ -142,8 +151,7 @@ public class Server implements ServerListener
     int port;
     if (args.length > 0) {
       port = Integer.parseInt(args[0]);
-    }
-    else {
+    } else {
       port = 0;
     }
 
@@ -173,8 +181,7 @@ public class Server implements ServerListener
     byte[] message;
     if (dl == null) {
       message = ("Invalid identifier '" + request.getIdentifier() + "'").getBytes();
-    }
-    else {
+    } else {
       dl.purge(request.getBaseSeconds(), request.getWindowId());
       message = ("Request sent for processing: " + request).getBytes();
     }
@@ -201,8 +208,7 @@ public class Server implements ServerListener
     byte[] message;
     if (dl == null) {
       message = ("Invalid identifier '" + request.getIdentifier() + "'").getBytes();
-    }
-    else {
+    } else {
       AbstractLengthPrependerClient channel = publisherChannels.remove(request.getIdentifier());
       if (channel != null) {
         eventloop.disconnect(channel);
@@ -252,8 +258,7 @@ public class Server implements ServerListener
       ln = subscriberGroups.get(type);
       ln.boot(eventloop);
       ln.addConnection(connection);
-    }
-    else {
+    } else {
       /*
        * if there is already a datalist registered for the type in which this client is interested,
        * then get a iterator on the data items of that data list. If the datalist is not registered,
@@ -263,8 +268,7 @@ public class Server implements ServerListener
       if (publisherBuffers.containsKey(upstream_identifier)) {
         dl = publisherBuffers.get(upstream_identifier);
         //logger.debug("old list = {}", dl);
-      }
-      else {
+      } else {
         dl = Tuple.FAST_VERSION.equals(request.getVersion()) ? new FastDataList(upstream_identifier, blockSize, numberOfCacheBlocks) : new DataList(upstream_identifier, blockSize, numberOfCacheBlocks);
         publisherBuffers.put(upstream_identifier, dl);
         //logger.debug("new list = {}", dl);
@@ -315,12 +319,10 @@ public class Server implements ServerListener
       dl = publisherBuffers.get(identifier);
       try {
         dl.rewind(request.getBaseSeconds(), request.getWindowId());
-      }
-      catch (IOException ie) {
+      } catch (IOException ie) {
         throw new RuntimeException(ie);
       }
-    }
-    else {
+    } else {
       dl = Tuple.FAST_VERSION.equals(request.getVersion()) ? new FastDataList(identifier, blockSize, numberOfCacheBlocks) : new DataList(identifier, blockSize, numberOfCacheBlocks);
       publisherBuffers.put(identifier, dl);
     }
@@ -422,8 +424,7 @@ public class Server implements ServerListener
               }
 
             };
-          }
-          else {
+          } else {
             publisher = new Publisher(dl, (long)request.getBaseSeconds() << 32 | request.getWindowId());
           }
 
@@ -457,8 +458,7 @@ public class Server implements ServerListener
 //          }
           if (subscriberRequest.getVersion().equals(Tuple.FAST_VERSION)) {
             subscriber = new Subscriber(subscriberRequest.getStreamType(), subscriberRequest.getMask(), subscriberRequest.getPartitions(), bufferSize);
-          }
-          else {
+          } else {
             subscriber = new Subscriber(subscriberRequest.getStreamType(), subscriberRequest.getMask(), subscriberRequest.getPartitions(), bufferSize)
             {
               @Override
@@ -494,8 +494,7 @@ public class Server implements ServerListener
           logger.info("Received purge request: {}", request);
           try {
             handlePurgeRequest((PurgeRequestTuple)request, this);
-          }
-          catch (IOException io) {
+          } catch (IOException io) {
             throw new RuntimeException(io);
           }
           break;
@@ -504,8 +503,7 @@ public class Server implements ServerListener
           logger.info("Received reset all request: {}", request);
           try {
             handleResetRequest((ResetRequestTuple)request, this);
-          }
-          catch (IOException io) {
+          } catch (IOException io) {
             throw new RuntimeException(io);
           }
           break;
@@ -636,9 +634,13 @@ public class Server implements ServerListener
         {
           final int interestOps = key.interestOps();
           if ((interestOps & SelectionKey.OP_READ) == 0) {
-            logger.debug("Resuming read on key {} with attachment {}", key, key.attachment());
-            read(0);
-            key.interestOps(interestOps | SelectionKey.OP_READ);
+            if (readExt(0)) {
+              logger.debug("Resuming read on key {} with attachment {}", key, key.attachment());
+              key.interestOps(interestOps | SelectionKey.OP_READ);
+            } else {
+              logger.debug("Keeping read on key {} with attachment {} suspended. ", key, key.attachment(), datalist);
+              datalist.notifyListeners();
+            }
           }
         }
       });
@@ -648,6 +650,11 @@ public class Server implements ServerListener
     @Override
     public void read(int len)
     {
+      readExt(len);
+    }
+
+    private boolean readExt(int len)
+    {
       //logger.debug("read {} bytes", len);
       writeOffset += len;
       do {
@@ -664,18 +671,20 @@ public class Server implements ServerListener
                    * new byteBuffer and start as if we always had full room but not enough data.
                    */
                   if (!switchToNewBufferOrSuspendRead(buffer, readOffset)) {
-                    return;
+                    return false;
                   }
                 }
-              }
-              else if (dirty) {
+              } else if (dirty) {
                 dirty = false;
                 datalist.flush(writeOffset);
               }
-              return;
+              return true;
 
             case 0:
               continue;
+
+            default:
+              break;
           }
         }
 
@@ -683,8 +692,7 @@ public class Server implements ServerListener
           onMessage(buffer, readOffset, size);
           readOffset += size;
           size = 0;
-        }
-        else {
+        } else {
           if (writeOffset == buffer.length) {
             dirty = false;
             datalist.flush(writeOffset);
@@ -694,14 +702,14 @@ public class Server implements ServerListener
             if (!switchToNewBufferOrSuspendRead(buffer, readOffset - VarInt.getSize(size))) {
               readOffset -= VarInt.getSize(size);
               size = 0;
-              return;
+              return false;
             }
             size = 0;
           } else if (dirty) {
             dirty = false;
             datalist.flush(writeOffset);
           }
-          return;
+          return true;
         }
       }
       while (true);
@@ -751,8 +759,7 @@ public class Server implements ServerListener
       if (cce instanceof RejectedExecutionException && serverHelperExecutor.isTerminated()) {
         logger.warn("Terminated Executor Exception for {}.", this, cce);
         el.disconnect(this);
-      }
-      else {
+      } else {
         super.handleException(cce, el);
       }
     }
@@ -836,8 +843,7 @@ public class Server implements ServerListener
         if (len < remainingCapacity) {
           remainingCapacity = len;
           byteBuffer.position(writeOffset + remainingCapacity);
-        }
-        else {
+        } else {
           byteBuffer.position(buffer.length);
         }
         System.arraycopy(array, offset, buffer, writeOffset, remainingCapacity);



[04/50] [abbrv] incubator-apex-core git commit: APEX-158 - buffer server tests intermittently fail. Temporarily disable testng provider while investigating why unit tests fail.

Posted by th...@apache.org.
APEX-158 - buffer server tests intermittently fail. Temporarily disable testng provider while investigating why unit tests fail.


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/9990e159
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/9990e159
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/9990e159

Branch: refs/heads/master
Commit: 9990e159599998bb3a6f574c8b51a73d5eaf2c3a
Parents: 0a85586
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Sat Sep 26 09:30:39 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Sat Sep 26 09:30:39 2015 -0700

----------------------------------------------------------------------
 bufferserver/pom.xml | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/9990e159/bufferserver/pom.xml
----------------------------------------------------------------------
diff --git a/bufferserver/pom.xml b/bufferserver/pom.xml
index 1c5027f..bbd27f2 100644
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@ -35,6 +35,7 @@
 
   <build>
     <plugins>
+      <!--
       <plugin>
         <artifactId>maven-surefire-plugin</artifactId>
         <dependencies>
@@ -45,6 +46,7 @@
           </dependency>
         </dependencies>
       </plugin>
+      -->
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>


[15/50] [abbrv] incubator-apex-core git commit: Merge branch 'APEX-164' of github.com:vrozov/incubator-apex-core into vrozov-APEX-164

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


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/88ee4ad1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/88ee4ad1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/88ee4ad1

Branch: refs/heads/master
Commit: 88ee4ad16a5a3daba98a5e39d3eedda81c97e3ac
Parents: 9193ec8 7957570
Author: Chetan Narsude <ch...@datatorrent.com>
Authored: Fri Oct 2 14:12:30 2015 -0700
Committer: Chetan Narsude <ch...@datatorrent.com>
Committed: Fri Oct 2 14:12:30 2015 -0700

----------------------------------------------------------------------
 api/src/test/resources/log4j.properties                        | 2 +-
 bufferserver/src/test/resources/log4j.properties               | 2 +-
 engine/src/test/resources/log4j.properties                     | 6 +++---
 .../testAppPackage/mydtapp/src/test/resources/log4j.properties | 6 +++---
 4 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[13/50] [abbrv] incubator-apex-core git commit: APEX-174 #resolve #comment modified the import order as per recommendation

Posted by th...@apache.org.
APEX-174 #resolve #comment modified the import order as per recommendation


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/dec20032
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/dec20032
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/dec20032

Branch: refs/heads/master
Commit: dec20032b42d424b14b683ef7816f8c0f49bfa8f
Parents: cf97d0b
Author: Chandni Singh <cs...@apache.org>
Authored: Tue Sep 29 14:00:13 2015 -0700
Committer: Chandni Singh <cs...@apache.org>
Committed: Wed Sep 30 13:01:12 2015 -0700

----------------------------------------------------------------------
 apex_checks.xml                                            | 5 ++---
 bufferserver/pom.xml                                       | 2 +-
 common/pom.xml                                             | 2 +-
 engine/pom.xml                                             | 2 +-
 engine/src/main/java/org/apache/log4j/DTLoggerFactory.java | 4 ++--
 5 files changed, 7 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/dec20032/apex_checks.xml
----------------------------------------------------------------------
diff --git a/apex_checks.xml b/apex_checks.xml
index 24ce9df..7ba9ea4 100644
--- a/apex_checks.xml
+++ b/apex_checks.xml
@@ -136,10 +136,9 @@
     </module>
 
     <module name="ImportOrder">
-      <property name="groups" value="/^javax?\./,org,com,*,/com\.datatorrent/"/>
+      <property name="groups" value="/^javax?\./,org,/org\.apache/,com,/com\.datatorrent/,*"/>
       <property name="ordered" value="true"/>
-      <property name="separated" value="true"/>
-      <property name="option" value="above"/>
+      <property name="option" value="bottom"/>
       <property name="sortStaticImportsAlphabetically" value="true"/>
     </module>
 

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/dec20032/bufferserver/pom.xml
----------------------------------------------------------------------
diff --git a/bufferserver/pom.xml b/bufferserver/pom.xml
index bbd27f2..773add8 100644
--- a/bufferserver/pom.xml
+++ b/bufferserver/pom.xml
@@ -51,7 +51,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
-          <maxAllowedViolations>123</maxAllowedViolations>
+          <maxAllowedViolations>121</maxAllowedViolations>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/dec20032/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 20a9d1b..12a1d30 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -74,7 +74,7 @@
       <groupId>org.apache.maven.plugins</groupId>
       <artifactId>maven-checkstyle-plugin</artifactId>
       <configuration>
-        <maxAllowedViolations>77</maxAllowedViolations>
+        <maxAllowedViolations>76</maxAllowedViolations>
       </configuration>
     </plugin>
   </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/dec20032/engine/pom.xml
----------------------------------------------------------------------
diff --git a/engine/pom.xml b/engine/pom.xml
index 15e0565..ae334f2 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>2238</maxAllowedViolations>
+          <maxAllowedViolations>2209</maxAllowedViolations>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/dec20032/engine/src/main/java/org/apache/log4j/DTLoggerFactory.java
----------------------------------------------------------------------
diff --git a/engine/src/main/java/org/apache/log4j/DTLoggerFactory.java b/engine/src/main/java/org/apache/log4j/DTLoggerFactory.java
index 04a4238..04e6856 100644
--- a/engine/src/main/java/org/apache/log4j/DTLoggerFactory.java
+++ b/engine/src/main/java/org/apache/log4j/DTLoggerFactory.java
@@ -87,9 +87,9 @@ public class DTLoggerFactory implements LoggerFactory
       String loggersLevel = System.getProperty(DT_LOGGERS_LEVEL);
       if (!Strings.isNullOrEmpty(loggersLevel)) {
         Map<String, String> targetChanges = Maps.newHashMap();
-        String targets[] = loggersLevel.split(",");
+        String[] targets = loggersLevel.split(",");
         for (String target : targets) {
-          String parts[] = target.split(":");
+          String[] parts = target.split(":");
           targetChanges.put(parts[0], parts[1]);
         }
         changeLoggersLevel(targetChanges);


[39/50] [abbrv] incubator-apex-core git commit: APEX-212 #resolve Adding a check for physical operators list being empty in fillOperatorInfo

Posted by th...@apache.org.
APEX-212 #resolve Adding a check for physical operators list being empty in fillOperatorInfo


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/ce44d237
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/ce44d237
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/ce44d237

Branch: refs/heads/master
Commit: ce44d2371501a8533d498a5e93de6def2a6cd142
Parents: 0aaec0c
Author: ishark <is...@datatorrent.com>
Authored: Wed Oct 21 16:49:00 2015 -0700
Committer: ishark <is...@datatorrent.com>
Committed: Wed Oct 21 17:50:25 2015 -0700

----------------------------------------------------------------------
 .../com/datatorrent/stram/StreamingContainerManager.java    | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ce44d237/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 3931fad..ca724db 100644
--- a/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
+++ b/engine/src/main/java/com/datatorrent/stram/StreamingContainerManager.java
@@ -2396,9 +2396,12 @@ public class StreamingContainerManager implements PlanContext
         }
       }
     }
-    loi.checkpointTimeMA = checkpointTimeAggregate.getAvg().longValue();
-    loi.counters = latestLogicalCounters.get(operator.getName());
-    loi.autoMetrics = latestLogicalMetrics.get(operator.getName());
+    if (physicalOperators.size() > 0) {
+      loi.checkpointTimeMA = checkpointTimeAggregate.getAvg().longValue();
+      loi.counters = latestLogicalCounters.get(operator.getName());
+      loi.autoMetrics = latestLogicalMetrics.get(operator.getName());
+    }
+
     return loi;
   }
 


[50/50] [abbrv] incubator-apex-core git commit: Merge v3.2.0-incubating into master

Posted by th...@apache.org.
Merge v3.2.0-incubating into master


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/712138ac
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/712138ac
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/712138ac

Branch: refs/heads/master
Commit: 712138ac4fc31a794e54db48f8c4c5b778d8fae5
Parents: d61ca61 832901c
Author: MalharJenkins <je...@datatorrent.com>
Authored: Mon Nov 2 18:07:16 2015 -0800
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Mon Nov 2 18:17:04 2015 -0800

----------------------------------------------------------------------

----------------------------------------------------------------------



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

Posted by th...@apache.org.
Merge branch 'pom-fix' 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/5324f108
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/5324f108
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/5324f108

Branch: refs/heads/master
Commit: 5324f10827afa9693a8def66d4f5c554185888c9
Parents: 2bd6a8d 0aa7006
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Sun Oct 4 17:08:21 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Sun Oct 4 17:08:21 2015 -0700

----------------------------------------------------------------------
 pom.xml | 15 +++++++--------
 1 file changed, 7 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[32/50] [abbrv] incubator-apex-core git commit: APEX-198 Modify test to create user directory under target.

Posted by th...@apache.org.
APEX-198 Modify test to create user directory under target.


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/5a90bb70
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/5a90bb70
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/5a90bb70

Branch: refs/heads/master
Commit: 5a90bb701cafde8c88ce69645496a082c04d7fcf
Parents: 979a0ef
Author: David Yan <da...@datatorrent.com>
Authored: Tue Oct 20 14:50:02 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Tue Oct 20 15:51:53 2015 -0700

----------------------------------------------------------------------
 .../com/datatorrent/stram/cli/DTCliTest.java    |  7 +++-
 .../resources/testAppPackage/.dt/dt-site.xml    | 39 --------------------
 .../test/resources/testAppPackage/dt-site.xml   | 39 ++++++++++++++++++++
 3 files changed, 44 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/5a90bb70/engine/src/test/java/com/datatorrent/stram/cli/DTCliTest.java
----------------------------------------------------------------------
diff --git a/engine/src/test/java/com/datatorrent/stram/cli/DTCliTest.java b/engine/src/test/java/com/datatorrent/stram/cli/DTCliTest.java
index f8e1789..59a45aa 100644
--- a/engine/src/test/java/com/datatorrent/stram/cli/DTCliTest.java
+++ b/engine/src/test/java/com/datatorrent/stram/cli/DTCliTest.java
@@ -64,7 +64,10 @@ public class DTCliTest
   {
     try {
       userHome = System.getProperty("user.home");
-      env.put("HOME", System.getProperty("user.dir") + "/src/test/resources/testAppPackage");
+      String newHome = System.getProperty("user.dir") + "/target";
+      FileUtils.forceMkdir(new File(newHome + "/.dt"));
+      FileUtils.copyFile(new File(System.getProperty("user.dir") + "/src/test/resources/testAppPackage/dt-site.xml"), new File(newHome + "/.dt/dt-site.xml"));
+      env.put("HOME", newHome);
       setEnv(env);
 
       cli.init();
@@ -83,7 +86,7 @@ public class DTCliTest
   public static void finished()
   {
     try {
-      env.put("HOME", System.getProperty("user.dir") + userHome);
+      env.put("HOME", userHome);
       setEnv(env);
 
       StramTestSupport.removeAppPackageFile();

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/5a90bb70/engine/src/test/resources/testAppPackage/.dt/dt-site.xml
----------------------------------------------------------------------
diff --git a/engine/src/test/resources/testAppPackage/.dt/dt-site.xml b/engine/src/test/resources/testAppPackage/.dt/dt-site.xml
deleted file mode 100644
index f247ac9..0000000
--- a/engine/src/test/resources/testAppPackage/.dt/dt-site.xml
+++ /dev/null
@@ -1,39 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!--
-
-    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.
-
--->
-<configuration>
-  <property>
-    <name>dt.test.1</name>
-    <value>user-home-config</value>
-  </property>
-  <property>
-    <name>dt.test.2</name>
-    <value>user-home-config</value>
-  </property>
-  <property>
-    <name>dt.test.3</name>
-    <value>user-home-config</value>
-  </property>
-  <property>
-    <name>dt.test.4</name>
-    <value>user-home-config</value>
-  </property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/5a90bb70/engine/src/test/resources/testAppPackage/dt-site.xml
----------------------------------------------------------------------
diff --git a/engine/src/test/resources/testAppPackage/dt-site.xml b/engine/src/test/resources/testAppPackage/dt-site.xml
new file mode 100644
index 0000000..f247ac9
--- /dev/null
+++ b/engine/src/test/resources/testAppPackage/dt-site.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+
+    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.
+
+-->
+<configuration>
+  <property>
+    <name>dt.test.1</name>
+    <value>user-home-config</value>
+  </property>
+  <property>
+    <name>dt.test.2</name>
+    <value>user-home-config</value>
+  </property>
+  <property>
+    <name>dt.test.3</name>
+    <value>user-home-config</value>
+  </property>
+  <property>
+    <name>dt.test.4</name>
+    <value>user-home-config</value>
+  </property>
+</configuration>


[16/50] [abbrv] incubator-apex-core git commit: Fix license header related info in README.md, remove obsolete files.

Posted by th...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/ec03f12c/engine/src/test/resources/codeartisan_ RSA Public Key Cryptography in Java.pdf
----------------------------------------------------------------------
diff --git a/engine/src/test/resources/codeartisan_ RSA Public Key Cryptography in Java.pdf b/engine/src/test/resources/codeartisan_ RSA Public Key Cryptography in Java.pdf
deleted file mode 100755
index c054f0b..0000000
--- a/engine/src/test/resources/codeartisan_ RSA Public Key Cryptography in Java.pdf	
+++ /dev/null
@@ -1,1932 +0,0 @@
-%PDF-1.4
-%����
-15 0 obj
-<</Type /Catalog
-/Pages 1 0 R
-/Dests 16 0 R
->>
-endobj
-17 0 obj
-<</Type /Page
-/Parent 1 0 R
-/Resources <</ExtGState <</G0 18 0 R
->>
-/XObject <</X0 19 0 R
-/X1 20 0 R
-/X2 21 0 R
->>
-/Font <</F0 22 0 R
-/F1 23 0 R
-/F2 24 0 R
-/F3 25 0 R
-/F4 26 0 R
->>
-/ProcSet [/PDF /Text /ImageB /ImageC /ImageI]
->>
-/MediaBox [0 0 612 792]
-/Annots [<</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [45.8182 582.8052 173.1559 595.619]
-/A <</Type /Action
-/S /URI
-/URI (http://en.wikipedia.org/wiki/Public-key_cryptography)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [106.684 569.9913 298.0909 582.8052]
-/A <</Type /Action
-/S /URI
-/URI (http://java.sun.com/javase/technologies/security/)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [302.0952 569.9913 380.5801 582.8052]
-/A <</Type /Action
-/S /URI
-/URI (http://java.sun.com/j2se/1.4.2/docs/guide/security/jce/JCERefGuide.html)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [294.8875 544.3636 334.1299 557.1775]
-/A <</Type /Action
-/S /URI
-/URI (http://www.openssl.org/)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [301.2944 530.7489 322.1169 543.5627]
-/A <</Type /Action
-/S /URI
-/URI (http://en.wikipedia.org/wiki/RSA)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [267.658 443.4545 334.9308 456.2684]
-/A <</Type /Action
-/S /URI
-/URI (http://java.sun.com/j2se/1.5.0/docs/api/javax/crypto/package-frame.html)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [153.9351 280.8788 248.4372 293.6926]
-/A <</Type /Action
-/S /URI
-/URI (http://en.wikipedia.org/wiki/X.509)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [67.4416 216.0086 159.5411 228.8225]
-/A <</Type /Action
-/S /URI
-/URI (http://java.sun.com/j2se/1.3/docs/tooldocs/win32/keytool.html)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 627.6537 470.2771 640.4675]
-/A <</Type /Action
-/S /URI
-/URI (http://www.blogger.com/profile/00968563763838190660)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 606.8312 470.2771 619.645]
-/A <</Type /Action
-/S /URI
-/URI (http://www.blogger.com/profile/16766484929210129406)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 548.368 429.4329 562.7836]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/feeds/posts/default)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [433.4373 549.9697 507.1169 562.7836]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/feeds/posts/default)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 449.8615 433.4373 462.6753]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [433.4373 449.8615 459.0649 462.6753]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/search?updated-min=2012-01-01T00:00:00-05:00&updated-max=2013-01-01T00:00:00-05:00&max-results=4)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 426.6364 433.4373 439.4502]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [433.4373 426.6364 459.0649 439.4502]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/search?updated-min=2010-01-01T00:00:00-05:00&updated-max=2011-01-01T00:00:00-05:00&max-results=10)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 403.4113 433.4373 416.2251]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [433.4373 403.4113 459.0649 416.2251]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/search?updated-min=2009-01-01T00:00:00-05:00&updated-max=2010-01-01T00:00:00-05:00&max-results=7)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [429.4329 385.7922 446.2511 398.606]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [446.2511 385.7922 488.697 398.606]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/2009_10_01_archive.html)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [429.4329 362.5671 446.2511 375.381]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [446.2511 362.5671 483.0909 375.381]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/2009_08_01_archive.html)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [428.632 339.342 445.4502 352.1558]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [445.4502 339.342 467.0736 352.1558]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/2009_05_01_archive.html)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [442.2468 291.29 538.3506 334.5368]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/2009/05/public-key-cryptography-in-java.html)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [429.4329 260.8571 446.2511 273.671]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [446.2511 260.8571 492.7013 273.671]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/2009_02_01_archive.html)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [429.4329 237.632 446.2511 250.4459]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [446.2511 237.632 487.0952 250.4459]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/2009_01_01_archive.html)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 206.3983 433.4373 219.2121]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [433.4373 206.3983 459.0649 219.2121]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/search?updated-min=2008-01-01T00:00:00-05:00&updated-max=2009-01-01T00:00:00-05:00&max-results=11)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 183.974 433.4373 196.7879]
-/A <</Type /Action
-/S /URI
-/URI (javascript:void\(0\))
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [433.4373 183.974 459.0649 196.7879]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/search?updated-min=2007-01-01T00:00:00-05:00&updated-max=2008-01-01T00:00:00-05:00&max-results=27)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [45.0173 706.9394 169.9524 732.5671]
-/A <</Type /Action
-/S /URI
-/URI (http://codeartisan.blogspot.com/)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 524.342 489.4979 540.3593]
-/A <</Type /Action
-/S /URI
-/URI (http://add.my.yahoo.com/rss?url=http://codeartisan.blogspot.com/feeds/posts/default)
->>
->> <</Type /Annot
-/Subtype /Link
-/Border [0 0 0]
-/Rect [416.619 497.9134 580.7966 516.3333]
-/A <</Type /Action
-/S /URI
-/URI (http://www.newsgator.com/ngs/subscriber/subext.aspx?url=http://codeartisan.blogspot.com/feeds/posts/default)
->>
->>]
-/Contents 27 0 R
->>
-endobj
-27 0 obj
-<</Filter /FlateDecode
-/Length 11803
->> stream
-x��}ۊn9���~�x�ٵ|����u5=��t5CeU��0K����Y_;`�*22�u�-�Z��y]�������c����?�ɯWk�s��-�ҿ�?����o���_7<}�u�=Z�xMo�Ͽ��������o�|���7��۞�F�������_���~��m���?n<����[.o��ۿ\W���|���y`�R��`Y���i���*�Y	Ϊq���_�����פ������'}����s�s��'��C�g���=4�n��5ip������ˀ��b�Ճi�9p���ݳA�{�f��OJ�0�[����l�_Ł�a�x��A^<��h,{�W�s/CZ)ޕ����`kL��i�O\��Jj�RKZ����X����Yj)��<i�й��<{ʝ�y��J��K��-1/)�����>BfJyỏJ_��a��G�1�X��J&�eV��eR.�̝�����ź`[>���O&��𢡊�6R������W�ȍq��7/�z��ϩ��+�D���i՜�2��
 �@7gK���V�H�g����z��%��<Рu�C�����ֹ�̄�id�1e�J�:š��ҕɤ�2�^��X��KIS���j���DʙB��Β���P��`v^�*�l���6Y���E5�J,5�N�.5��n���k|���CY�I�H�S5�wP���yY��杓�he�c�z�׾�2�U��x���Eh���|���˼4rND9WbW�� ,�z�EK��5~iL�$ ��3Ml��0y�k&��l#�q��o��~r�񓋊6�Nm�@֌���Zr���~�6�����F�O��~ؠ��`������߿V�諼�4�z��B"���m��s��TO���_&��~�����	j��^#����{)"��o��������;g7���;�<^��]��K
-3��?؊:�r�=��X {�_
-��!�u��@_�i荩�9ڮ��\��]]`zdq%�		��Q1Aw����x��kR�$�� -�jʕ�k�K%�X���h��R������>��n���
-���VT[�_�둾���H����:�\
-K*��
-K��g�%U��’�hv`yR�GzJ��j3,����S��w鱾����'9�y%]+�_�T�#H��#_"z^XR݉�м4
-�"4�l,�WO��C����m�*][TN���*���u&�j��&F�O��=\����� ���AN�+��;�>:�;�sX�Wu9�V5M���	��u�F:?�ce>As�9��ILm�.��ǻ��Yӭ���Ql��&�E������JB�U�������l��|X6� �ۦ���'ݚ��$�D�+O�"J�jl��AU�y�F0gc���&k-���\�A璉��¼3Ml�g;/�-���o��E�2����>�1S'��K�Ũ*FG�Rg�����d�j�����+�bb(f<C3sh�1�gF�`�\���<t2-,Y|����2c����28�6q��B�^6?bV�iˊG]�R���B�h=wK}Qah �� l.=�Y�0��0?+�se��'�5K���^�QPѓ&#���`R6b�9�$FUDͲ�T2w�#{J2�'Fs�L���y���2���O'V��LLW
-��_�+�d��������"Ɲ� �ȍ��΍�����`��ZyI�D|.O"|^5~�K��7���5�Kjd�bt�ワ��[�A_X+��QO?�J5��,��s1��&@c�6�sy��(\L�����;�Ʃ*U6�DVoM�Z����kӇζ�o��R��[D~w޳e�ܢ#(	��H���H����󥀗�~�葎ڮ�=T��Ny3��C����2��U��R� ��<ޢ��)U�P�k�e�ʱ2���p4\����;�qmOP��Irt��d�!G��f�BXLG;�?���"������1��\��w����s+D��i9���R5.��<&�B��q��Mšg-2o1>�[���/�o��&�� ^R\�6�"�4@K[�bKg־wϡ��ɬ������ԏ���W�s�H��h�^�����P/~��z�<ԋ_�S��y�?�C����P/�X��u���G�/-"���
-��p��U�O:��Gk�p��`c[%�"W/bŦ���b�dTI0����y1�r�(�d��9'����v�7�֝�b���G�B�N	�RXx��tƴ�'�~^���6]W'�(�x
-�3��%<�s�r;�ۄf"ٝ�B���p(�b5�7��"v��[5}�T����m�Ԛ�7�ʤZ5>��ޖ�^Q'���&o{�jQ�q�h❯+4�L�6k�'�R@஑9�!�זA+�wF�F&h'�vjj�R!#%`<s�)��w�
-�5Y���-+�g���1�W3ah'�P���:WJ��Jn��6��yf&�Ŕ��Y�y�2o'��B������f�Z>8�¬5��i�����N��3Y���5:�Z2�wu��\��&e�G{��@�� �/� 矮�j:��]���7�v=�ͯ[;�8�~%=v*�adP�a�6�b�^A��
-5]c>��(���O;��&��S+���� L�r��D�p�W���������'ޮ�������L��Â髿�P{�V5��+)�KK�F���RnD�¬�p;�[��hK�f^H��a�Z�p@l�K�il����.=��N�/F���X��jv�� ����-F��0�-����=��]�3\���睔_グY�&�b6Y�~q��<��b��llDV�hw�ڞ�zTe���0�A�\:�뱘f�!�F�J�bVW����t'��
-�t�bu@���h�υz�7O��g�?��
-�̡�:VD�)cj�5�D�p/x�o��J� Ǚj,��mDHr�%�oc1I�7Vy2��y�1�o[l�^摙4���<;�w1O�ќ�!S2f�A��2�����0ClR)G�h���x�\Z"T��0��$9�w���D�r�7�:��K"�ՠ?V4::X��~�`tG�53���ښ�IdUz!6Y��
-��0~��XjY�j�lc����@}5��������[�K�B���ol�HϜ������u2�(�O� 牳j��5��Hg�K��+�I&���=�
-��u�j���j��H�b�VJsc
-�y2���yd�y��뻆���E2���u�L8��=1�'9�̐4G�@͝p��d<oaZ�˽
-
-;�Ƥљ$��\����3a&��y2��b�h��g^��;���5�YNQcu1Of6�	��\�0~��b���6�ь��<����3�MN:v��$�ט�EC��)�ykll.D�rU7���xh{5�����@��c�k ^Fz�~#�4Z���G­ݢG�Ҭf%}b�2_��АQ���4�/L�z�N�UK!#%�����vdB��h}����
-�sM�V�o)��ԝ�ov	�l؁<�goџ��*7��Mc��l�B��Y������ǣy2kK�����?��Ӌ�&!�rE+�j�c̙y��#�V��$�e���&.L*��<(̓Yl6ڇ��b+o�b~�e��.�����~�+�F���1��	+�oiL��3I|��/,�&���/d�Ŭ�^��\�aޏDYO��6���4�P�~�f.%��?f�����X��`��z��RS�=dj��~h tjj�Q�Ԥ�P����~�������,��P���"�._�H]s�,�m��ۮ���3��-�V.(�T��XGrA9���~�9���`����мkc�M��f��o�{f��Ũ�Ir�p�k��jb�ɋ����7n,�'!�LdW�OB64��L�C��k|J�y{���~M�x{�X�ަ�%h�����/]_�Ě䎕�zi{�;V��4��3[c��r%�QMT�!k���]AX
-x(�+���U�
-�7ѷ�)��Ю����#�9��y[��C狌\-<Ҽ`��]�4Mx�Y����NGұ��U���T/�QmL�P�+��kY�����c�INWP�W^t޵�4��/��3����x^���̓��d�خ��ۆ4VT��,MU�T�qd\�t��$�b��L�r��Hce&�},�W�#��ؙ&��ؠ����%��qC�2��F��	��u.�9�#�ah�����5�}z~D�a�z[b�5�X�����%:������B��M�l���%��H�y޺&>#�	�yi^��j���t%Š��g��P|���p
-9x�i�/nL�n�]�K'�*��+y4TN��;h��"ڊf����l���'���*ÕHc��P;��.�3e(��)C~sMĮ��.�MK����q���rx�K$93q1����D��ۍ%�sah���1���>�,��E����"s̯Y�j'�,�C��d��,^�k"����4�m��}0�hMB�w�	C�d��P7�D��f���K���%����,[��z1���76g�<�~F���9�1��_��EF�i��C�C5�w�7�!7)O"%y��M�y���m�fh?O1���c8��ƨZ�6y�����D�r�J< 1C�F��HUp�3h�c��1�)A���� �;���r�X���]
-���U����W�#9DU4�yM�ÖO���N�1ͥб��k��,�*�FW8Fd�/M�X�7�fh���عG�6��̨��6 �&�y��P�;͆���6�X�"M��uM�1xH�:*���$�2����_m7#��~��Bu�h]�����欁Nǫ��}�#��8���vULp2:=ݮ�Ŵw�=�����R��1|&�c���P����;�<��B���{�y�0���<�����$�/sg4W#I�Fs�]1�J.gi��h�a��qd��t9;���dTP��"���{�chΌ�<���>8����>P����w�UF�س�29Osv
-��]�x�<ٍB5���ٙh_�=�Q��N8��٠A�dG�8�+ Vj4^Bk��"I�#j"Q����P뿇f�t�H G�FZ�Zh�1��vE���I�I^�ԓCvv��/���=jv}525&)���@}����h�MrgM�/�Cj��4U�T�~��@
-3l��~㗵�N��ce���=d����*�I��L�j��Yn)�{g4�P�^�f�}(�X��Č�b<3� T��<eo�(<�j�rJeQE�*Kc���T�A� 7�ȼ�y��4VQ�T1���8Y�!�L�w��M+<�ox�ߴ�z	��HY�輺k�ПQ�⣩�ӊ�%�>@_�̈́�e$g��0�o0=���/"�d4�w�4f�Fq����N/vb�՘�>���GӘ|ͧ�ɭ��ē� �ًQ�ơ�v�8�5��R./����`���VU���1��Љ�����*�>�`����f��н1~����4i����x��Hƶ�h�̞K�4J��#�XT�Kӌ+�`��з�0����#_�-�e����0��H�79ra�@ș0c���WL]��D�G�;��@促gX񗐘V'I���d���	���/�f¬���'�ɼ�����L��:T_	�=3Y������(��k:���L*���:vÁ��]�_��ƶ��E�+�ۘ�\
-�(S�g�}肜�md�װ�#8�wd&�I�Z�h?[���Q%�ubIʮ��5��y��������)̭Y롰k����y'��Dg�Е���y��[/#I��4>�u]&Iy����~�rtu���(�O���dj�o��2�A�˯-�-.��+i�ߏM�qd�ȷx�C*�6�U��[HFL�f'����.��F�Q,+�`T��Ǝ�h��hW9��	њ*��
-g�ɠ��M��n�ߤo�s���z��z]
-�]��>@hD3��i���mUay�Ȥr7/F,�K-]c�5��v5f<�l�a������5Ͱܚڝ��|�;�z��bB�#�x鑾���@�r�d�gN�6��9�����Sc�/q��ϥ���Z����}PHS2�:�sIs��dƯQ4^O�l�^�X�'2�JV���p-@�6�Mz��V�Ad�����L�^�`��x��N�5{�S�:��l4��h{ L��l57X���E�Ո���84����o�Dz�w�j,�Ơ�k!�^��Jc�.���6������`Q��6��oYL�F�÷���tr&ڗj#����v:���݋��>���J�W����R��>��c4�ʽR@�GbM#!g�S�8���׽����񁞊�j�_we��~ધQ9|�T��Sہ�����&\�'l�j�*3.��������P^$ZY���$�X+��7�O)/Ɩ$I76�l�_}T6��8�!h��刏�l<����ےޭB�~�׌�Ό�X�c%ˣB���!L�I�)7bb�.4 �b_nĨy��b����
 ���*�	�:s2��4��ߙ��JƏ5(�1UҖ?�J��rh^Na�䬀�E�􋃩���* �]d���%��h�&:�u��ť%ߠYDB�H2�E�QĈ�Q�_�ȍ�����dT�N��r`l7.ƑQ����d��%�E�'� �y�מ�ٿ�O�p��ƌM��bT�J�R�Q(��(b��^�j�?3�f�X��<)�s�J?�s�2�H��1G�j��U��\����ؚUu��O��vUL�}��A�48�e�� �md^	�s�_��J�<ij H]�}�z
-�B��i��IE�<MS�=u>��eu���O�֞�a"�t���4��OlNJ�	+�i2zÒ�b��Q�\)͝���jd�y^Lγk�32�2&��N�/b�ʓ-s�Ė�U\�Qn�HR�~G!r��)�yvb��h��]�#)�ƒ�))�j��x��,V��L,��i�Ai�0���&�A��U�S�i��pɣ���F3Q���Ց?�=E3��jt�+U��Պ:k�ZC�^�NLۥ�����9�CvC�?R��H��pα�HYV��tw>��A$�s����I8�UӐ`�Z��B�4~���|��8*�co�B3�۾���m_LU�rz�o�fbc�����}�]F���X2\�l����wAe:�FG�'����gs��b�k�q�۾XV�šl
-��*�7�J<E.a¼1ڳ˥��v��l�ܘ�2�슉�X|�j��x�)�َ�����.d��Q���&󕲘���f<V:��1��_.M�A3��$��h���/`?�B��4�ǁRG6=',��?tR	Z��A��xΩ�B��$����b�vI�{�g졉���]����d�kgT��8j�a����%�[��q4/&�ٙ�Va�.&���}���������bLҗ������ո-�f��4��%�3x,�a|����<5GP�]�6v�6�aW#Z(��YAfaL>{�@�'���NJ�*>KcVW:�]�}����vF��4��G�bT�B��.b9���jc,\.b1�����q������g�
-�����<�O�3���g|��5^��1>4��"���sBc��R�99�|��ş�L�`_}ΓA�ƌG����j!�u����/\I�`E
-j��U�d�L�/-�c�OT��h��� ��{�*(6��y!Js5�Mq����yIf��l5xh�k!bi�b����C82e~X��~7�B�����c�f��L�ţ���m���3�w\�0��w�ü��}dJ?�ǻj�����8
-W�e?l������Ï��B9��Y�h��������,����/����mL�©Y�qx]����=}�:ftW��:f��˕a6z��}�`ɇ<�r||�ws�m1�����s$D�P�F�>.�:���"{!6^�
-}��h >P@X�v�������='瘑�B�#�6nq��1����G⋰L� �*�45�pY�`.z^��Q^.�1�X�8�MB:��*.4��NO(h�ε���u���2�_^y���K2o�l^�N�����+��l4��&��S�G"�!-����e�.3L���@��b��]�G�X$��"r�B�L�RĉcAi��� ,����Q�̌f��C�D�#���W�$Ur���]��ՠ�G����#�+�G:�� X��!}�9���1ZQT�v6��b�.M-~��.\Ȅ&iH+\�/G�V�~qd��l;�ŘӀ\* ��ό�1YM�"X��ƮN���v�9[���Ld�����N}.Z�C�һI������+��P�r�<�v�KNde��IL���7���b6)�
-b9�|9W��KYl^��Ur]D��x~A.9ˡ���W�����'��E��}�.�x���p>��p���H찤��iqa@=���w�a�f�p]\��A攥ܵ��"���$$��a5��V��dB�Nh��ڈ��vI���=8�<iF��B��!�_0%�������CiJD9��7�f^_�ύМ�sa��XT���܃S��=��&��o�'yP�&�hNbW��s���5����2HJ��[
-�+�֓�&B�5^��3>����V��$�~Ri,-<�fҨ�"+�xůoD�/"��G�8��n���Ik |����� �+��?��(��u��${�� �(��ө�\��c�|+z}JJ3��s#�H�0����锑?<�b
-؉2���}-vP�Sc�<'��j���(C�f����4cN��y��P�;Q��ȅ�N��X�F[b���D���f���kD�y2��F�����������u�xޒL��;�llç���4�����XDΥ5bWҭ�p�;��9��Ԙ��\(�9��1O6�x�Qc�~ÖY�e�/�$��:]qI�������H�R@8o���	����	SH��h����S��Oj�!�K����d�=<��9���Ĭ�25�o���t��t^��d���~�y�$��7�1Uc1i�A4��hl��w��F��C+�ֵ�'����UFUa��灡4r��0�S�Qe4z��c�d��'^�D��ǰĴ_���4��6��p=,Y�C+��T��kC���tfW�FP8-�4��~^�f�A<K��\��_�1�2ˑ��1��0:��Z�̲����]a�R5R�z$4��mz���6�eݢ���h5_F��>_z��>��cdL�leb�V��leɼŐ�7��b�3l���P�%T�᭴Ψ�ڞ�
 ��t�5e��h6��
-�Չ4$A�e��ƅ�խ�qy����B�P�k��:�R@(8u2�=ҷn��H��C��5�J�\�aM2�|�"*�[|��/��&�ZJ	�7j1%��U�	Z�W-(_����
-
-g/�Y�DC�
->[<���1�|��D�.�+%%E3�.bp�1����Sc��j"�#%�X��1�W��C�mZV�r�3�����Y�kM�Y�x�,-\�d􋃥��"���CE������%�,!�KK��CVvd	yّ偞"K�͎,-Zx���#�z�,�Zwdy���֢:F���ш����j���Ƣ_Ɖ%�#K�ю,��wdy�~^�Y��hfW;�<TA9Z�
-�����,�ø\5��~�m��
->ժe���R�C�J��p��n�n�VR3�j���
-�pi�a%��uJ��9P�Ļ]�x7�G�B>���7�5ɜ;�=�B�1Nw���E��Bvv�Ŵ�]��w/h�{�V�nDu��B��q4�=�xc^ZVp��*���(�h����nǻص���_��xr��ݣ��Վw���x�@}D���w�w��9ZFVP64v'�Wʆ$�e-+8p����-�_ysJ�6�r�2�R6$������}6br���\����O�((��J��QT
-����b��k���LsX���ޑ�����_�� Ʌ+��%�pE��$����<��C>W"�,-!��]ZDpg3eB�΅/(Tr'�\���ս�\cD�;��ع0�F����G33��1���?�����>���Hc��
-����wa"��\��B��ZV�D�iY������y��X����)���C-�\λsa���C-�\G��e���\ja��G��v�y� Zع0���Ck߹0�wY�3��优�\���U��A�����΅a2ٹ�> ����|�x��o������������?_?l�~�����Ǽ�-|�rc����������o�?��o�~����o��������m7����;	M��܁�&�F"?�}�4f����;������Q}�͓����l]��zWk���]l2P���)lO��`�¦��^���if+
-���dT�����<���W�\K���
-��[+=�����z�ܮ�@�SJ
-�E��c<o+����5�hwx|��E�w9L���M�׊61[^+�my�h��嵢����z�W8���N�Z�,/	+/y]{�������q ���F^$�@��.Lf�?�Oz^�0�X����K�ɨ����D���U�	�~�ӫ�����u��������/�k�c3X��kwB?Qڽ�xp�H����B���ӷ;��
-:ؼ�P�o/�6�5Gn�ц�z�'
-]�ji�.�&o�~��Ju���%����B6P������M$>)���f�{UCA7U��}[O���>5U�&FgM�4/�/��/#�i}��-�N�J�n\i���ew%�~!&<��̶���Ț�c!��I�,y�7�Z�����Q_������R�Q�r�/��z�"@����I�
-zТ��c���DZ�_�H���H8�Uc���k�б�+%:�����:?��O�����z�9�.���u����O:��k��3�W�*�ڣ������{�[��ZM�p�h(d����4�cn�CXw3/��ͼ_��e����cj��#���g��-��>��b��X�:�(*qOv�~����n_!c�uw�un��>��YZ�d���������Z��X@���b\�E���g-����{'��i�*��%�����C	�_7#/����֍�~�%�&^:؋榡o���/֢��ߙ3<0�
-
-�����o�PP�G�#��M��G+ii���ki��*������v����Z߶�g���IA�n�5�K�{
-
-+G=/d�{_��]���B4���|�H�XoywL�i��:���a9|��ѳ�$tSʆ#�i~^#��n��_W�|5���wƩ�躂���V��A�ҷ��_$8&��Y:��ǥ@��(\��!=�s��Y�+�d�@�Xۥ����3���0����XGI-��z�ڧ��9��4ǟYv�V�����)��_��.�g-J����h�>�,�[�C��~�>�Q��_���Ds���g�����Xr��Q�|��C�P~xp��
-z(?0����}I�� Z~P}(?(�%'�Uc���GND�/�Q����\'+�A��4�_���X��:&��T+�/���Y�n��O{����������䕣��[�]����F)�-�ҙ�hM�vFB�\����(�i�?���6�u=��\{K��<g?d�~���?�!W|��C��,�I,�ם��P����v�2�/���7��4�ӟ��[��*���nJ���˒�6r�YLy��E)��U�EOm]1��/N��D�5l%�lD��u}�6Q:�<h���>_h��(�����C]�(/l�O]����v�<�^��Cݛ��lm��n�񿒋q��?.���=n��2"�(*H~��/���_�
-endstream
-endobj
-19 0 obj
-<</Type /XObject
-/Subtype /Image
-/Width 16
-/Height 16
-/ColorSpace /DeviceRGB
-/BitsPerComponent 8
-/SMask 28 0 R
-/Filter /FlateDecode
-/Length 553
->> stream
-x�m�KOa��k�
-m��BY���C¶�X���������A�R����K�)��l4Q-��ô4��̥��[?�@�������<�=';�ƒ�9{͆@�!��
-�Ql����9La'�:�mw��e��Ï,!{�A�.�f�vP����1	���g>yU �
-�!��ڢx�Ie���n0�>(���/���n礡��G�Ͽ�&ʇ��sp�E���V?.�b�k�q����p��枘��9gy�Z��ȥ����w�^��T%4��-u�<R��o�\��	|�N�Z)D��񙪌�d�
-�����Qo1`��SșhHc7�5�&��:g�i'�uf��&Z>ݛ��D����+�ܽ�Yj���zL������F��h)�����q��ݚ��=�X]�!��4dﳴ�g/4�Ok8���%����L��]ݞF��8'�/Ј����b��[���Hd(�d�4_����En�9�z���m,�Ys��z���PHt�A8C����[Q,�¸1>9#:���R�/X�
-endstream
-endobj
-23 0 obj
-<</Type /Font
-/Subtype /Type0
-/BaseFont /Verdana
-/Encoding /Identity-H
-/DescendantFonts [29 0 R]
-/ToUnicode 30 0 R
->>
-endobj
-31 0 obj
-<</Type /XObject
-/Subtype /Image
-/Width 91
-/Height 17
-/ColorSpace /DeviceGray
-/BitsPerComponent 8
-/Filter /FlateDecode
-/Length 27
->> stream
-x����V��f&��Q0
-F�H4,U�R
-endstream
-endobj
-32 0 obj
-<</Type /XObject
-/Subtype /Image
-/Width 14
-/Height 16
-/ColorSpace /DeviceGray
-/BitsPerComponent 8
-/Filter /FlateDecode
-/Length 38
->> stream
-x������G��G��G���@1�>�P����h
-endstream
-endobj
-20 0 obj
-<</Type /XObject
-/Subtype /Image
-/Width 91
-/Height 17
-/ColorSpace /DeviceRGB
-/BitsPerComponent 8
-/SMask 31 0 R
-/Filter /FlateDecode
-/Length 768
->> stream
-x��1�A��#ܑ�*^�'8�Z�K���I�v,�؆���k�&���*S)B"b�%�D��Xܭ��?�nn��=�"���f23������JeeGm���@���dm�������O�< WK#�ni�]^������Z6�����t��͗#�߰ҙ{7����;8d=}$����G^���~�_������bn c��������w.<Щ��d�����݋OXK�����������_l���^2�)�8qƛ�����d2�|>/&�b��r���nq�P���G�ί��;�${�h���@i��B��ڞ�2SY8{B�.��0��ڍ �n��Sy��;�Hh��`�j�.d�٬X�E�����p0���R�'K�WBR,�GO��D����;&k�����n_�E-��Դp���@0I€`�VK�0b��9rB���˥!��+'����̖���P.
-K�w�6o�; �+ 	
-a�@�$� VS�&��4,M7Y/A�{hUƶ�-T2��.�Y �A�H*�	@�ǹL�yLB�h4L(�D2W2U�W�(D����nQ�R�Kc�����!`�L��,6��/�	�c�g=�ԥ
-�a|�x@4a ��Ç��KK"�atC�]Ӄ+	XJW���)��~}Ir{���ly <w��s$��E.V���)d���T�~IM?Bj�׭���	g�C�{cф��< <w2��
-,v�c?�����@V:@Vf� r+s�'	���
-endstream
-endobj
-18 0 obj
-<</Type /ExtGState
-/CA 1
-/ca 1
-/LC 0
-/LJ 0
-/LW 0
-/ML 4
-/SA true
-/BM /Normal
->>
-endobj
-21 0 obj
-<</Type /XObject
-/Subtype /Image
-/Width 14
-/Height 16
-/ColorSpace /DeviceRGB
-/BitsPerComponent 8
-/SMask 32 0 R
-/Filter /FlateDecode
-/Length 336
->> stream
-x�}�;O�P�q?�::��ap���	����� ���B���o`��T(�P/VBk}�$Mmo�C���rN"1j�L�'��wG���h4M�!c�x��U����KR�u"Y��>k��/������H��
-O�e���ǒ��,a ����An�o��p�px[x�.*���U�޽{�M|��B�pYR��Lj���-^R3EyD�Mo�M���I'\g�������W
-�RS=�i�h�pfgܖ�@���<����~�F������c֖�$�4��23��9KY2E��k�}wz[H�qb�㡖ҋ�9�3knj��\q8��̼\u�~��
-endstream
-endobj
-28 0 obj
-<</Type /XObject
-/Subtype /Image
-/Width 16
-/Height 16
-/ColorSpace /DeviceGray
-/BitsPerComponent 8
-/Filter /FlateDecode
-/Length 27
->> stream
-x�����P�@��C��GͿf&��
-endstream
-endobj
-16 0 obj
-<<>>
-endobj
-25 0 obj
-<</Type /Font
-/Subtype /Type0
-/BaseFont /Courier#20New
-/Encoding /Identity-H
-/DescendantFonts [33 0 R]
-/ToUnicode 34 0 R
->>
-endobj
-26 0 obj
-<</Type /Font
-/Subtype /Type0
-/BaseFont /Verdana
-/Encoding /Identity-H
-/DescendantFonts [35 0 R]
-/ToUnicode 36 0 R
->>
-endobj
-22 0 obj
-<</Type /Font
-/Subtype /Type0
-/BaseFont /Arial
-/Encoding /Identity-H
-/DescendantFonts [37 0 R]
-/ToUnicode 38 0 R
->>
-endobj
-24 0 obj
-<</Type /Font
-/Subtype /Type0
-/BaseFont /Verdana
-/Encoding /Identity-H
-/DescendantFonts [39 0 R]
-/ToUnicode 40 0 R
->>
-endobj
-41 0 obj
-<</Length1 37624
-/Filter /FlateDecode
-/Length 22041
->> stream
-x��w`TU�8~ιe��I2)3�I��	������C-�P�I�PD�]�/*V\	E$ʮ���ˊ�Ȫ+Ⱥ,����>�ܹ!����}�￝�sϹ�s>��9�;#��h5�PNۂ�mRۏ2B�J�H���9S��D�i7�i^�&�V�p\�=oJ󖪝C(�/i�f�Y���ٵC�x?�L�1cj�9M��{J2�|z����>�p��e�^2�KA(uBS\�ڦ���֜�E��fʒE�&%��=�2����ثF"d��� :v��fܙ�+�%�Dq���W�g���'^}�������%?�"��\'��c2d�����ٝz}��il�"���҂�F�#]+,E.!�(�K�>�	�xzQp���{��w�B��vp��������w�w�vؿ��@�W����N���@	�Y��e9���x<\w���O�l�h;̺CX�\�JtD؊��P���NtḊ}�&���Q�p3z�_��s�����)�������i�gh���;4QX�����*x��P�B�	cxFpvw����o�|��
 ���h&Գ��P+�����$�hy��� z��j��N���Z�up�ʮ�˥�Apl7�;�Qu	m��((��ˡ~�=��M̛�92�6:~6�_+t�0��ƴ��v��?�-��B�uY<�uG���	�I'm��P'���h�A�N��Oq�9�L�p�~��H�)6�u��$�g�/es�,<������6(ˠo�f��3�����(]AY�����ks�t�0�'a�
-'��Mh��S���	Ͽ�{����V��}
-�u{�^O0��ɉn:?8�B��s�4ם�{�3d���š�����I(em�1���Y�P���}OA%
-`�:J3�:(g�˯B1d(�a�
-4Ci����y������1���	`�}|-�mZ`�;#�xn	�F�0Ξ{mQ�QkF�@�t�t���Ԛ�^4���� ЖZS��9�d�$����0g�YJoj�x���#�H�E�O�#j��GiQ�#�Pk~��������Bs=j��#�8���.At������;�K���	��}@eي��h��@�wC���+`u%�Z��/j.y��<dO�u#�]P�b��7R�F�
-�k [XQ�,�\��oܕp�<i��TS���R���txFd���)oF�A�q�z��u���w�w���3ڭ�#��?qjV�ؠF�,��L<��Rc��jм��{t���:��	��T�/�Em�q��f���!"�ˁ��X)?�b��CiG���#�i�st�c����7�F�1PHT��"�Y� ?:y����oG��h>���@M���ū��)w��\E��*�{�'d����������ѐ
-(�ڟ���K�*���h�5P<�#���2XP�����G4��A�A�4��D��P�A��P�FYL�|��/�N�e�￐���d�����h����<�3J��~��sXMu�P�?�|ͣz�y'ȳ��:5.���;�n�҄Ɗ��~a�F��e��J��`�1��(����<����5@w�k�`.��ǂ���I({#�?�������"<f^�v#���Z	�6"�9��)P�z�'�s.�/c}W���������F���W�����o�Ȥ��z��XP����Q��(�UY�֌^)�U}���Eq�Fq�"�"��c�GD�9�� �i{���0[��kk����ym�g�S�<gm@��.a�R��|%^�������s����7�/���+u�*���J��E�d����Z؂;��v�UJQ�����P&bg�a�w�]2a،/1����|(����&�Z5�@�\�������}�On@v�f�0&Նf��\\+����@i!��@�
 �����:R@����_������!ĒB�f6Z�.t��{k��.8�.("�q�1!�ǛV?�c�Q#iuz��d�XmQv�3:&�����x�|�)�i�����>����y��E���(8�T\6��b�ʪ��#F^U3j��j뮮o?abpRcS3�<�e��3f�Κ=g�.Z�䚥˖�Xy����]Ík֮[ӆ�o�u�ͷm��w�ߺ�;�B��{������;z��G{|'����ձ{��}O��@�3��ӟ�;�<z��r���^�ͷ�~}���?��#�ɧ����c_ ^�3}�F"�V�v�	o����M�Hy��J>�0�qZ���qøU���~�-�����2�Z~;��������?��'$-�jSi���_O�{�<O�'Փ������xy�=�=K=힧<<g��7���L���>�II$IL2'ْI���$ReRs���BR)�{JL��);S^Ky;�djk֜������"�>���9y�v6��x�@b`.�a.s貹�s��Ǽ��/�W�\6�[����G�'����
 ����+2��0��o$��y�a.O��82�E�k`.�e."s��\b#siJj�b.�����G�YwC���5��B�=���ހ磾��u�@K�w��I��k;�E+d�;�;�4h���?C�	�گ�:�7�JC�g��Z����N�V�8��mo<��/�_��k_�z�i��
-��x���/}_�h�b���K�m=����:�0�E��k�:�تcײ���f}v�g[?=���ς��>*�(�S�@��lG.g1�T)��-���� �*��ǥ^:�%��<��L.�rn�GJ~S�#n�RX����u�2=�nDk��h+:�֢��ft/z�@�&z��AG��mh=��/�Gtz���΢����h2��6������*z���@o���i�(z���D��itz���C3�w�o�&Ԋf�Yh��h;���6�-D��"�]��EK�r��@ע��it?jG��˯CߣP'.��`�X@Q���J\�B(�E�����q5�G�p�b�c�G�s�g<���p-��W�z܀��	x"�I�7�ft}���b<O�-x*���؄��،-؊m�8�Ga;��[�;q<��s�\<��h�v��]x>^��p<N��x!^���.����؍=؋��b�_���ex9^�WbN�)8_��<[���ףgpN�8�@��Ї��	�}�������^|����~�ށ��G�
 ��؏Ï��	�����Yҁw�=x/އ�������Ɲ��,>������Ç������_���_Ƈ�+�~��_�o�7�[�?~����\|��w�{�}���?Ɵ�O�g�/�s|���/�W�k|�m"�$�d?�"٤	�җ�<n3�'�����-�nr����������݋o�[�E�bn	w��[F?f���Sh?zߎ��}�%�0�=��q���x.!��G�c�q�O�{��<���ɓd�$d7�C��}�n-��[��D���!��<���'���
-s}��RS��(��hx����R*|��<3<|�M��Y�c�+��ކ�,t�{:p���c�1*�	6I����C��	�r��G�.9�}�^�<3;��t�gw֡����M~C���yb}��ڍ`03���1��4�����l=�!�c3�`�+��~���C��y�uؠ���;��Ö���6T������y:����}�K�1��a�ohӆC}���64�^=���6쮮��V���@��;0�?ss\��[:,M3p�2���1��q^+�������&�N����ʾM�ۃ䀿��4�#��#�Zzd�z���&���zL�X_������M�Gz�){�	*����G������;A�{֏��C0�Tְ;��wz�PX/�����x����=Db��u��fGy�������IjFS�Ϣ���>���L�W�{�P����ɉ{2r-{<{�=����Y�g���=��9�Gwhϙ=hMn{*:��]��u�:RS�XK���k=6�3�ɏ��W��V�êr�
 ��J(U�|Ii.?�t ?���)M��J���PJr�ܼ>� �/���$����?���>�w_Je���{�Y|P�������}�J~�޵{aXg��eg\���j���+���G�m�ۖ�=_�G�{����=θBy[4��F��]�6�h^c�h�d��ѽѽ)�q����7ݶy��u�כ�뵖B��"��
-�s�����K�x�x^�_&h2F�-��ܼ���'�l��ϲ��~{1�i��3��mO佞!��^¿��]q��8W	���8/
-�k��x+�6;�탇�M�n$b��n��nݡj��p�������uV��3�n��ڍ��v��B���s��?�u��g:���{�/����b#��0�`),W$r��Nb�_��f�~޼?�y�|n�����K:mo0�]!�Q��n�a�F���:�0�c�vks��-c���zkB�V�܎�	$8��ol萪�F��O?-\���OW�!V�h�}�鎉�@Z�*:̴m���q��bF�Z���B��_9�<�m��_{&�"���b�h�G�r��E�����exi�%�������
-�L'X0o�FA����+���:�;�^�}�cO�`�D������;�I�HϠ�o=�>w�Vy��y��:�	���{8!��"�ce�!9
-��F�BY�M�h�h{�<�B�(�]6����*e����Κ���"�dB�8��ZY��v�M`_=���q-��݆����]G��.B�����>=3z��H��p�B�2آ�/��o��'����Y���"��ک�ȷ�
-����{�'�p�����:������%N��i��I�i��n���%���3�c�33�m�c~�Rj�h����Z���{�����\�\]㍹>6#�^Wכqs�#��x+|���&&�'~���i�j��9�{"�.�|�5ii]�f<���o�rg=��<��}���rZr�s���Ƿ���������������E���4H���?-�x��7٦o��굦��YV�`
-������N�n�c8�5��d�b����\P8���>���B�}f3k\�g4�����z听�jI�YpD��v"�?D����}s0��|y�s=��oG/��]��ϣ�Q��l$����"��f��%\�.��	��N��J�%�����7��"�C"Ch��r�̈́Wanq`�����:�"�M�"j�sE�
-�̉	N�Ow`G��l������b��(�c�ɱ��mz��zk,�Q��3F�poy**�Z��Yc����Ȏ�Y_�'z	�K�����)��@^����}s��O(8�V��Yz�� �r^���E�R�e%�cE�c«�q~�����D7��o�g^7��&3\>=	۸�����C���K����iY��}wx�^LK4�G�[��V9�b�xڲ��&'��9m~���X�,I.�1w�H�/����_0K�Ꙛ1�ߚ��?:A�����&%$����mK+�G�NlOĉ��:L'��w	�z�|O; ��'&F,��h�Y��S�D.Q�#�b�ȉ�`�. @OAD���S��Syykqq��7'�vQc�Y�ം�D?�����yV�w���ʎ�XM�aw���4� ̯��%7~[k���'����7~wt���+M��I������gm�M���^�W��fTx+����6XKx�d�2Z��7����d%��g�6<Z�
-�{���JS�O���tulx�a2�{w_�(�!��Y�(�c�P�_�_!k��pg�+�N��ZM,�դs�B��*�M#n����4�x)Y*�kl��y,��ض�.b�����ͯwZΞ���(K��=�ؗJ�[Q�C5"q�m�Ng4g�f�K/��f��%�U�J�ny���=������w��X��
-�*u{���b�l���Q���S�ԨT�0,sr�0�(nTT��-!m\[��Æy�b�m�<�y�����iC6�A���@��m9�r����
-�IZj*Ѐ�FN��G�=`P���������{a�뺊WV���;w����U�^{`䏐%�|�m��5Q�*k-E���Cc� �ӏ3�"��}�iY!,�/4�h��M�U �A��G�י0l�rGd5�l�4����D8^Ћ�tm�zx1�P�afyTV�򀰱�Ě�(_�����.�K���rx�x��6_�a0n���Yr�F���3>����s?���w�?�M��wQ��BzT,��'0?NzBg�`�l�9F�H�N�j�q��S#�'BP��s�Yb{��@Cy�gB_<��x籄6�o�FI)��*�S]8�I*&���@Z��|�3����W!�@������Z�"��[��	��d���/�%'�َ���P�j�F�(�B#T��@O�Ax9mh3i#}��׆`��g?�������u����h*И>k����l��U�n"��B9��H�N����*T���t|^�u�F���r*a�N���`nb(��5�~����ꦛ���7	?|�19���lb�+p���
 ��Pm��)O48���[����σ�-��F�>��5��
-��E��0��(��Qm�b�!t_�9ŝB�$ziO��cZ=�+�Er�J�P��:�d�,?�-��P�p�,��P���paaH��T_�
-)����B~�����ٟ/`-6���/66ٷ�qҊ�Xg�wYˤ����Mx%ހo�+��]O���;�_5�j���xϻc�s�;�2J��=#��$Llk�Щ�ؖ�1�j9h�Ŵ%Eg�*�d���D n36�&6w��`kMR����8�3P��>.�)<8'�Ga�@a�%ҫ8��;��haK����p��OY8����MIP9襬�-��TX,×t}�ï�nN�S�ߍs��N�w�cG�����0�ۄ3ȃB��Q�&78����d�����T�P�X�*������*w#��m��1vR\0�1�1��k�Z';�%��E�vW{|{b
-L��}t��~_�Bf�9[
-��e�h�E��$m���ãqǒ�H*JԱ��u2��dZ��S�9)�b靜N���s_�9ɝD�wz-�Rt����3���Ha�7�
-� ���rJd�¨dϣ�WJl���o��|o¡�w�4����a��}�yf���Y'�pޣ��5<�Y���w}�?�S��e��V�|y���}+��Wr�^/\zG C��()���&���+�C���p�Р���ZK���(q��/�O���lk6������%f��H�F�p�Χ��GaOa��1,�))�6E�}��Nzf2=����$�/���ў����Rs����d9��������XB
- ��,��*UB���T��Kb*�2iQ�5�ۋ�0SV��8�Nޔ3h���ח�<3�\{rV�Ƭ>�y�٫��z�S}2��5~�H�?��!��w]�����~���K��;�oxfFV��Q3���//�r���H��Y?aeg�I�y8�~/��:d���L�q��m1����k��-a[��l�e[̶�my�ݘl�7�ی�!�B�`��1���
-T���nA�4'��jtt��L4�Y �:���DO�=:�Q�1(���R;��S��a�ڑK���,��R�R��	�1��-��K���5��cB�8�X���P.XT�XB�!?���cS�8Ϛ�b~��BE����SB���A��I��k�j���Y�5�L�\�e�Jb[����@��:�Y-�[�K�2Z��!!�UE�	(������"&��ә��:O��d$yjpuc�,�t�	&
-�&8�*��)��Qڽȓ�E�\�,�%ڻ�
-��}��A�?p���`���B¿M�@�V�*q�kF�U�r��2��K�5�Y����O�p�:4�?޻l��Q���<�:/�����\�Қ���Tw��t�
-/�����'#���JBE耜'��┼�yEU)e}�5�:�(�(�T��&�Q�=<�3�P쮬��rQ�SD�iA������!F2�7��Sl.v�c�������B��C��
-���J���QQ���\*)ȷ&S09|rH�M��������WQ��cg�Q���>�F�q�j�mD}��E�}��nI���>�Jo{�u���!c����8����:�s��?��+�|����l�==ә��邬�ݔb�	A����N����4@�]����:��+q5�'ˀ�1��2i�c������
-�q��L�5�����h!kq��ةu�	����p���I�7�:���OMB�.sY�}��G��E�_ՉRAd�)"�d��SI������w�%���;'��U�rL�7�=.��ߐ��w\޸�&i�i�mrl��)m�i�mE�Z[�HK}8@R���%���J &��\��,�R�E�(�3ԣ�h[�9S=��W!%#%O��x�t�є���\v$�Y�W���|0��N�l
-�5/0�T�n/&8d\��ؐuS����
-�B_�����3Uty|\j����6@��L�4�L}���Ԃ�����1�@!����(F�i�,)M�~��O�/7߰t!��֭Z~��N=�z���Ǥ�\6e�{�@[p��g6mم��|7����W��=��CS_,K:Hͬ���*gf��G�m-�8��35��:������Y����%>���KRJ�}F����S�7Fc�ĵ�)fkBn�=��.&Dde�U��l�u1n0��(V���,�,�,��Y�.����Jټ��#\^�̪�*���_���"E^R�"��&��p������§;�'&�Z~���<����8���*j'�&��.��H��=���C1Ǖ�~�m�J�Xk�뗓�?��'p�!c�w�}0?;m߃�K�ݎ�tG�F��Md1�ى� �	�e���O@ݴ�PI�%���/�T�N�	����sK�^#�"��]�K��b_,V���~����%j�$Q,�k�}�Q��0��ꖲ�#��s
-����)}j�!�G���j�u�S��
-8������)��ƍ ���%�#��B2�<�<*����u�7�zl���q�Dj��p�Spv`,������k�q$�+	�^^L!�O�*l�d(�=�8���,kq�“�f���1{x���'�z�¶���h�L�C�S������ټo%��0�g:LQ���8��`������<i����ߜ)Ǐ��×��L�'l����ص���\�އo�7�u��a@�)��x�f�è��l�
-bjHW#��L����gZZb�,�7HmZ�]�N�-�E$wm���ݢ���m4��hE/И#�s���d��fGP�74������a͝Z�o˘������\6��~��5}���7��N�s@qK�K�7�>;^��y�7��G/|E��݉��`Fe�}(?�#�tZs�)�\j������8w�=ɞ�.uN!Y|���''y�������ʺ�� �5T��N5�t��NM�V:m�"犸��Eŋ�m�U�c�ˎ�";Ϗ+egg�6I�&��;���q�`b˶����<�3��d��Efv;J�(�5
-T�B>��`=���R���偙�x�[劀�h�iL��A�O��$-5ٗīH�
-!=1��}ƒ��VUT�)��FW�+ㆯ���s;������	��ᮇ��nl�3�q
-��[���G�	��`�굋�AO���[[�6��_�eъ!kɶ��]����_c�כӵ��ˆ���4}�����z(�c���Θ�s������N\�x�ڕmm��2b�=��#�(�&�ݢ��}�B]9�K��PP��Ն�J�%�5\���V'�u�Fv��Vǭ�1oY).�._'S�B'S[0G'��t�N�9�\��d6`�s8���Q1��xԊ�}�\�	~�P)�
-S��Fhу��9��%�4"	z��=�� [���t�y�H^���D�Q4�-��[^%�XZ����n�v-%��R�T�,��'�m��aH3Ԓ:G{�h�f�'R��Bu_b�&!_��5u�e-��,s�1l��hŁ�d;�9�gh/l͚5T�B�7��S�{@(�T-�o�ELꟘ����}�!9��H����أ��4ʲkL�V�:h|��4�Uj��[C�{vkh��ݚ6�3d��gA6�	���Au��E�dPĒ��S���H���:����K%�b����7lܫw����>f���۞�MN���F���@Z{ᰜ�&�_�����ǯ�{����p4&տnR��%��/g��JLl�5�5%��0.9��j����C����Y�+`�V�;q���Hq:��'H��z�˄�&���F����QN�+�F23S�E���S�/�U�5/K�1G���@��3�)��؆~r� �(K���:J0ET<��_�o.�f��E�7�=�yr����q����W��a�S�*F��
 ��M�f�dFG�Mhn�)/�I-���W�m������l���aÇM��O)~P�	%�dG���6��0�N �h �%�Ю�b��c�)2Wj����Y?��"�zVNaD��mvܲ���G3HE3������K�yJ�vP���R�D���6�Bs{v�>����xѧ7����˗����3�>h�7������	�@`�E��>�u�"7����þZ�����辧�H��D��!��QD,NW�%IXB�f#�����@k�X��f0�*�
-h|�`@x��Z,����t��˂���v����Ιi�v�gt�>�̦f�E�큑Ӊ8�P����H���S��@��2d�EoփŦ$�)	�Ū�W�=�"��ru�M��쬟)툤a+=tR O�����-;�=-x�;By0��_�'��XU�)A�2yM�*<�4�4U���bZdZoN*�}��3pKt��$�s�%Z�dI�Z�l&��^5���E'㝍x��W�S���/��*�/���^�jN�S����ń�ˡ5(�EK�ѻj�����)��&y��K ��~���&/|D�X�*X���������,����i����xv;��is/)��5�B�s0�Ŧ�*���ع��655cߋx*�w`��b�r�3�υO��?����<�P�0��v��
-���AE�HD�����%*8��kҞ�,tb�����.싨ų�"P>��T˅�\���9��x)p�le�$��9l������;ҥf�D�U`7T"T�e��`�*\���4�ځJKJK'�����<��	�b&=L>	�	GCH%�ݛ�<��ɗ�;�k�5xb���U{�뉟h�\��q�X2��I��:;'��%�s�1z�Tz�F�͒���g�Jsz&w6��r�-��r{/"��[�uP_��о_$#^|���;��s�����YVkd[+^��-��i4�|Nec#�IA��~:�����/�uZ��*G}*�*hg'��D���8�I���؃.{�/��'Wb)��|5g{�>ci��9�8s�e��/�o5t�3������y�{1�n�|s��O#V������ݠ�X�N��cI&�k�q��@_�����x�~Z&Y���#�T�KڽD��:,�s�����F����d���e��0fL��n)��R�O�M��`��e�[��ه-���:��P0����x�X���
 aaq�;��w@��	M'��k����ӆ�#zf	�N@7�'���n�9�1k0�&ȳ	��o:����/��yf�2�H�<�t�9L��^�""���x�<�N���~B�ni��	M�v�Ƶm:}��N7^ϵ����:�iE�ϖ%�ɌG����t�<�@U���8>�����h�I�z����)`;JT��ܨٕ�����\��1�	��3�Mn�('0="0���l-T`�z���Q��\c�j���m��l5��k�JK��!�����<	6@sO����QB[��@Lb�
-c�p�px���Nl·��8Jຶq3/���^�F0׏�0�q?6�	�h�eи5��
-�hM�f�F�D�fLD7v��b�8V����v�&���x\+R�GW��DL4����hT�F6gU�\P �4Օ*�+yt/��(�T�8��)*)������H�"���
-�Fز,���xa|�xv2�"�Wƫ,ϫ�{�X���W�m��q����`ĩV�ګ�?C�:q��Sȿ@S&e�x��>Πh�C��$�Z�-E���X���ja��Q_o��6F���8h�G����l�v����;�7_��Q����Nkc,i�G7F��IU�}��3�����S0]),M���Ȁ��8!�af�bf6��Y_f�pa�7��2�H��3��g��_A1��is(���]�)�W^.'4Oi�x��{����77M���]ۻ�����:�ұcX#����8��y�S�`O�~�>y��P3N��/���?��)>�����X+��L��W'�2�3O5kb�Ƣ!�6&_�i%���%V�V��Y�ԭE�!��M��e=#cu��j�}�ڸ�G��fo���j�D�0`k�Z&׵*ʴ=�e�U�W�үV�54�a����\��������x
-�S�"l��r��	�[+�|w����wk����UX��b�K�,����)�92�XvN���F.HvDž���F .��x�<N�	8���	�J{e�x\gn���3I�n�~^��R�oA�+���j´݊�2��1��
-�葽ËN���F��@�d��@22bd+�F��Ƈ�&bU[��Z��aG����K��s�������o�����2�MI��͙Qr�zbBQWx�w��æ�w�����U�������|��Q��tK�l?��������O��򔤄��[���9�^^���@����e�(��٪ �C5�}��fͶN�u0��΂����'��RzL�;9P,Z�E�3�K���
-����zRW�͞;-n�{��)����n�,�\������&K�D7`X]�|������|Q�Mg�Hg�'��b�\��˪�;ʘ��q�y]Z�J6�q���ޢ\sn[.Ѷ�U�nz��'�D/�B���zu����e�%�7Yצo�ޙ��˚�?���d�dj#�=98i�B'u"�/�q:�]5ꃭ���ט������<�z������ָ�6�����|�2��1?����reɀ�mX8e���g���?�d}_�x�`k�)ݑ�'�3�i��G���7\�ф8�aS r�dֺ�5ZM��Y�v8���c�pB�@c���z�R������~A��,�-�i0g�ʋ�� 5�?��p������f�ta� ��c}6_�@n�v7B{M�5�R��sIĩ3�S��8���|^����VfF0+.�6Nc��D]/Sb���v�e7gf��h�l7���2qˇ3�x�vO�a��T�n`�pc���	�Y�{S�JK7Kӳ��v�5[�2����M��3l��޴�m���f�y���>j���c�}�
 [ѵb�-Ń���>���TW,Dt���?�nE�cd��Cr+mÕ WyA#^-�9��%h����<��9�I���e,�+�����m�*�p��2ǣd�?�F!���Y���s~����r+8�k��5
-�9YY�ti��m
-��Ddi��mx�Y�¤�W��������	@�JH�yNw�Vꇠ�;��`Z��fA#Ҁ��ƒ�D5�M���-�8#��'��h���R�>�ː2�H1���K%�<]}5)�˥�!�j}-�1^�ZӠ��Mŭd*�*�j�R��_(��.Э��1�at�(�a�y7Z�E��x (���(_�F��r�X�pHJM��v/N7ZN��b�c��A�W��%U���~%�~%�in_�E���[������q>�;��B��b؝�����`l�?@����8!`G�zJ����_��k��t.ޡ��}�"�H�LW�5�����Q���3�9�a��I7]��_���Sh�F�젬�(j^�uz�H4�LjE�$���#�V��2�b�R����Կ0"'JE�h�4�(i�X1C,+�FQ��/̥�J���R���Q'FKZ���ȩ_(L�7��p��DN���]�ѐ'd&SCw�o�w�I��U��I����T�^M���+Im�z�^��-2�,>]S����˚Q�|�F�c�41]���b�$I\
 �X� �@��y��xp~�B)�tn]��]ǃ���GX+�ޞL��xj	F��3�osA�m.���"���B[�PŖ*�)�I�y���@�Sx�'Cg�~�o�w��y0��,�B��]0��K��"]���=���1 ً,"��b�"a�cj����q��l�H���:O���x|x��M���54�w������9Lt8g�~�7�x)�	k�q8���D��7z��Sf��1��~GC�1;a{��������F%�|?��,�qh��V%��T�)��.<�Dp�$/�[��� {C�aV��nWь������{�̺�"p�¸��Eb���`i����O����:Qb$�.���-��x_�����6JYl�Ͷ�щ�f���e��-��%���)}`�U���#��~M��������1���`I�%3K=�4{�5�Ei�[�Q�L��D]#jp�6d�nN\(A�DL[O4[��,�6�Q|2�`��"��J ȡՊj���s�����료+b���K�H�>,�Ӈ����
 ���S&Œ]�K�=[c�X���z'}���̮г�=��kz��
- ���e-��TG�:	�!c��~eq�%}Qn"R(�,��Ҕ���_M��bdI�f�%<[�X>fۻs�NÉ;�3���߬+zg�]riٳuߕ�nYt͔�Xڢ�G�n�7;�#%��b�-i)ϙ��}��'�.�GE7�65��>]'P�f�(��rF>)0p�x�I���!{��M��K+�&�]f�8Quzɮ���j�g�d�̈������z�sF5���zF&5�vC�z9����7'��J��)N+�s%�\Yc4(KḞ���L.j]*���GCR��}�Us��fh��Jf2��@������{e������PC8u�/354RL1jS%5V%Wws͐a�Ok�Xa�8X�g��o^���1�V�ZXu�nRt��55٩��=����w�'��=�_hur�� �fv����/�f_���q�)ә���;b�p���T�ӀkM3�T�r��e��K���*�f����k&=��c*���c�o�E3��f�[d]��L��	e	O���/$�	�pZ��il���ga�5I����/��;Y Ȳ"���P8�����c�c��^����
 �n���{���O�(^Ȩt�f������؆��R��;�}��X��g����p9�ĕ��S�*���ԋ3�6�ֆ�U�lⱔX&�����̥��\�59hT�`�!�s���C�!K/�&��@��i��=�� ��-
-���q�Yz�I�R���$���Ebx�X��Dr��{�����Vb/�H�o���]��U�׫/�2Ri��Ƥ�/�X�hc�̚��N��1���FW�?�\�����K��ױ~���^��u㱫F�JI�'.t��\y핷�QI��}�o�W��Ƣ�����9�cI���X�8�8��jl�^j\�h�%n�9~�������f�^�)5�l�2��e��z��*FcY�a�yz�<f0l��0U��s+/�(��(/�`I�RgcX�1,�l6u����]a#��5�+VS�`��<%��ga'��罔1C3�<J��������A�������������#���>�oߺm���M��oN΄�7�q6v�����#�w�Q�/8��?Ӊ,��	p-[�gd�b!=�^l_,�Mȗ:��4H�
-F	��V�Yk�g@�3��-�vU�0��LD��1��.��d����,c��g��5��m�C�m������D}��˅�^��b{nnnn�����汅^���㈰q(���<P֔�9��ݒ�l�{����S3�IڨY����5�{W>0H�^g��!@�>�����F����S����Y�U�3���V<[hu��<�$��u�ˉ�F���ŗC�2KTk5���vh�"��6U�T�d�1�r�l�p��‰6d��\H�a�i�!�\��J����\�g��3��3����J�q1�b�c?İ4�*�R��b���a����a��6l���Q���щ�0�����L)�|)�JIUt��
-��g��s��i�ϗX{�R\C?3���*�����w���1��O���F���H�[>�|�b��#�դ��t}H���9|�R�v�4��q�ir�p:;8�Q*��M����9ϴ6QV��q".L�le�{!�Zn�F8b�%���>�`j7e��H��wLS�\��
-[�fe�+���f�y���N6�6�Q�F���7�[i�Ӏ<������x�"�gA�O�����Y/�~��dN�x�;�z�#�?8�?��#ץff���g0�ެ���ǻi7���}|���Sd��y�b�"���/��z�{]p�5����ғ(�����	�ѫd+�1���#���>��Z�'
-�DS�0B�b8n��#�)��.��\�����Y����,A[d1]E���MPȉeXt+�&��O(x:�g!�R@OO.M�Kӧ����������/��A���/���A?��tB�@�T�.��sP��כe��&G��7�~�W�S�(��HN5�T��T��b`KT���n��
-�Dz��Xf��Rc��'�-'Ų�QY�D�
-%�9��*E��w�(E�n���x���'z�D��s�Ry��~���+��ȣ�7d���+�N�w�!
-G�KG,����������i^�� �v��c�\�N>Ɵi9-6�8����8`����q��d9��],wg�ݡ�;��̝&]}错H���@F��
-�cB�c���1��q<�D�ujWoj��
-j�!��;�	����ʂ4��n2�����0�+������-����b	>^ߥ�~������A�>��nQ�f���"�P{s6װ8�7�@���=�c��T	ХJ�.S������r�ciĶc��䖈�V�5&"#)0�v��
-C�ñ�Г8	�E�&�@G��~�X�_�W����r�͡��8�9x�-'P����O��D�K��� �Iջ"�ڣ��e�%�x^��/��E2z�.���E�!�^mp�4��g�v,�ˈJ��(���>��ٟ�S�	�RtYz� `K�$gcwvN��=*�-[�m,d3	L!�̲͜<�I����?��19��n~b���k �
-�p�%EY�b,e�(i,U����Faf�0�,*\-*�Y���Q��
-��U�q�Tz#3���t,ն��E�d����,�H*/�������bcr�1�ؘ\�x�8�{V�����*�ǫT�c����-��ҐM����n�lYm�,���_r��7ժ�t�����X	t���AI(��N����+�����^aO8�Q�*4�����5��p��y750vMY�g�EVV�M�g���* +��I�����WV�i�m���j'�9��+�D���N��e'*�[�QVWf�Sb��(��r���y�C�H�m���ev�"�z����߰��u��!�J|��,�Q4.�]L��|�g���Rn�vJ�R-�(�t�*�*�j��AE[�d`6���<t�q��
-G�[�@����~Z�S��f%�k�uǖ�΋�mL��X�����Ɩ�lK6�[�Ȱ(2�Md��d����os�D���b���`�2��1خ��5�<����K�����y��:c�����T�;}�o8�Ƿ����wl�I����K���/����>y���hd;��o��_���K���!��Q橳M���V��Ԩ��(�Z��?��Ex[�4�1�D�#Hy�Ҫ���o��u�_b��/�����ܽ�ۆ7�ݓ����+������:���8WF��V����[��~���/_ذi��Q��� %뿼j����p��ť���?�vdhqh]��}%~���'5�7Е��3��v��Z�X���cCb���ڕ@+��ظRD�{��ߍ�^��UV���Z%F��k�H-{�NYd������>I���(�/��aZ��w���W?��ʩ��e{������q�C|������WuXȦ͟WW�JO��z�,����K��X��p�l�lGF
 �`��A�љ�����3rJd��mvb��M2�h�4�	�_c�K��������4�RY��&S���ԵF�����(�>�7�J�%�+�r�P��]fM��T<ض��oV�Y�9��|��Ղ�����������"��9Hl\�639W[�\��<I:�z��r�	��M����\�r�ݺȳ,}�^o\�Z��;|W��b2�DnuN�)�$%�Rb�2sJ5� N�5Q�H���`������� ��h3#������a��1L���bA8��.&uuΤ2�I�-��w4�!o���j.WO�x$����=B�*��i��6FV{�S>�P�q1��gK��b#������'f��&�����z��T��Yn�V2����G��V�=T��i��[RSg�_Wp������8���r]�[S�o�gNʿ�ða��Η��R3��,))FcJ�6�byN����Ղ���*|>�!F�X5�r̠��IH�šW�q�%�No�2�uB\\t)��IhO &tK�d�0Z�X"-��-���Y�-
 .���At�SX٫,��)�M4=l��g1fv��b�h����/����I�r.7���S�JY����<���{)).��
-��sO{�A|cx��Yӧũәl^'c�Ã�������S���`c]�ĉ���z1V�\����e�K����?�+�����_X�$"�Wͬs�_cGGq����;�r�#v�8Yc;��p������'�q>���Q-r�]���>��s�#P��?%jӢ����C+�"ZG�	��j�V�R�Ÿ��Z!��%>������ڻ�73o޼��cgf��F��-��>�����K�j��G-͊�Ws�bQ����>�U$���;0�|��KZ��>�wxc�^���l'��f}���`C��������CG��𛅧G�]�:���x�����B���mn����v3�;��}	!����Y���b��U�.�e�S�.E�c�iI���TXig��m����謠z����
-���}+ڪ�U���!2KDr��*2V�L�c"[�[$�T���g⸻ǣ�R~E�2��B�+�^��=E��}�]q��y��Ҏ��
-ځv��=�"/3WT��/T$ϸ��N��w��Ɲ�8�aMl�(t�6q�Ԫv�{h�4@����qL:�O
-��>��3�<}L�EG}D��f��*�֔Y����h��GJv��7~^Xf���X_��#2��� �h}-��,ջ��=�m}ڂݮgZ�Ԩ��]�itѰ�4[-o-|t��I��\�w~I�M׽�5��os-_a1�����	д��~�P4I��?���^BS��i�Q�v֮� �����=r�ү��$Ct���2��#�4���F�iz�Y�i%�M�[V	D�-t�G�����������P����DI��mWg�W�0��J
-*������Pe`�"����V>X)ʂHE|x�O���-h5���#����cs��1B�'n���g����\�}a�������x{�3o�
-o�
-���q���~ܼ.�wE�p�ۧ��Îʧ�,kL�ְ:i?d��4�4A�ULj�8H��a���)J���X�o��EJ䢶J�L�����E>�?u���J���Zb��R�6������GO�S����+���;�c����H�&H��ď�%g~6jt�j�>֫`�ԯ��i�˨>Ea��%�.��*E���������s~Ix���|�3�丏�ҫ���^n�WK���۳…��Xb��\`�/}�6_��+<,��5���F�7�)�іDK�E�k:�$4�Վ��[�hͱ*�z��kV��O�}}�hQ�J��Hd�Žpx�q� Q~���vҞ]�
-��~ZC�&>�]GᾺ��c�^���W"�W^���K�3�\~��ˏ<c~w�ū7�s����>���]���~���=�8?Oj��Ё�ڍ0�����(���G�h�[���׮q�M\�(��l��6WE�h�l�\]E��n;�&��i���R�1h�D����*|��^�4rD��a�7Do���\�C���Zǹg�ϧ��?�յ|&�અ~���CՀ�W�)1�|U�Dȥh���G��Tg��k�yS�%xwͦՑ�Hkdc$4P��u`�@�D���O����Qsf�D�CMFk]۹zZ�ic����u2!Usr0�G�H$<ܸ�-�U�9�\{��Ycl�QM�+?�)�O=C��1�V8rT��'|��->�ee��x^�g5��VS%V�
-���t�/��`�p�7�V��(أW�,|N+�0���֫c/\��7��������0��?����?}B��?������\���/Z��P��[H�c����o�`�Q�p$RANy�\��~b{�Fd΃+�A�S���¿�,2ȫB،�pP؉���V0|aaa(Bރ)	y��=��?=X,ÑH=�ǃ�p��e������\)|��Ń�|a_�~Λ�!�e��.?AΛ����R$�k��k9���e�kxZ��F��E��g}܊�����V�xV��W��Wx�_շtoެ��v:��t�i;��c��N��=������r����5���5m#�����t�.%���u/���@D�uG�^g���1ݱc����g���L�e�$�t.Ã��d&���l�P��}o�vL{,�ӓ���˚��M�S����NZ�c�D��9�bm�d촑�;���g���TYZ�[��t΀�NZ7�lf2��HeB�̔ҋy�S�y������O�H*UD^�#D7�TB�ͬc[q���!y�V?2�
 �aA.���c[����IM�c��1�S��A�4dn����0�Zq��L�әeM9N�/���	%�%Ň�|&��c��|�g���$�?Ibd�	����������qlj�1����.������d?a���D�.��l�K'��E‗&Y�OBZ��Ejtcb�"!����>a	2qY�p7�,�`���@>b� 9H�9�3Ǿe������ljNz!�npuұ��;�Y�'��ɀ�$�>a<���Е5�@tRĎ�=	~���� %זKg��sp��1L�c�y��Pcn�.w�(��|r�%!�A�M`�"���9e⦵�b2ȯ�đ��Z����ஜ���q�|�d n\�3X&y�2��</ˣ�h����\n?�P��;/��RN+q�������E�RJ@���� ��R�^Yv7�;��/��ĕ����-��we5 d%Oc�[YRWϱ%:5�\Ӟ�J�����tu��,Jc��p�i������Pshca�f�
-�F����W���Q�ʘ
-y-J�%�=����b撾
-{�%�)�׉��!�~q'�ۗPJA�Àw5�{�=g�FRH��+��
-endstream
-endobj
-42 0 obj
-<</Length1 65112
-/Filter /FlateDecode
-/Length 30956
->> stream
-x��wx���|f޺%ْ�f7�f��l6	����wBO�@�   �A�@����"
-��P��E�{,(V��%�~g�݄��߽��|}Ovrf���~&dQPϼm���;���X�n3}|�ә�.W���c��4�2H[��6c���<`h	@�S��ϙ�z%�1��g�I�ƏQ?C��ل�=�x�{_�S�~!o�;&�+r?po"���'̜8��1�5@� �����8�D�ǹ���V��>f椴O�ğp/�کOg��~$���ӵS�������߲����MHL�J(���MV��>�{�a�K�HI�^�_������D����~3:$���Eگ8֢�".�	KA
-zȄ!��0x��i��b�����\Dh ��2�/`�C-9_�%��/‡p��=��Đӈ�0�'1�`,�G��#��⌽P�5d�$8�PGa-�sx�����$��2��8c���9��CpO�`=�U`�۰F@h��>W�4�܏c���l�A�����Sm$��ӈH�#�w�Z2C9��r��>F<�t\i,܏0��y�$|NR�
-�mf�W�;|�{��e��b���4	��C?���%�ċ�lœ�C�K0��h��k�I?\���J{�4II��TK�q�|Ҟ9@jI�>bo(�y1�\���zX��?���z��v2���p��B\��^���`Q�=�{�S�|���+���!��`b��1`�E�0�Xd��P`1��p��a�����D�X�J6�a7�j3��6���Fm,C�,�^6Z��]��?tbc�.� �}(%��j�-)��v��s��*،�:�}�L&�����F�5b)���M0yw*tD<k��C�Y'W������a|.h�e#x��M?R�7"��L�J��؏��+��Z��*##2Cu�>E��!-�>��H�x��EJKLF7�9nA�9�g�;� {��X��*r�����
-RrJ�^�+��u<�0�aW�KR��a��ɋ1U�x/����;���$tn�L�p�O>�b<E�r�RHEm�+���m��C�݉|5K>��4�����#0M��?߳/t'�޸�.�<�+�>����s�Av���|�1wބ�
-&Q/�Ez���C�Ŵ�l�Zo�y�՛h>�
-��7��!��8� �HO��$2iuN��X������>���%xބH�e�{�����?���I�;��%5�<5��u�)k���#�"���iڙ�&��C^!��5�BEΑ�Α�^'g�?�-�ٮ��d0iCd"�dxGE{�wȯ$�$R����N9J(G#;�^2�Ķmd,)G��*C4 *#�x�Y��g��>jL�5�O��O8��Lx�������<�S�uoC:������<�6���P���b�!����8�C9��,�y?�*���]�"���#d�0n�ʛ?�~"%�G�6+q����R��J�����T+�����W<+���(�nW�'�����6�$Ԋ�F��1�1ry#
-9�q�D8@@:�����R�e�l�m�2�����=�&Wɗ�G�!֪Pn��G.`˗����
-b���#�ނ�d
-��'<������"����^�-�?��2L�c:N���ױ݄�)�6��t�T��'���[���)�M<�f��S�r��KȹG�%�BF�.�BxU���<G� /*2�WR��BM�b�����	G#4����ێ��K����h��+�l9��8��;��o�L�C@]��لzt�S1������SQ�2{�	όk!?�&#HrS%�S��qb#7�$E�k����_��/�!���$���f��/���J&э `b�F����(��)���ߔ��⿕M�D��^�JK�W���w�C)�h��Ú���ÉY�&JЪ�$;���	�4�&���2��I�`���V�	���Th�:j�j{Z�m�l��k�^{��Q#M�3��(�yH�Ub�2"�@�G`^�{
-�{�?f-p%�6ܶ�X��/��j�Xle�]>J��T<�S�;m@����|���<�G�t��S�s �1��0#
-��#��J�ԸJ.�l�}N�s�xV#���>5��y�7랰����eШ�����>�r�Z9q�*L�Sn�?L�L�.M���`��s�?	V`Z�i/�ıC�M��Зd�1�*������~he��l%�E���>Loc���;x:� =XLX�OW02[���!��]��]��[ѳ�Pzԑ4����I#��$@�0���}#��H=���o�(p,�yhQ�c��v�8�ʈMJ�';0��&}I)�%A|���s��X�V��Ӂ����l������Z����{W6��Z򾲧�����x�Y�p�����'{Hy��wϙ���lr��bؾM!�F(��H1�8ҎpH�w���9�;z���Z��f�^�t؎)��j�ʌra^���>��ȋJ̾��R;���Ÿ�;)����A���)���".Ԁ�hWXyJ�#
-�S<R�A��A��L��l�b\�b9�F�(�n$���BJ�:�rš��"r5��0�o��E���Ĭ������&{��xbj$����MRĤ��Z>e��E�۟A�D,A�+���Jך����7bR=��Z�W��'<�*��9�@j�7��`~�	�'|&2�,B:b|������6�3��0J���h�)�3�-uXX��e���_y"�Qo^�jo�0´O#4�3``A�h��;��w	�ᠢ+M���=���x�p��Cs�#�0h����€i��{
-Cp~[l[��:����J%�f�y����y4�\B��ꛮ�߮o�؜m�N�N)��y��L	(c7&}�i����ή��B*�^e�aJ\>����{��e�f�I��(�y���fk�H����)��M
-v��%�7��pO^���q��L�hP��4;O��4��f��t|v��C���bw�>�rs��q��p����Hb�'����=Fl��FG�a\��/r"�E`i<K��ۨ����a/r��x��v�P<���XWpV�m§��gC��9|�ѷ��Q�)���į�8x~FOɈ���#n��O)��]h@�����&uy
-�����|�������,�z��h�Q��aki�Z�S�s�_��l w��2������7Ƶ����z+	-�}�$�;�x"?⨰i�H3��owLEU�侍8�ϭB:$�8ʭxY,mB�=��l��0�B�*N#/|����*�h�G�ʛ����=����x���?<�FO��h��=�9��[�ޕ"�_�~Ċ>'��y�o(�a�)�k�%(�xyi<��4Y�4h�kK��������a܊܂�CO��f���Ƣw���;����nB�1�����==HF뿑YJ�!�g���N+���/i���b�
-7i\)���t�f#����c�#u�N�Ẽ��kL��U�1�ƮQ�3;���'x^@����3Y���g�oR���̊1�Ŭ@8����L.`䑯xmh�*���-:�O&�.����x�bU(Ve.Z�I�s��9����SR~�3�-��W��3�b��	��AB
-.B?ҁZ���97ן����q��j�Q�-�w>��/�6��pU��X1A����QbH�n�b�j{���-��8遤�I�$���=�n�w������G�����������_�78t�X���sd9r�����.�Q�Ŏ��?9g���t9}�g?�`�(�=��O��Kt�\FW�+�ew�����1��n�ֻ�(nԫ��&�ś��xӽ9�|�4o���J�j�F�ާ���Z�q�)�޷���~���}�|�q�	�������e�=�|r�5z-�Z���׊�u�v�ڷ�Bu���:c]�.u��J���M��Y7�nm�����v�=U���麪��>���>���~e�/�u�P�.B28`�	'mK��g����+��p�{��q�sg���|�/��?�xj�I�S��R���l�lI�T��b{G�/v�[�{{qۣ��#����?��b�ݡuq
-�[;�5��:ǎ����9�9���	�ĶՕ�v��ێ��vq��ywx�4a�u��Lj��M��.O��؎{r�5r�v�-b;x��n�޻VW'+�nUWXׯ��nb{z���lok��k��O��l;*��}	�_e������*�p���	kh}����Խ!HX�{�]��C�O\���Wh�/�������K./������w]�����_�wy��9�o�<��;\������s��ϟ�0�/<����\�����秜��B�w�_�/�w���O��?{�3�z>��f���l�g��)��gٟy�u;��\��v���e��:�z�u.��9�����~s���/ج�/�=q���a�}��g���z��٢�����γI-���3��m�Vi��9��R��R$�� �&^��/���/�/�/���cb�xT< n��=Ŏ��š�6ځ��4�W�Ĩ��H�_����v
-Oc~D@��Y�q���;�]��%r����Gs�h-u�0(���_?��~JY~S���:���:≪G����f�5��93���G�M=���ԕ�r���mʹ����q���1�
-��e_�n؉v�I�6bl�3��xv%����G�<����\���ix^�}h!ǡE�}���
-�o��&���yZ��<���Go�=�8��wp/L��h����v��(f���Vi܎q�7h�`4}'�pw���(ZKv'�.��p���E/�'\�:�!7�����g���!D���цn���Q���E�v�1�8��";1��E�$O��dF�O�}���T�O]M��IT�U�9L��g1��"��(�ED�^�.`�~�1��c��]������$/���a��PE�腽HN���Hl蛼D^�?��h{�$v�\��E^!�������&Z��C�/�%o�3��.y�����v��y.�ĵ�:q��A�(�/> >(>$n7�[P���.�q;�w�������;�'�]��S�nq��W|���O��ψ��*�X-���#�b�x���OCi=�R����xB<)� �������+��k�����i�-�m�����������o�_�ףL�@�
-��� 
-� *�C�#�c��xN���y�����R�(^���A�qY�N�����O���/��C��������R�d��$�d��)Q�II�CrJ.�-y$�䓒�w���ԴմӴ�t��k
-45����H�_[��������-іj�kGhG�K��P��K)�Z��Vߥ^(-��K+��ҽR��JZ-����J���i�t����I�,m�ἴ>F���]�!=*=&=.픞�vIOJOI���	����_�T�#핞��I�H��*�T-T/R/VW����V/Uߣ^�^�^�^��W]�^�^�^���_�o��wH��#ҳR�t��-�JǤ��s���	����E���������tZzKz[:#�#��߯^�^�^�ޠި�_���AM'MgMMWM7�C�M��RH�ʜ�˂,ʒ,�*Y-kd�%G�m�v�v4$��O�gr��B~%W���_��'�7�k��ԓ���"ǔR��T�"��LUTMTC�mGS�S5�j"4��I&ɢq�B�4�ݱSM�v�9W��˜�5u�l�^��?��4�WoQo��hr5y�����^r;�N4�f�,ڊ���4����Fn/w���.��.��i]B�K�=t��O��r��Q.��r��I�,w������r���K�-�������r�<@(��C�6C�T*�G�
 �e�(y�\.������[�[���y�<I�,O�����Jz�-����^0�3䙂�f�+ϒo�g�s�'ϕo�����;���]�By�`���%��M^,W�K仅$�.8��T�G^&/�W�+�{�V��W�k����:y��A�(�/?�-��7ɛ�-�Vy��������vy��������S~B�%?)?%���{���������*��\-�ɇ�#�r�����|T�ՎюՎ�ޢ�� �������'���?��S�K���+��k��ډ�I���)���i�-�m�������������������O�����|N�T�L>/_�?�����/ʗ���o�z]K���t�H��5��Ct�L�Эt}�>B���j�k�54���F���Dkt�Ơ1
-)�A��#��[�-�Vp�rk���F�n3�0F4Op����>n?w�;���^�^�^�Nӟ�w��r�r�s��o�+܏���g���^��������I\�&Fc��jk�h�j�iJ4��O�oz���z�@C��rǣ�/�?�<'r6!]��$!Kh-�yB[!_
-]��]�#��a�P.��%��i­�L�6a�p�0_X �%,*���{���J�������ƒ���1a��GxF����o�
-'�p���&��s����s�����pU�C���eQ�}'�E�h��D�5�K�>1ELӹ\1C�s�<���Ŏb���Nbg���U�&v{�?�K�-�������J,��A�`q�8T&�h�4�F�pjN�i��K�2q�8^���iFhFk�i&j�jfhfk�i��,�,�,�,�ܫY�Y�y@�Y��Q��ݚ}��Ú���4/h^Լ�i��M;�|J>#�1���|����M5[5G5Wu�j�j���՝��TU�T�U�Kr������������S�PM+I�#�"�1�?�I��1�_A֐�4q�x��~#_������R�����q�R\%�׈�i��)�Tml����l ���a<��l �<�!w�gi�i�k��.�ީ�K�P�H�X[�]��[�T���q�N��{�˴˵+�+��j+���{�{�Ok�i�Ѯ֮�����>�}J�[�'�З5v�C�Ը4n�G���4���*}��Nߠo���-�6=Cߡ����g�<�@?�_�/�Ez	C��9���X$t:sv��99r�a�
 �09�@�("���	��w�	��#X/�C�;&�C�#��!�	���\2�¥ri�ϓ�)�T��y��s���@����t���\���2�,�ך��r�O/	_	_#�~*|&�GN%ȫ�(�rj���j��w�e��3�7�����C��"�����1] W;��S��ۈm�v�@~���q�rW���$n27���M�s�r3���,�6n6���F��芦��#�?���(�*�F���1�Xs����hK�;�.���KNIM�22�Z�����kӶ]���E�:w�ڭ{���z��ۯ񀁃:��t���e�F�����n?a��S�N�~댙�n�=g����߱�λ.Z\����,[�b录�V��o��6����mڼ�=����>���'v=���={���=���@��C��<[s�����?q�x��K/���k������>����|����>��|˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝Q˝эwF����Ž�ڷk�&7'�u��̌@�?
 -5%����]N�=ɖ�o�ęcM1F�^�ըU�$
-<G	�wuw+wT�ʫx��G�{v���1�ʫ����1U�re��ƑA9ᦑ���`�H�w�C~ ����:���!��`}Mw���R���)�(�;�8���2�����;�Vu�}Re��.�������xu �5X�`�*�=���H�
-����9
-U��ҵ����NP�y�����x@I�.	Ngi ��t�[�NU:�2:+�T���$e�d�6��q �d��=�-�koq�2fdI7���a��]��\�\�ō�KV4�M�*�Z&;�ce�
-GՎ%�{�,/-�5p.�v+��[�F$�����Ғ*��t�7ao~���|��J���T9�I_Y�pV����.@|WG�����0�]:�K�T��5���H?�7�{ Z�h��W�7�)5e8��؄Y�N��Q�������ڲl|[���᧔ପ[�"��T��+��Y;�_%x�nG�o���|c˘H���������հ��^��W��1�:#M����@��5t�{�ށ���cJ�g"��NF�U5A�UJ���P�Li-g='{b����ƞ���n��C�mol��k����1]'��"���{|��� w��K]+�#��=���pۦ�H�*�s	�@#5��)�Ȕ#���m��Qa�[j$�Ri!�nU���T�t���jB?�YJq}Z�U��7>w�����i+9<0��T���5Pee7��[ey嘚P�X�C�<��8_�̮��	ծJ�궺_bi��J��7Y9�@��4����rpI5���;��`_�Q�\��6��'{��9��֕p4P���J��<����
 �&7�WC�mz�?�Z:ɝ������R����`�&J)�UمE��I����mFc�8��BֺV����*��gXK-��bK-��bK!W�{�;R��ևZ=�,��B�r�U�ĵGE�ёr-�iX���k�U��">���w�Vݽ�J�M�R��ز� �؋��6<�6<�6<�6<Տ�\u+�o���ؾUi�
-DYʙY*R�V�3GZ�R��J����(��ø�խ�'�ʹ!��~%���|���V��J�X�]��g(�J�P�F�,�l�ە\�rn 7R�e�K)����Ų?>���S)�rݕ��[���X�┿��z�s,{�3+�sݪ�س�f��h죸k�g�g�Hb-kv �WZFc��mNI�.�:c*�pF�bO8.��SG�#���̃\���8*w�G\����H�|$O>H\>�.����������\�C:�夫���#R��*��Q\]U�d��!(F(G��PAUF]�	DZ���F#,F؎�A��pOPCi!ן��x��ԃ����2;/\&�¥6����6.є
-�8<r*9_��Ɏ@�u�����“Ɉ�d|�d�����q?"�8d�d\��1�2ێ��l֚�-)���sRpl
-��ǜ(3X1�Z��>���.�9]��O��y�R�an�\�T��A�����6����I� 6� ��0�L�3��02b-�~cգ�K��Oͼ�΅ɉɁ	)�%!��aZ��>Lk0�ƴ
-�a��?᧣sg�.�]��=w�\������`6�%4��"=�a$D�+�>%�MɃJ�uqdԫ#�6��z`dT�Ȩ~#�����UC���Qg�Q��QC�Qy��\T�?*�Ud �dD��J�I�[+�K�mdXu������I�!��%gO��K�52w��F����=�9ўn���9W�!�i��?�.�&���R;)C
-H)R���I6�z9Z��jY�E�����&t!�g�6�zV�<�y��g1��ژ�?�)��7�=��]ur���}����Ѧ
-�N���z�d�j��]#�V����R�(9@�}��TEW��\RCB�iYs_�!���$D��R6��O֬)�텖BcGC�n]�"+���������V�`�A%U{l�U�Y%d+퍘c��Qږ�u�r��aEi�Qum�u kWWt)�>���(8Y�����%�6l���qIʸ��(pv�r��lS��)�q���LT�L����c���H���qJ�cL��0���c�as|'���!G���@�,T(ww�P^���I�����Q�L>�D���&�r���{|����.���g������,�:������.ս�����t)=�}Lھ���q�ic�b�1l�4�W�}ѽ�uwg{�c{�c{uvW�R��R�N��*�A�Q#�'8K;��3;*���iY�P�y
-4�k1�B`]��@�B)c]�,"�tYup&Ԓ�"]zl6�;����.�3{v��?�̞={Ψ٣f�R��=g.#�#�9�oP�U��1�ͫV+:��=�t(4�=�jsXv}��\\��n�0���?���=��(6pn�mf���e�2�
-�5�H�2��B�#�EâpC}(D�o�G ���%L��K��W�&�!l�&o�n�����o�ˇ�0>�!����	�Ï��`R�A���'�o�jﱿC������c�����!���!���Bj|>Hm4g�7��rz(+�9ɿ��|�!��	W������U���mW9[���V��8k��\�OP��iRJ�н�w����g��G�*G�n ��6�Qxކ��!DGRH�ݜ��N�z�Ɔf@W��P��6�%Et87���}T�eÅP�=n��p'�U��#�'�%U��t��@�Ʋ���G���y"�Ҟ�r�4����O���!1�C��z؊8}��)8��?+�iE�!#�B���G�'O���3�;*Џ9�[¿���aH�ڍ�&@"8��MG�Az����~i$��w���s���74d����^
-}nHƱ��v��0O},�c�2�=o�W�/�G�Ĉ�p7H���oZ$�Ԍ�kK��j�>��N���g�5�6T7��
-�U�^���7��(��(b�b�q���"|��"��Y{�����p���I����4���:�5��oj��0�aS��PN�/��N�r0�Gnb�7[�{�Ǖ�l8���!�@,$�d��d()!�d�Af�Y�Nrbu79D���Y���"�E<��8z7�H�S�Cz�n�0��;���!������,�/_���/�%��uqu����o��!��K�ԆU/4|��EH:���h��&������oG�؃g����� .8�"�xb�B��x�x�a�2M�4�LA�W����W~O�y��KΑ1x���:����н����o�O�ӕ[Վ\9��
-n%��C�9�O�X�?�_̿"p�-ƒ�V�w�E�8"�#�k�Oޤ/��i����Lߥ�d!�F��6��f�x��v��7:�\>L�V�):�	��/!�f��>Nsط5��t9-�]�8\�=��n�N�t4����w$a|�4��EPD:"�ރYH���g�����	�iTh��@�wQʽN��+���[�}��g=��eO��"�E��-�[M{ѳ�66����4z�<�ti��x)&۸V���Bl��)�~pљԅ�<~%w�X��kH�<E�����~�iY�|:V�JH'��$�I�C�=_g�O���
-9������:���6�\�ϑ���./���!״�b��T��n�����4�L6qߒ'h���lڍ<��_�e#�jQ�t�� �6>)�5tT�;�I�y�nV��㮆JCΆ�Bt�9X����m�R����(2���|(4�����P�'�B	k8L�'� �B29|�nQ~����߅��j����?К�D���x����g2ڈ,h��v�j���WCQ�������,Լ��p-To��(�7�`�l�Pw�"���u���ޡ{�v�qWҗ��t2|�p�pA2����� ���ܹRɎ�V����R!�J)�}�Ї������v�o�;�wbgH���w>����?�`,�<n1����t3��J���0D���@��u0��y*���x#ĥX=�;sl��G]ǃ�:o|�/\�Ԓ0�7���<���
-�Z���0d�{@�/m��[���|\��.��
-� z*��tA��>�'����s.�з8`�9�@�p��U��P��1��@�@�@�@�@�@�@�@�@�@�@�@�@�@�@�@�@�@�@�@�@�@�@�@��(�; ~���UV�nD��hQ�*�E+G8�ƫ$�v��U����O5�o}~?���}���P�_�ϠUV��i�:Ή<�9��uA�=��$P��Or/��*�'u��h��.��Z��T�t'h��A��p����G�`�%f����2�j���Y�����f0�Ϥ,~��eW���/�r�
-�#_��gk�E��(�]���ܫ��Z"��T-���!�g���W��r,��7��;�™`vqLq�v��������������Yl���3���H��(����UF[b̈^5��ծ\���R���u$.*�H;.I;H�����F��hS8�m�69=rz��#����`ڃ9Ơ+�4�NJ.w��khq0�7cDn`D�wDOψ^��rz��"�rbF��YvXh�b�:�g��Gjp)}̺�1\�q�3�A<`x�Z����J��
-֮^��p�v�|�T�^�kl�N�����g#���G��J[�$H�[�cM���(���͍i���A
-���eg_���eǙc%�Y-<���9��Ѥ���'�4�`�:�wP��h%�����q�=>���3���2*�8�ކ_��y�^l�#.&&�!=����u��5�Nd%�f��}C��5e�ȳCE�7�L�[�\Z}a/w/�|��p6^��u�u�×ȑ�9@J���׷O�^Rρ�5�궟w�!#�[�N+��%���5�&Q(#@�ɈC�	��y��BF �kp��`�!Q�[OL��o�г 9~hN�tF�k�5F�L�T�>Gg�x����-�Pj�Qj�^��"�������y�,���L�r���׎A���w���z��}��� 8�1���$�S1K�$��1��]����|}����ӷGJ� �����(�M�����}�һJ3�w�w��Cݓz��>����`CH
-}�V��¬2����fs\^�fR���%ƚ��|�������%��p�M�3cwv��eN2�6ځ�KI>�b,�����1kcQ+ҡmq��'Y��ɒ9�h��1F���}���i��ѻ�N��)�ߦ���%>iaR|�5ӗwϠA����Cn�g��/���]|�Τ�(�=w����V_~��i{�KQ�P��翆(&�j��T���Q͢(���1�c��*U̎h���5��Z" Q�˘V.�[��"���|��];�2E�r��6
-�L�HcM�8E�\�>�e�{���F�t�Q�JVc���T��2�P2Э����jv>�P�Vm�j\�Wh���#�MUQU��Z�yvb�Q�3�O�V��:��y�m;kI������{���Ӛ(����3��P�P�|F3]����yvC�a�Kk�ӾFz�|��xOkѸ��i�*q_�i$'�Ԕ�w���#t����0q�����K�G�3��fܯƫ	?y��4)jS�lr�Ӓuj��B ��zL^�ǃ���$&���=1FSL�Q%���d4^�;Q����A%D�0�<xk8.h4H����o����~t������F��c�?SԐ���ʌ�b�x帥�x"�V�]�o�_��诔]a5���ۡ�2����_��^�?�":�����c����
-%‚O��P��!,yo��DIh��FJ���"d�m-1F1Ĩ�C|Ï��V�9v���X�հ�������pk���E�祢ՠ3˥��E��o��Y�Qq��cu����R�����W�j�$r�@��������
-2�^���:s���'����?O����Q���F���?�X��%I��0}��p���.)���-Іe?�o۾d�K��7��L��ݵ�_�]�ȩE*����Br�_`F���Hb�*Z����?@���`3�B�ݩBb�=�e�.'%!�$Q.w����^ !*���7���D[��U�v��������V���.,3������b�����V�Ul	I	�^L���R}i>^�h�Z�V�JZA�|.�'��� �� �� q�A�`�̧MB�L�bE_��ǿ�F>�m�O�%�XCR��Дd�+4�̜�d,tՄ��XI6%0K�cf�a]�fY���5�8�㒌�€33��LV'[��`Vt�8;�e/�j��c��_|;v)��Ka��}�^Q�h��RN�nP�"ff�[s�o����ki���.k���H�7�fMi�}͎�~KJ��wгg~~���Cg�!zVwm��x�����p���yo���<�8x/:�!��#�>����	P�GVkM.���>,u	G��>�'xrz�D/O^��<
 uWʮ�c�Ci�(�ڜ�m�Ƨ�Ӓ����֤a���l�b����̧�2���A �%�@����*� �P<FR������!e�c�.h�QG��sC��(
-}��ak����N�7������2���ʕ�|}�E���/0�˴������L�{����uZ��6I��$Ff�Z��"��,ig��w�z�t�h�ѸKbD��E4�(�TO�1MS�>�_f�?s�{��J�7�M>:��/|#no��v��9~s������L۵��"���Y;G}R�~��9�e�ԇ�Bh!l�w�Q�k4�)��fk�����Y��Wy��|���H9�J|�݁T|��x���ļ:�F!F��%2ĚA�m�X��.�=�f��.� r�j��$#��
-��x>V�R�.b�<J;Aɟ���Ỵ��G���gz_���c�Y����
-�8�I�������U�,��%نF]�XqF�m��M8��=���nL%wS�WY�pz�02z�\�&z?���Xn1�<���w"o�
-}��\:|tiT�.U���E��I�zk��׬g-�Yd���\�����[6��=��mNx�e��ظT�
-x^>oL�2��+P��%"��e�J�Ƒ����zk�˫���%7��#��J}�`����+��^	�v��T��bIT��A�
-��$V9.��(rh[,�fC#�1ϩ�)	;�nW�A�����ʂG>��м[�}�!��'wU,]�B7�cJ~C�x���ï���m�4��5�=y߾�z��g#~���k��x�Y�b|��:�?h�x��󏧂&��$D�,����y�fc�v���n�x�azB]����V��<᤭L�ՉЧ�Y��Чh��8d�SR<�--�U��7t�:��d���O	���T�V#d�Q��Ϡ>5լ�p��������\	j����T��7�/v�)qE�̯����h�� +�*�n���$�#�&�C���'K��݀I��, g���Fiɒ%a����͂Y�����C�D��/��A46�f��,�$X�FX�X�6��0ւ�cY[,k�em7��Rb6GB���6���x�mr±c��fu.f��;v\�:edZ�|����Q�Ν�����m�{����}���,�O�dNn�c+�~�rlq�o�m�6e�ƍҌ�;o����PQǠ��!_x!��̧I~�����>�ɓ�N�,ڂYTtR�������
 ����Ct#Z]2���5�j��Af+˃lV#��Z#�8K4nu��{��Qr�|��dM�*��E��!��yR��'����S;�Q�Z�(�+����s�?�
-ҟ�OAiK�D,����Jz����bᕈ���z왜��r�Q��h*�F}�ޤ�E���BI� �$�\���Ɛ�9-r*6Fc��;��0�LE�����3�w���/�YD1C
-yg%h�P5N��b��<��S�D�6y܉����=^~b�m�s:��m�h��v��6.9��ڔ��䩏=6���l'}y��[^������.U�^�`f�}�8M��:��76Zs��`ЏtnJ��~�J����g�?�_��V�S�z)�㞃Y:��!�A�\�_�/��bFir��ՙ��+��
-�rŃ�����#��`�~t��`o���m00���b�ww����6�1��!�eL���0:�T�9-�5�TKRC�&�ztk���o$��3f��њ�L,Cæ�=���ߙ1Ja5���ķ����&w?˛��L[2hx����<�����>��P�'tLo�n��'�[��Xl?:�8�G��ii��ҀZ_Cw�Qk��G��Q$*��7���;�uE�|�i>_j�͝���&Q̦��Q���1J��<�h6�1�1�YS;�+����;��&��襻����Ғ�ME�$@�"�q��*H�_�aϠ*k ���P;����G�1hJ�%u�B0:�~J�jH��,�3|�Z����B���X������GC��p��̮�0�T;�r~?s���z�b�yLg�)W��o�Wcƍ+��!G\?���lV��S��_=*�%�i(���nrVH��In��87��Y��,�v��e�����<I�)ͯ0�f��o쟓���2c�k�hrnV�q����a�C��C���Ʉ�
 s����_V���H-�M��?������+��Y��X�^kZ�Y땇����g�gfg�H�}�==��ˣ�ֺ�����´����zY�iu$:�	i	�n]�_�C��J&����{�c��2����9]Fw�`3SY�b��ќ;%��#ٖ�3'��$[R�֭�25�n�Nk�fj�����30�!K��/��3�3�4�1\0�d�9�)萲���Щ�f��#�LDgEn��G�ŠH/ŽRܩ�-�͎����<�F�Z��{�L����nL�G��n��=ìY����5��ɽ��&B�8[��S\.':[J�f1�R��/=�tA�c�W��H���]��p��0�c�#��&��[Fgn5x�XFr��N�Ҧ��:s�)��;��#������-���H&B�s�yh�x�-��3M��3-���m�z�{午�;%nR��͙��V:G��'G��ڴv��tY5�C~���f�����x�WE��r���>��keo�٪����nE3"�U.l�����#�2r�׎YMT��V��
 ���O"z{ԆoUc�h1G��R�&��7�\�.T%%��&P�[ĿaV0����1b�qX��0t��c�}����д�ņX��?����z������W���I���[���~���l<��ݨѱfC\��2wB����������GP�nGٳ�������լ4�s�iu�Z�ZG�sMre��4�&�$;R���U��;ig9���&>��m`��){�R�h�&2t�$�ٖ'��T*�$S�O��T��Q]|FzRq �)X�H;"7��eׅ���(J�b
-h�d���B�#M���iu/����o��c�&#��4�'�8b�ī�g�ў	B��K$T&�n�vu��&]ׄ�T0�$Y�[wE���?���e��n�g-[I�)Gˬ�N��ގEI��S�N���᝘TT�0jS�����(%H�L�G.1�G�g��EΠ���6�+ў�v�9��v�.����t��PJ�*����11����ക?7ז��4}M��|����L��S�Ӹ=��\���4�T#�|�D�cb(�&q>P�b����O*A���e@@����f�׃FS50&����'�j���a�P�
-�����*��Ь�6�����ƌ�����T�h4���o�hjG�+#�ƋCS\�H%C����c"-d]��^7�Qd��ø٬�~Qb@����6�;�p躵j8�ZN7�������H��H�9H��98�\ :�֯��T΄�\�˖��eٳh�?'�@3��̈��7lV_:��Si��뵥��>kx=>+REe�*9��� ]�^�Υ3|�{<n >��	�Z��#��	=}�~��'=����~��Q�IA��#�`�j�>�:-n�>4�B�_�0"�t��5��I�mġ�0��:M�0C|���p����k .���-���|���:�&GlfbfRG![�eʊ-L,L�'t�M��މ�m��b���ZK���juH3�]yÑ�D;جa-�A�����Ȟ�c���F��b�Y�>��R���T2s��Dou�>oi�^��1��<�+���/e���vnp��t}����N�x�|�:󍞗ⰳ{R���o>�%x�"nq�ӽt5�6���YVQ���q[����k�%�%���'�菘�ֲ���E��a�
 �'��XlvIg�hx������D����/����St��dYb먐�dM~�(� ¨�|�?t��/�G����;�L4����f*ƚM���(�pq��H��1V�ʯC�"��4l}#�l���*��sf����4ƒ��C��l3��Y���	a���o�0l�Sx���S��@�jB3�R��˝�-�2K��_����n���9�h�sZ�<�]E��ʢ���E�����������rx�˟�-�(����K�����ﭥ����z�^���t�N'�:$aQ�E��B� Ač8������{���,@�����/̨���'#��(���{�;�����w���{�ԭ�{�9��sέ�"u����:��M�����ϗ=]��]�w5��1���\2��-�}U"h�/W���TvwUQ���N��U���)�Һ� m�r7"Q��{"��Sc�X0V�c%2^��rhtZ�a��)��@HB!�R2#��Xq8V�H�)Y�� �qSu����F���
 �E4衮�
-���=p��Ҙ1)0&V����u�K���*�"E��������N��lt�>�"����F��I��Oa�k��NT�2]_/r�=}nT\���+x}��i줻у�!��[X���տ������ZD��5�jQ"V��U���Q}]���썪K�]��J^��[���ZV�s��L�-�Tw� ��)F�Xہrt3�� ��9������k��b�+_�bqx���$�up��Wᚇ��s�wW���E#W���[v���fo;W���<�m-��k�)��c�F���:*����"@��^����?����o��(���X��t߯{G�x��7���<#M�R�c�2�ʄ�j��~S0��9 O�)9��h�LV���������'�,)���e�F�Cm��r'��!���E*��L-�J񒒢�����v��@���H	��:��P@U����1�������<�����R�#�ʓ�X2i4�{̤���������&W�a�ύwğ�q%�H�
 �����+�^�x���>Q� ����aHA��|E1H!_�8��L`������+;cG����&a�'>lۈ��߲[JJV������XU�Syg�}j�e�bl�X���rw	{#w��}A:F�?�COC޹���ax�O%O�^����*�*�0���tB5N��c"N���C/"u�\�Ϝ}���@�c[�NDų�A8����@���Qo�w�wO�������]�V{n�Pq]�n�gs�.�ϯs��p)�8� 孑����⑈"���I�!Aȉ��.d��'�z�sh���Q�|cD�[Qb�6b��,��J	�!������!��h�v�H/���t���?O��d��+��AS��l�i�(<������x�O�u���\����w�F���2P�m�P&K����n�]�0�:EkKF�&�fyf�����k��~�x[�p�h�}�X�h�W1R�R��Cז�<x[�=�n/=(�:Qb�����y�h�D���T\��e�a�����ze �,��^�{��\�&�
 �Xq1��{��@�U1� �*�1�-����B˰X��p�*o�ߓOɌ��զp0��}��&.jAjY����BbV�I$aVbYD�E~�+�'�؁������;�&io�i��w�������ow��TO��򾡺�H�X���&�s�,� ��s磳������S���l��~���=�6{�2d'%��o��xu�#_������iS��M{�>{�,0ς���J�K�S�Z��8ou��l��1b������&��G�z�Q�	�7.�?��]���3K�Ĩ��mek-�a�<��N6#�6KۢM�ۙ����h��f㇈��6:7����BTI��1etGb]����lB��^Tc&��D�V��RmRy�T��������*�/H@�0{��H	P$BOu��C�r>��q{U��&u�]�}�۟,���GO�y�?�w�zd�c��<~\Ń�3��g_`|xt��&�[�WyyU����o���C+�/�uj��-�(��/����kS_w#�n����|5�W7)�)�)��.3�u��_V7ϴܹ�|
 ^�c�w��J��{B=鞦�����������O6�����ݨ�J�7���0'��P�����&���4�Yӕa���R���B��!U�`��PL�y�Ěb�Xu,��UƤ�ZŇ�נ��2T1�U�cMM�uu��pyy�	�,�8��b�&����N?��6+�d�)�\�e=�*caT�;�Џ�������;��_�ƈ1̡��8��$wZD)�qR�����"�Q|�ԉ�0Pْ#BD'<u�	!�a���J-��ZN�����D�I��cg�YM�����ى��{P�N#9:�ǎ_t2)��q�chV�Ardei������j�Ym�F��	�h��qlGU!7:؄�m��Q^��Qn� Y���_<�Y1��Uׁ�WP[�]�<�5No�>�}�VR>�pB�-�!�˟a�t%�\���N����x7~���������p|bp�H���ћ�ρ�Wb6��k*����Q��h�t<�=g�0L5��o{����8�n�:x@����%�p\���{�ߏ/����? �������E�
-.ݭ/2�i4
-(f���~9l�1�xp�f���;��5i��?q����C���^�>����ňJsb�����0�6���e^v��3�N��O;�bTX���#}ѠH���3�zC6b�I���{ax��V�1P��
-m$ϳ����{����;��.f`ޥ��L2	��xJ��g� =�ž�^льr��@��\�M�dp[9�F��˾�s��[���^a�g�v����n�+��]�=�C��)�������Pח�V�p�6���7SGS��tEh�N�D��hT�E���JԄ@M�U&}iM<�̂���*D�!l;\QJ�&2&S���k�3��������PchJhn������c!]�S纫"���;4�k�k�i�X[�oPÀ%��N�4jןl<IB
-�-'���ĝ�H#l#�iH�s:�~�EJV���F_���N^W��Ǣ�E��^�>@(����(u���Ņ�i� 0(]�s��y���c��z���=����Fn���=�EIϹ��G�>�cf�73�\���k:��k]�Ж��^�r�-ap�6��S�hk��K�����������bm���tD�>P#/��x��\R�8� �}N\��I�(:�/���Q��Z�=�uw,��BhvhU�i�Vo􇬨�)��8<�$��]�V�C8&Ђ����Á�7��7��^4�D�A��1�T:�æ�`(Ӗ�a��"2�芢EO�OE^�{��^�`�0�zٿ�lN&+�Q�&8�xq�
-�̶h[��Bv�q5u������������I�S;�\�
-_8����r&��1`$<L)��<���?�@M���?��f�������#�e>�{���7�Ȟz�),u��x����q��Y;/A�YN��
-�@j�R���G?��w�oqm�w��&E'ś������=�Q49q(�*�0�~�������V�V�O�K�Q�m�����͞�O!�p��YE�K���ϥ�`�-��w2߿�Ba�%���5�����R!Ã�5����K�3�6�lY9���zsn�A�����l!�+�x�@M��,�4�eP�H�{��:��Q��}�^����.wp6W�e�@����>�%�	z\�	�{�#M\��1WC��CЄy�&����/(���䥋�Sx��jbL�����q�M��^�K�r��ŀ�`gQ�7bI	`2
-Z�s���^�ƿ-��4�� ���YRܝ��X����'�[43���Ә.lmHU\�3�;-:^�E� _���_8�2���3���2j`�W�c�p�R��C�J1�~����k�����Iy������7�|~f�-�:����k�]w~�R�֏?ފ)�ݨo���E ���v�s�����P�S��[�[�wŏ��ď����,���,��}i�.0�{ip�wV�û4x���ߖ�ˬu�s��}���N@�{��$4��!��	ٌ�i��VW�9�3�%K��*�a�p�"FC���褓$��do>搄.2;����]^!�zr������?�ts���	��n�L��_;?{��ߏ�[87W���d�A���C�t��MG<��'�n���6W9z�yI��}���=s'�q����e��!%�������L7���v�v��^r����Y���'��� S=�r9R@z��x ��H��-��^����� #ZT5�J�Ä47+�Cq�p��`b��=�9O�$���N�
 ��9�xA��Kfˡ�-ü��/�� (�&�����7_���\]>��#�&!��}8=�N_z�k�W9܂�����k�F"h�}��,���%�8uN�wѮ��֩�#n���܄(3�`@I�V�ǡ��_�z�gB�#����Z��UיoH���#�]�h➪퉭U{J���|" �,��������XZ�R!�P�bZ���@,�KP28�V�zڪ����^X��=���������������0���+���8��
-dc/���h�ÄD������zG��F��#�=�T���Y�T�+˜�Ac���:4`��JAk�rрU.L��X�P�i��rHT����q��3��?�1qMܹ��g�<{t��;�|��;�<D|�H�=�Ɣ^Y��R7�����g�@�{7�K����6����!^hF���B�Tʶx�Hp�f��nx�>I�wR��4���bwk��~�=��zt6��V!(P�� �ܲ-�$
-O霊��d�����7C���l�˜���sr�k]%.������ʔ\q`	�(*B�]-g��%��y�	�8������TE��#&����B�pTN�k���]-C�}.�����^�a��|.�!9���j�^VB�֧�$�c�����ӻ%SPP����YF�p�QXW�c�2��˧n�=�+��#���,@�z����B�8�l���>a�]S��9�����ʤ5}_�+TU	q靖���H�]��b������b�x������c5&���MN��ف�X�M����Oc6���)��0+9�P���\�r}d���=9l�
-
-c�j�9^�\Pw߰N����&����`�=k����eg�.�TH��riW�&��<�<��p	g��g[g��v��v�n�Muӛ��ɺ�t�����X�΋��>�ͬ�w�}�ad����Ũ�L%�nqn
-���UlEӁ��,��SȈ��p7��?|.�3��������۾��Q�&@��O\j�V�TS�n�9kx� 5aƌQӲߐ�_}~�����[���,H��E����q݄�����������L$W����E�_کiX,G�������Dcz��$l\h�a{��*�$>�~��2�e�L�LZ72:2�$��z��=��T��@����L�a0�/j��$8i��9�`P˾0(�"Ң�b|���*-���mFҐ%i�X�ن=sh����[�_N�
-�����H̉&��\�D#�:-TW�Â��K ,���������&�)�ʔ���R�o��^B�;�0�P4�\�nQTc^��NSR�"}U�Nci䯵���>+��z��K�
-�yH�k��GA��|��_������cBrN�|X�k�z�\�������7ڱ%�W��b����H�c����.e�Ƥ>G�]��sЇ�����wz"i�ŝV[ڎ�#�Ɇ@�ȅtiЂ.�7g���U!v�DX�55����Q1-�Kb��_V��C6�s�~�,߄m%�W�Xp��������)\��>���V�t��b�A�����9`�[�8j�(q��(���U��9��a��uF�%��"�vl��d7@�!����p�9��Bi��H�J�{�ʿ����/
-�:�W��G:�x�X���Z�,#�#���QV��d��c��.�R&k �웯����v��m�o�v��Y�V�jG�<6��0r���a�U��S�����)�<�+<͏���";�A�(��)y|@��9ZS�VVqUT��dUUER���ʶcM�i|cS��(�%5�XyY�ߧ���Z%�k�C�'����ښ�h�a0[$�S	VW8����l���1\�uǨ�����!4�o8�@7���u�L�[7�2�KV?�����#��<�?T:7�#�5,��21\��iRQ�-Lk�&�x���
-�"�8�&O�B�r[�ZZ����q�h0��0���]�#�����=��OK����@�GQ��d�Ս�-v���U젂���C��4��L���KG�խ9�f��}xrU����	8%UV:��TJ����7�7~��̥��vcj�P��k�{���4M�'������ʧ#*���ڣ��:�怎zB�C�CG��vk�V��/���)u}p'�EѾ�  CQį�N�:(�x��p�2?\�U�$~}��ܬ���rQj�bk��*���:��	�*v��"��m�A���'���wWlEꭈu�A�)�٪��wM?*j��I��>V��it���"�����n[Lt[�X@���E����J����ip5���9�1���Qo����̫���&��	�#��e?�ne�v���u3�&�_��ՠv;������c� ��cL=�#p���:��8��~�c<J������Qs���F�QmL��ݰļ��f_��9B�5���z�)H���Hr����Q���W!��Z��
 ���F���
-������לҰ���N7A�u��N����K����l(��H�O�������)�q?�i���I�H�_*F��y�ذy�n4��(��Ć����Ƨ�<:��`s58�إ��@�`0 eM�����_�5�n���B+W}�������ס����[?�_��k�Sж�5h˞����z��G�Ö��M�{��2�1e����ou�e��j1/�hL4W����ŝ��74�>�~��(�a��3�N�K��w�[�gh��GXVL�E�痝�,e���F���l͉�5��e�Z�!�!��-f��XȪ�:OU)�H�_��2~����NT����d=1 ��~��a��"S^w���
-�����b<<0�dıiב[D#�Yj�-��gV��Ve�^����Q`R���9��#��-��t�ܝ�9�Q}�H���������}�{�շ-0��5H28�,�O1,�����5�ڵ>u�<�����sH1V}N�/�b�)E�#EEш\b�&�A�c�fh��3D�P�14rHp"���^�U{JKi=e@�"��԰j�9ք�Ҿ_�6��hA�s��d��1N�=<��>�dž����D�g��n�`q��l����테u���.�Y�,�pZ�}x?�"j�2��;�0N��G�kn��ъ<왱=�&C��\�+���p�1:��i,���Ah��K�L�>��������1����dž?��o{o��1g�����'۠�;�������}V�t�����%���n�l�ۘyJ�!~���.ӕ$±�2�>�q[-Z�iHWV�����Dm3�F�l�&C��EK尼ޯ�b5��h�G��p �OMg���%��NefG&�qMZZ3��W�l�$��6TXr2-N�b��Ō(���r��=�u�cx�
 ��G]� �xGa��~*:�A�r7��G�'S_�	a���]���x�V����i���ρ��K0-y�Z$rw��zߠ�l�W �<�#5#��Kg7�@%����x���X-�}��3�e<		8)�h&8��2����`�·t9�15�'��va�7�+���|���3�+n}~켹���u:��n.�������.Ͼ���#���K��R�x�b}�nDIu}��jw�o���_�ɂ�x���<X�x���ɤ�^\�t^�܋f�s/(�(�3^h�z�ԓ�݆��z�5�[-�-���e��Q����R<ة8t��ɐ�����+�=�	��D"��@c
-�Fa��V��R�^���>�'�V���X4[|��/
-���i� ���;�:���o���=^������k
-���^��p�0ܝ�j�w:�jx���+�ߪ̜�,�s��H���x0������p��*�,ڬ�Ф�VSI\I�*�~���q&=đ$6��N��	=:��H`SRg2�u�Uu���sN�x��@F�K�HH���i�P�.w&Xu�g�!�(I��X_�'8��;��Zl�-n��'o#�6]�J�?JA͋[�,l�!&Wގ�G`glA�k��U6����U�@��D'��iλD���OƦv��������+?Q��'�n;����y�#M3VS2w^��Tm�x2h6��A�UR��v�;-$���$Y$������G�R�l�B{�y����+A�w�?�8��̙�f�<�l��6^�D��a���C���z
-�&�F�j#�<`"�QJ�[҄0�]�v��C`�ü\�@gzM8#��%��Ne��F��6ڶ.v�W�
-+�~+Z�3���yD3Qʽ��t���[���.������'�����`�����LB�@K(|��jx��/�����H�n��a:
-��^eX���ʹ9�2��	�Z!o7���g�C#YD�*O9�[�mΧ\=��.�P��y��y�o�ǜY�n+�u��u���f܎"*�9�:��qs�c:3]��!�(Z�f;�����g�u<���Kj;�k�V�nj��x����r�;η]vq����Ubtz�%T��ĵ^\_�s��-�-�c��׿�3��lj���8�8�*]� "E�}
-C[#�E����E���-E�H�H�����_���ӂ�)z�+=|^�_W@�~��kX��2��P�ߟ�~�(K�{3�tI=��JCK�0�C�tT���-"z)H�Pr��Vk�%�a8��h}�]���c�<��0Bzc�gB��dϘ5SH��f����o��\c`��TT�JQIZT�q��(=(���bF��탿Bk�]q9�)%5"Ma<
-�Q
-gKS=�W����r@��IȰ�,RQ���u#ҤX��eH�Z 9:��1�+�ӕfG�:�.���b_���D�|��20w����O���"���q�T�>�wc}�\L����\/�H'y� �� �QR��ᖖ�+ϭ;��p���tw�N�1ܠB;CZ���â)��*�}X�a���%03~���7�\�s�l�<���"H��za�.���/��
-����%�K��{��A[���:�t����|\�w�:�W5���~��>���o+_O��d�7xP��L�%5�Kj�0��k	J{�τ2RƓ�Tf"�p�xL�)S�Ig2J>8^^o�ɦ{`�.��F;�88r�L,pB����@��W���?�	^��L�?�[r�~q���1k2�{�V/P�{qH���n��88�#
-�ts����\��a1�$aח��|���L.��۝|�b;����[�pN���jphq�@h���p<o��g�S�n��lMW����=n�_���T���F[��7"�O������2�F�F���Z��⤩��!%M���'!�_jH[QR)���ݕX�F9�˛z���l%=���GI'�y"P�a��
-Dν���Ț0����`�k��g�d��	,�U�c�����m;:���@/�/�,�>�of�"/�%��ٷU��٩EӋt�^�m*W��:�U��71"���� e�y�\b��a�m�Yq��'��K���p��Z����y��U�^�-�O��&_��*�N0��ʌWt``�ȲW�������]����<U)�T��j��nQ���x2�*\Е��\�]��+�#�vY<��㑰\�핕RX�aن�@�v+���.��z��x�B��AMa�'�)NeJJ�- 05@���+��T�c%v{�=�jX��x/����۲�;��_��P������ʉWn�*�H�X�(_�cku\����J���xA��3QK�׉���$��+�tx��\�;'��}�3B{dc�"�̛z"��P/�$���̛y{"�9�G�d>)����Q��"a�r��)��t��#�6���l�l�-Fz(�n �f+V)! y@�H���F��zē,��B���<�G��~�q ����I���1�
 D��sRDY�^z�� ~����æ��4(Um�`�;���wΡ5~�USɻƤ��G�R=aL����G_;��1���E��\֐�2�b�֛�U�k��k/ah�`w��%�p�7�y�d�ى��'���/{�A��b/��u���dz�}�Ѳ��ał��� N\D)K;�K�PN��B@���������I�I@Ң�ů0#�[2[�������u�g���\A��
-����Hx?��_����E�I�\�8HE脱|���M�wu��&���M��z�ʆ:ZқEqjzBS�����\�Q4�]��Mdh����Ucd$mHGgDm�Mz�E���q2@���o~��,!)�!p��)`�9���/r0S��"-��>�oGނ���%�#�Os��z��e��j�����]���� Ӡ�9����XS����Z���r0�v��s��l��f8�}�]ˤ�%0�`Ns��\���z���:�����
-�}��j��ڇ*��}��j��ڇ*���
-�}��jb�Pp�Fro�6�[ȳ���ݢ���v��+,�;H;�	�,�ɹ��Kpn%��'XG��.&�.#��
-�_Wp-SA�)�,O	T��u�#=X�@X��]`-XAj�P��8����	F9:2,E��-B�w�U�Ԇ�6�}J L��jTn'����5o�_�Z�<�#/p�H�j'��z�FWH�?	�V�A+:ځ�w�������C�W[�
-~�ژv�kK���p�v��X�ꗑ����+���$nu9iQ=���Q	����宼�&Ii{q�[�/'��N���E�n�;��%���-%g�$����(_M�L��)���"W���e���w7�����q��/ET���Ypd��*��E��אk�����U����j��nJ��+����Z�Zsm��ӬtΓc��*B�%P��t����w_��i����`�HK�P]'1u�Z(��O?H-C�+S��I�g�"���n_}��6�w:>���==oH���ho8�^�Bx�ə�n�!O�6��\�0~p���*+*�i�ۤI�;�֮h��::Wtt��j�X^.�^�T��}��U��m��:�i[P�Ա����S�ܶ��E����̟;����k�:W�6���TJ*����ٱ�caW⊂�>B���I�
-Ζ�WI��yږ��Z�Xx���ڗK]�X������O��u����/HvtJ�H��ڱzyjzU�Ek�u.�.i��Z��9�c��l�Zi��6���Ž�]ҼUҊ��e�]���kI��/��v�ŠΎ�[���,no]\p.�ۗ�.]����!-h_�b)��tV;BhEXm˻�%)��K�JE�	�m�||�`[����%���}�"��mz�V��'ݒk+C]��m��vt�k�/�WxQt���[E�6Ё��V���]��چq�-]1쉐�� �8��9�ChFt�Q�ٜ?�JY�;DZ��/ү���w��~����>P��
-��6�%U�_�2&�\�L`F�tĐ����'#�kG�s�b��7��a91av��<��B���P���t��k��Y!y���44d���-�+��i�2�f͚���U�t��Y��>��lß䡯��'ō��M�l�~��N?z��@��n�1i�^s�SK�R���:��kV)�&h.�(�}ӿ�{t�^3k�Q;�S�@�+VR����M���7��u��$��Фsw�I��2NL��mC�����6-4;Y����������o�G�\�������~@��u���nz������l��^t��V(���6�z���d*9ּ����$�=��sKP���<uŶ�%��󶊒���m�c�l�A�<O��L-�v[�����}�8x��G��L�*�8�4��<k���Xs��)4f������x�6(~�y]��yz�g��ߝ�p�<�����MVzx��t?��������Pw!>C��V&���,K�,�g)F]&f1<_�!�/�kr��,�
 ����k�� *�����t�����eh������\�=��� ��@�݅�i-9n���P|����729���o2�K�Mh�K��o6������,���[,����7�z��3�o��!�kfj�g0<�������g0<�f#�C���/�����Z4*��{ŷ�1>id�~8�����P|A �C�צ1���1<���[<h�q��5��a�NRo�a���8�~�m�3PO������6{;��h�l�{0��[��Dvγx(-x���n�9G�c��Zq����a԰��,�b��܀�mz`ҟ��k���p��z�I�`$�1�-
-?�O0��̷Þ�z=w�2�i���I`<X:�������f��4�?N�ej
-endstream
-endobj
-30 0 obj
-<</Filter /FlateDecode
-/Length 285
->> stream
-x�]Q�n� ��{L�1q�V�,9i+�Ї��lX�H5F��ŋ�HE4�3,ɹz������A���F9��I�/ڰT��ү�V�7�%�\ϣǾ2��� ��ѻ6�Z�cɛS费���\\O��`��gE
-�p�Kc_�!!۶R�������s��po#��m$��\��<���0
-�F����j;�ݸE����|�a�?*Ȼ���[����6���KB�XG"w�H�c�'B���,#t��!]o{.Y��$'�B@�+�̒�6x�8;�ŵ�_�h��
-endstream
-endobj
-40 0 obj
-<</Filter /FlateDecode
-/Length 324
->> stream
-x�]��n�0E��
-/�E�G	!�yH,�PI?��Cj��8��f��R-:3�wƌ�C}���<zw�l��N�`�N��M�����[��eQ7�衯M7���<��ѻ���j���ޜ�͍�>M��n�7�`<�YUq]pzi�k��P��U�k?���o�e��rB��A�h[	�57`eV��sX���R];��:ܝ��q,�j�$!:��=�Ȑ��� J���f�5��o�Gk	Y$9�.NK���X*(H�eg
-.:*�fK[[��\0Hm
-�HG��3uN��� �P��������;悗2�Bx�;�Y5??r�
-endstream
-endobj
-43 0 obj
-<</Length1 67316
-/Filter /FlateDecode
-/Length 29406
->> stream
-x��xTE�?~f�ֽ��nz�ݴ��j ���� H��T��Q�@�! "`�JCyAPEP, ��X����ٻ!��������{�ߓ]>s�ޙ{�3g��@��`Po�]�Ƨ�`��|��!c�}�r��W���r���G��V�1���n;0��1v�}�]��M1�?bİ!a����%D*&�OL��#Ʒ#�1���Y���-@�����1���s�\P�z�$��O�����>C��t���t	@�m��.�.�<�����Y�h��ϊ����t�ڽ��X�'hvL�X��
-�-ʺA'�����N���?�\���O�σ
-�P�Τ�� U�&4�E��M����j"�+��2��0��%�_Zwa�՘n�t+o���"N"��qV^W�Do�ƺ[x[��x~A'����~�5�o���#�b|���T2a,�_�v;$�&��Y������ˇb�R�9�^��خ���y�)B��x�9��VcoBci���K.^�b��i;Dg���5b&���s�#�G���<�m�x�G��%�K��#��q(HCI��t-d�؎�.���}#��~�������g�ع"��!Rh�y�^al��H%td��hD<�'�c�.@�_��3�8P29�N n���	���\�y�U`�yMZ��%h�e��x������P�~��4����������|���A�@:#dhb.�u!�'�L?������5��F��y)@������r��y'��2��i�3���޹L�6�=^+͒��)��:󐯧�J�H>� ��Y���x�X��HD�A�A�@�F4CtFTǾ�eB^Qf�l
-�@ِ� qlBf��T�g@g�[���$)ka��$~M�/\fq,���:�e&H�|�r��O.S�uO:����l)�;3ׇ�/�B���.�||A���eM�u¢Y�� Œ�G�4ȋr:^�k�)��MY�IЁ=�K!�Հ:r=�����E���vB���^T�.�P�Q�N��5����<�t�t�&K��,�1�����Щ"�'Zdg��S��e�i�zD^�6s�yV>l�x?Or�Pϑzo�b~1�QSK'�ѤT�N�6�8��d?4�v��D��G]������̓��a�()�zf��0�.@��}�#��>������,KA��ʔ�|K�<HԿ,��p��Qg��X�6p�,��шy5�(�Ͻ�2�9A��$��+ɧ��\V�bmA��S����s��m����q;�_�Vh_HW�s;�Xz�l���KK���|�7M����Rb�da�J%�"d�U��������ZOk��@>�uTnc-{���7?��b�'Ƨ+�a�|�m��2K��8��R�|1����e3Q11��D�@_��
 ��ȞA>�h<Ž����6�X/ZB�^��k*�<O�+�s�!�E[���\�����s��-��a�/��:�`�z����v4�\�u(�(�ݰ��r��a?^����2�y��T"���'��r_�:�\-��J_ԹHX��x���c�vqb�~nE����i��?������܇v�
-�Gk F.@���-l�L�?l5���(Ϡ���3P(�C[e4�üy2�I�w�MG����;�{,���l��m[r_��\_T?�+�1�`��;X�:�,��V�3ȇG�6�4w݈���Tsy�%I�	�|�>����[��`��2X}�]Ԗ>D]��c�0X��I�0���p�Ί��Kз���ϧaz!����,�S�����&ǹ�v�c('���'���`�P�f`�ws-�'�(1�sH��hWb�AT3�w�	����7��Z>���b|�>�u��#=�^;��iZ֓΃5�e�3hú��d���ڮ:TLK��D���x㵐��XH����!�k��t#�p��ИS�[�X�x/XV���ʯ9��zCz�5r���Q�>�1��X���ʁ�؉C��=��a��UJ��O� �������O�O�
-|�W���| ��7p��rj�����[��NC��"2!�yi?r��n�A�kc:<���<a�S"������y������z�4��+"(���$��Zb}D崶Zp(�`�;NK���&[�DŽ2X��i�;T㠩8�8�uQ�>�6���h��u����~Q^��rT�kc�W�8P���T��_�:"�!�D�i� ������r^Ж�U�J�Q����Pw�!� v����*‰P��������
-�\C[r�.��C}�~�y�z��@80�¼;�>p��߅��0��,�2�6[m5�z������%��@�+��0�#��+�#}�B��MG�V���`L߃؎�s�����|��Hk!�a�~�G���_����w)�,Cq�~�tJ�=ĿM��yZy�����
-g�h��g�����{���)�gYEH}�k�Sڹ�}Y�?�Ѣb�&�X� "H����W�;s��rqf �����|1.kݨh[�%X�p"�-:��N����	E���F/q`:�/��]����@�����ND�KpM��?�؛�i����t��/���+���hj�#G��?�����z-��5��:�?M�� �����ͭ���?�7I����Oӕ���8]�/	�+�O�e/���A\9*��
-���6]���c�����f��Gm+�@uk]���3�k��$�MծB��20�	��f�y����d?�6�a�aL;���n�����se������g���ㇺ��0����\�=$�}��������D%�L@��t(�C�G(.��~x���#�!��}�y��ϼ�Lu:���I�����a~�e�gze���s�Gp���0�φT/?/1K��<�'\��z����'�	���9�O�43 �:C��%��)�^)u�)�1*�#�����h)�S���/�xV3����n��z�Ud[K�=�Tˇv�4�i[�`��1X����+}��*_��쏟eƕ�iZ�\�'�]�yL�~��v���$Ρ�7�mp�/D��W������|�:�a���������BO6�}�3�W��ۤ�Ǖ���r��|��o����/𼇟A�Wx�N��;1_�����P>��)�|��t֧+]@���9~6��O�b����w���JO�gx�-`]��nL๙���
 ��5ۭ�ώ��G��<-��Bq�f����w���3�P�Y`�4��3���c���ܺ����ө��-*x��v���������`�ƚ[g�V]�uh��Q^h'o�T6���h�p�:ἆ�#�KpKMa(sA>ig~@�!EO������H��4��	�|�8��+��W@X�r9�q��$�zN�k�q�˄��k��W*�_"�Ч��֐OK��e8�9Q�*��n���O{�?�؍hSؖӺ�����U��W��i�����1����w���|_e`��awݔ����_�������8��ϩ�����0�n����*���c�v��t-�O���;�]�����8�/��V�S�޳�����W6[[@�g�=�L��#p_m�����H�i�1�B�D���ۖm�-`�Y�1���:��VJG!��'��w+��b��fY��^����e	�{튮�d�^�ރ��u�m
-�|�k~.����-~�{~�_�]�?��h3�p��PW��t6��k�xa����ޙh�� Cq��<��~���0Q���k��O<�-�Yzb�30X���f�ۣ��>�s�6����`��+B<�τ����`��D�/N�!}��];p�6K������㺾����Xء\ı��^y���)?�{[e,���ja�Z�W�W�����ȳ&��˟ݫ����W��!��#��1�x��po|o@�<���&�9�	�� _���#TWu���,=�)��>�S��s������O�P_�	���]9�|�� �����-�F1L<ײ��i��y[���JT�k�~T�Oa���9�)_?��ߢ�����N�O#!�?�g"��5&�o'ʒ�Ϫ;��ʐ�Õ�[�|	������������5Z�}���ùic����K�-���� ������y�[��S#�̜�_���(�������h�����}����j�K5
 �O���{.?�nJ��34������x�_�>�tD0�~�I��'������T
-��25@�o���}����*������,����,Ho|�%Ho�������Њ��\��i�C�ݳ;��-.H�������u��i�T��9f����$���ws���p=�2p#�q��	�
-
-�$�a���ql��<�a�́c~(�9�,��`����0���법�_�V;uo��� @E��U�Z��@/�C�`aN���{��A��}��=�|������O��:/�[��Oc��� ���)9n����Ҭ�p
-�ubb���8PW���Jl��0�by�?��<ܛrXi��EA�N�	��'��+����V�I�����}8�wl�[�/U�˭�d=ܶ������&���3{���
-\'e��'A;����<m�E�]y��k����|?s���"�^�*��n^���x��{We��+h{�oxW V�_����H�ӏ�Xv��_�ϛ�h���}+���}�������ȗle������j�>O�q��w����}uq�4��1�w�L�Ӄ���:q���B�r���4?d�v���No?I�Cm6���߉��	�q�A��E,A��E���?�>�I�~��{{�~�07^.�ۛ!}�|�^��a�&��[��|����(�z,Ҋ+X�(��ؿ�s^_������硝m8�S��ͭr+s+�����9u �\`��u��2�bz��^@�9�E�u0R�j���?8�rp
-������+=p[\��#��n8�X�K|�� x�����;����AJ� �+�}�z$ޥ'�{[�����]~�����Ct���x>�>(�xO���2$�8��*Cn�\z�m(?{唿��e������~���f������k�����&?�~��,�Tx�������-Z�9��=/�ٻ7{W�O���J�w7n�.�Mӕ����y�*���ẲCYm���'о��!�i��р�6��ۓp�R�3Q~N�F���3���A8ڦց�y��;q����_�b�� ��5��w��o��&��iB_nk�Mk��iho�m�{���A�p[$�%[�[*⴦eSZ�N�<5�
-��Y�z�ܞ���W�,.`���DO`� .!��g5|?-���}�Ubm�#`'�-���G	�B������_�|�5�� ��_h�Yc��s}���%�bM�5�����.�����~��s亟<o�sx�O*���>��=}�ܬ��
-tpb��|��2��]Dh����%k�|�r:�|����ͥ���}�z��$��o���}�C�C�^�2#ż�X�C`�k�;���? �Ω���|)����(�"G~�]���ր��
-�.~.��W�!~;�j�+n�'��`�H9�b
-c�� 9��ߙ�rAS�JLꥹtKa��8v7��
-�\��}�.K��.'�{��%�)�J��Q�)�)�:6qT��ķ�%��i���K�?��Dz=m=]=�=��[=�<zJ<�<�<�<<�<e�Po����m�m����f{o��N�>����⽘$'�'E'%'���$uK�t[��g�V&�d%949,929.ٓ\#9=�C��a)4ř��i4͞�L�H�IKHKM���0-+mLZA��YisӞJ[��6�8mk���]i�H;���/�����������F���[��:ѯ&�:�
-���J֕WZ]ɾ���wW̫�_ky�kW�Rʮ�WMY�E�h@_c����&��ȹ�؋� �U
-�z�q��A�����s+I�_�S{$rnL��27��s��?y���z:xzX��)�l����|���s��0�\Mo�7��s��s�˼��,�uM�t+rn~9�\ȹ�d�Ź��|�9��p�G9��-K[]ι}ȹO�s��97�7
-9�Wkr.��YWȕ�+M�s�+m���r��ի�]k!8�-+�3�B�N�G�܂�!ΞF�P�Z��Vü���'/G���_���t�+��i�Z�5*���s��\�B�9����l��s�9�|��CSr���zT��O����S�~?
-�����/|�s��ԃ'G�����s���u�sO�<��Ă+N����+����N�}9Q��D����o{<�x�����w�������#��c?;w��cg�}�[�}lDZ7��y�c/[����ZK=�|,���"^��6᭽��D}N]�.
-ܫ�����C�Y�P����x��w� �}���s1\ �MR���7��TDc��6J{E?`����0�7�ؚ��a8�VP!o��sl+���\�cV��f�Uhy�-�<��7u��F`x�mF�'+w�1�(�Se/�tx�݆���0�9�<�BK�Bd�#�\�a<�+��p��j�~�K��»����0�C>�a�������ܫ~� �w��|�!��9��`$���0�e0&�x��'��0	�{�[�&���<S�uX�`*�Cp��-dy�PˆDd�W�B��,&��5(#
-Q�&YB�'K�dYNtb#���E��������*YIV��dYK֑��zRD6�b�~�ä��!%d�L^'��AB����'q�08_�pA���$�D���_����$o��H4���PDbIy��"�$�$7y������"�%I$��!dy��O��H
-I%i�G���C�9D>����Nj��pΐ�J�2G���SSW�+O(O*O)O+�(�g��r��HY�(�)K�略��2e��ByQyIyYyEyU%�VV*����e��NyMY�)�be�R"���*�����J��E٪lS�+�R�Pv(;�7�����]�;�ne��W٧�����W>P(���5�L2e��Le&K�,+�*k�.�dC�P�H9�V�(�(�*G�ϔc�q�s�rR9�|�|�|��V�(_+�(ߢ��U�)�+���	��%��cF��T]j��F��j��ƨ�j���&�nգz�$5�7"�H#�8i�2�0�4�2Ng���o�o��/���f�����������̎�\qP5EMU�T�ZM���Pk��r�eD��C���#�t�Qu�:S���V�9�\u�����:_}B}R}J}Z}��Iu|�>�.T�{-���R�u��\]����G�38'��\}Y}E}U]��RW�kԵ�:�5u�Z�nP�Սj��Ɉ1b�8#�H0��1�F��l��F��3�ե'�����d�������i�IZ���
 ��I�uѺjݴ�Z���K����j���Z����n�j��۴�Z��Qèi����F��Q���8k�3�7�����6O{L{\��=�=�=�=�=�-О�j����s��sH���H~$��O�gr�\&�������N����)#���J(��JT�
-U�Fuj#��A��ACh(uR��4�ԡ�4��%�h4���4����H�ԃ>�\�=�I}�ASH�JӨ�V��iZ���F�1���q��h�����hmZ�֥�h}�AІ�mL�h�j{�d��B�S�4Z@��G�t����Π3�������������vH�X;��>�>Վj�iǴ����	�vJ�B�R�J;��Ѿ־Ѿ��j��������������vI�E���������Ag�ٲSviW��r��]���9R���5S��T��cu�K��+��k���tC���/'���=�C�Cu�앓�d9Ew�az��G�Qz����qz���'�nݣ{�$=YOq(z��ӫ���zM=]�%��izm��^W����3�zC���Xo�7�3�f�O��7׳�[�zKݯ��[�m�l�������Yo�wp�͡;l�a�;���z���M���{���z���O��8�G�é���[��� �6}����oׇ���0}�~�>Bi\�G��1�X�N}�>^��ߥO�'�w�
 ����y�1�8�O��Oҧ�������>K�Et1}�.��ӥ��^����;��x�x�x�Xi�2Vk��r����b��G�L6��c����3l{w/�Ul[�ֳl�����dﰽl?��>d��Q�9���a߱�����H�?�K�z��J��ʙr3c��޸b\5�e�i�.��^�W�5ZFM�0����L��˵��r��B�c��r��N� w��ɽ�~��o�o��ˣ�;��{X5y�<U.�Fi�<K.��ʏ���'��[Z(/���ty��L~I^)����y��U�&��ƽ�{��CV[>$"�O�_���7�9������|E6q'��G���p%��Sb�D�yѿOVR�R]���R�(�X#%Ci�4�=S��[+�LS�*��J���I�tQ�*ݔ�J���K��Q�*���J�����[��� ����2P�S��Hc�����b�G���n{��g�n�i�e�gohojϲ�����.��>��@{�=�>�>�>�8�qĒ��	r��"_�/uӆf�1�d
 �m�M�i6�f�6��a��ڜ6�-�n� _���e�W�7�w���q�8h|h|d2>6G�O�O���[�=K���(���;�!l���69�v��axf�n��;���'��G3������rXo��Vf3�_�71(ib0C�;�����E�P��`?`?h���1>�Ep����%<NZ�{��I��J�Cw$:����Hr$;R��4��Q�Q�Q���������Qӑ������������������p4p�stutstw�P&)w+����c����n��x��a�4ޤ{�t/�Gߣ����z��ҏ�!z������K�=M�Я�7(뷠l����}��yYKF	*���P��=�(��<y�Ag���%w������}y��j�Dy�|7��8y�<�Uc�YVu�y�� ��lԎ��sP[��tVu�	V��auY=V�e��!��%��2����y��z'�}��ޭ�B���a��Y��(�P�۠ޜ����D]��Q5"]n��S꣆��v�F�h�d)��
 5��k�~f�����Gep�rW��g�����n3쎐P�+,<"2*:&6.>!���&%�����U�Q3�V�:u���hаQ�&M3�5Ϻ�EK��m�۶kߡc��]�v�ޣg��}��럓;�ց�n�7n�?l�#F�=f���O�k⤻����'?0����
-z���Θ9kv᜹�{|�O>��3�]�h1,y~�˖�x�_yu��k��u��/�P��d���K�lݶ�_o���&����{�ݻ���p� |�ѡ���~v���'NV��T��T��T����)�u�QuFSuFSuFSuFSuFSuFSuFSuFSuFSuFSuFSuF����ߦo�V��-n�j�,�i�FdԯW�N�Z�5kT��KKMIN�z܉	�q�1�Q��a.gh��n�tMUd�Q�ڦ�����$_J��y:ef���W�Ŭv7�)��j�k����J5�������͂�ڵ�mS�E��S��d@����N����">_�OJ�޶1#��E$�۶��=#
-��e��6�6)m��jׂ6�Ɗ�S�o �-�����6P�8�����E�)�|E,����=s�f�'%�֮UD�M��RZ���*�FtS��)RE7ޑ�n`�wC���sK�p{^�=?%���"6$���J�~���'����ċ��əY�4�������™ޢe=s*�&�07��miZ���v��\db��^�>��SD�.��N�]�oXJ[��7�[���NQ8*�&��zݟT��b������>9)IE-�Sr�d'l���^�o��{co,�]k��`손P+bwT�+/1Q��:�*�,�#J�Q��ő��=5����P8�)V�O.�VE�8##��6y��f<��/�Ӝ)��_p��r��s�X9J���Q.'墆��xQzzQ͚\D�68�8�"ݨv�{JiJ�x�	�z o��6���OJ�<���c���gN ����_7=������Ⱦ�� XR�</%�D��Fi���Ψ�#���((��;�s�9޶�yo;��!(oZ^fŊ���xj�h<�(��+�D��HJ����RUC�9�ۮș�!�ڒ��
 �F��E�J��ͬa5K�1����ó2�䣝�(,��P��谣EP�ON��M�E�L���Φ��E~dY^�/�e%o�o�s�å�v�vh�
-ۥx��)5nO�:S
-�з�[�������:'����\�����B�)dV�~2����-N�>9Ÿ�i��:wC*��l�q����L���t&x�Ÿ����
-D�$2Dzh)���-��<g0�b���<��6�M����#T2�6��êo��xng5����鉞-�b&7��KY�ưȌ�V����+B/���;fn�wb8Q�X�؁8�P0�^�8��S��%��b��٪�Ŷ�x�,. L�u���#^@(�����؁�(J�,���8���9�l5&C$�������ڵg�fwTk�V�a �N��V+@��2
-8�92v��bQx�Q8���B	,c�P��L�r�,lc�/�L�������b�+�������� �%���W��:�/a=b����%zL_�4z���;
-�U�л:�>�	��C]DK�`����~�����I�<�A�q����1C�g��~�C;T�$3c���׵"�4+oE¢2J�Gſ�@���L�Dmc�������RS�5�SJ���M�,kU�~E�x�c��c�"z ��
-Ǝ`� �#�!�(e:^��>��C�==X�ݔ�ž֞VQ��h��~�����݂�G�t/R7�}tw���,l�D�DZ�e����0���Ew �<�E�DtGF<�P��\��	Ël�}`�b�N�W`��Q��
-���f�`��/��߷`&y�{�I���7}.�x���x�s�x���1��|��`�R����j�&�Go�Pz/r�^�ҽȥ{A���/�.�=W\�&rl�?�FMO�VR���"+H�0R0�<D
-�H�m� �$�7)�m�)����KnHf�cH�>R��L$>R�F
-RI��4�Ҥ�i+��V\����O(MB�&��'�M؁��)R~��MT�us���f�@�N��q�:з���8o�I���6���x�����DF�D\@�k'��a(�u-�����q�׋�յݝ����M�oM�':������&�ݦ�6��(t�\���86����W�t~��8�-�x�R��ط��*�<n	��d���!m
-E�$h�6��iFqB?lZ���JBx�͞�N{�K(��6a��o�D�=�1g�f��	�={�j���WJ�l���[�z��U‚�Ş��l�<���3:A�6S�PO/�O�^v���D��fO˄�<Y�Z�x�͞z8��@�&�F��4�-.طI)ᯥ.Ps��jc5C��&�5Q�W#�0ͩ�hvͦi��I�@�(5O���s�E��*��$wR��y�F�B'(
-g�i�ޭI碝C���ޢ˽SJ��9�5)
-����.j�޹T5{5I�\���5g!��bn���t��Rb�G���`�zt^<�����1Q���i�•�.�/�<+L����!�X��sՉ�E<b&�v.z�o����Ŷ�[ȏ���la-�Om{�|�";7�s)�'ꁗ���Pb~�4\�y=�j�@�Łzi��r��t�D�4]�$��m���6{Cj��텉���ho�:�ҰNZ��U�D�}Q�NQQ%!��D	�J�U�]�Rת2���l�#��$�8N�8Na���3�uz:��<w�@���Ki;�W4�1E�{���Z�1_��CGp:dXQnʰ좡)����E�@^�<%{l�'g�@�������mS�d�nlߣa���]�W�q��by_��Eq^ܞ�Մ�Մ����^�B�{�lРu.���n���5/>)�u�s|!�͓b��oEoe%����ف�E�[�nŋP�xQ�Y[E1S�'�o%+�"'f�RZC���'�1mGf�M�fM��3<�O�����Mr��I��j��\������y������-����:�ٌg2V^��e�<]�
 *�y��h�t�F�w�I01��;��h
-�X[���K��ab.��D�N&�a;=i��Ĥ���ŋI��&�,)�pf��sl^����_�X���e��Q�i�H�[^�)=�����JXGF�:�o����`n��.P6,�)�4��em�̆^��1�ik�@]X��r؏u��T�
-Q$�������z�����y��y7���#����0��9�c��K�2la����8������q��-��Ep�<�o?�R�5���`1$��A܋c��+�';i:^}|Cb����Yd��Z	0F�b�J��4Ihv5�C�q^u�f��¿�3b�/�/�!jAG�����de�*k���K5 K�����&'���/O6?��}q��b˯ɯt*��vK����|y�sށ/H�K��~�G���@���7F"���O�m�vz��(���(�e���<�Ûāw�%���������s�K���J�H�w}��y�~%a�)�In%#�2�<A��� ����}�hz��`ؿ����-M��g�s�o�r�v�}X���a΀�(�蟁�xg[���I���� !��O����;��#+ij��� ��|�K�/�
-ŕ�*4�?��o
-�=̧�z@�Og�,���F,��q8��l>~7�/�8�d"�3���Jy���|Q�������ռv��f�-(+.+1��H�C\=pÕ����Q8�P���!bG�ő��邜LF�	�>��t���,��َ\��\�1;h�sڈ����{F'�3�$-�G�Lee��&k��al��-`E�}�9��]fW�kJ6�#%K>)]j/�J�H�����3�M��PJ�ѫi��P{��������ZB�����t�S�!֖m��h)�0�<�|֕��ҕd}���T�>�9mN��Eɇ��M_��is֕t&�a�߻�G��V#ɒކ��v�����)v2�^P�PL��Փ��{�;ITi9�l$�����(��Z�9�Ė�kly6Ѷ�+�\��nd5څ>$���Lt���5a_�#0�~
-�Q�g��$_��d
-|��VԐ�Tj*�d/)�pRTZ��OR	�#`:�+�Q�H68������XW�܋�@xf��!�_Α>"w#� M:��m
-ː��NC�2m�f��hZ�������/Z���]�vBB	�:���P��p�B��H����+�ȼ�4���hf�S�+�<+ɣe�x�JE��"���vfmZH���t����N#1p��a���
-�O�7�4皇Q����]���z����	ʺ�f;6��$�4_5=�#�1���˪C�t��"���0��2'�ae#��#�ȭ���̶Q���x/	�I��B5@�NA�c	�Ӈ� �E#�����{,����D����gx�>���w��+h�0�:�ԛ�1#���4~	��]�Q���)��
-۴Y���Ktxp��6
-���+����*T�
-U��hO���B�P�*T�
-U�B�P�*T�
-U�B�P�*T�
-U�B�P�*T�
-U�B�P�*T�
-U�B�P�*T�
-�J�Ke�W�U�$W�+\���W��O{���O�"�t�5k�c���v%])%)@���X!V?/&���� ��P�����`�()��AJ7m�Wي�L؏WI���,�Ѭ�0��z��a�2i�˜t��A��CK�D�F"��߿_����)¶��3OGd�R����,#����3����{��l�[�ߒR�
-;�6N�+g9/�w⵳Zf͔�zй�~=2(==�4 d����X��?"��M���Fr�;�	�4���������u_������i9��U'�y��c�Z���r8Z�MPxj������$�������c��PڗG���\`�9e�ch�yP7==}?��?�C��?_�^I�W����������K:y8�v��K^�f��u&�����x�<Q�0D�y��MRf�YƬн!��1�mx��N�m�����+~�:�>&rtl^���^�cr�Le����7�3zD9b�+.-�$F~*�l��o�Τ艺?)�a=���ԩ�1^C�(����{\b^\!����L�D(5w�����oR��� j�|��9\����ǐGMQ�	d�M�� *��HOw�5n��JJr5_�3�AFc�ӗ��*}GZvO�֣-���'���2eժ�S:���DnY;xc��YYY����N�/{��E2���a�.�'Q����3�-0r���m��!�;\GK���t�&����L�.;%�l�6�E �/g���J�N�
 �R��%�+A�W�/�s,���i��˨e~�~-�xe�L�Xc+�"�B@�'�(>���z-Z���$�L�O�N�I).EQ5nܤ�R��P�g��;Iz���k���Qf����7��*j瀆�!�_��#"G(��F�Wm�]NGLx����e�����Ӊ1w����+�ݼԝ�%n;�[w)��S[t���tQ����2�~�����.��<ܕ�Տ�wh��C���~N���p��������ҁ�\ֿ���Uo\#y�7љ�qs���MޡlS�h{Ԏ�\{�������a��g�m;w&�b�}��z8�w&8�n��y�x
-4�:�p����e_B\DBB����S�K`���������JI�&~����P�-�����ڸ�
-kf�}��++�F/8IS�ݵ�%L��iT�[i*x����B#|9��b�oYײZ��6�+��
-3C꤇�i�Ju�)"��41-2��%�q�FQ��Zc���(Y�OR�6��i/.��r�/![��������b�{ݺVYCwN�uf�觖�8zv]���/�R�k`?�k)
-e/��lf1�ȗZ�#+�o#6��'/&׍t;&H���n��F�;Ar�H�k8R��Xa^'�6����꾺���˿�ٲ�(��󻝻�2���38��W�Q��������x�+j�sTD~�ݎ�#f8
-#fǿ��^&���:�J�_�����m��	i�7)�l�/A,��q�2�������Ⰹ�����U�[�����jyUu�O�m��G�N/��[��׎)%M�c���)��;�F���_��<iIG�y!�=��>��,_;���SK@Vв�t�q!r�y	o�M�
-�Iy4(\@TBJ��_����֯x�A��0cb�Q#�F�$�}�}���?<���#o�䑘E3���<b)����O��ݴ����K����β_���BZ,������������?E�0���9�,̹"B=`�E�j��P���9�5�"TD��0�,���G��W����.���i���z�]a�l3���ñ�I�I���T��n=�e~HV�}6��]�m�DOC[p��
-�A��R���P��)Ղ�F��Pʕ�pD4�^�(o��`��fކn
-)%s�|��`�n\�j����)�祬�Y�L>ə�3�KhBCCq�ş�q�����c�� �%��dRbb�D.
-��G��F���G���˴'' ��)lG.�������6"\"S\�E�k���O��]Rֈ~�m�����hʞ�6��	�&ɯ��\������!��pR�nD��<.��0���C#�(P��…�&B#F�7�$&�F+�r���KBl��Ȅ0�����Np�Pcp�Υ���nq�V�J]ۅV��aba���������X���DŽԌcz=������n���@s���,�!!��Hb��P

<TRUNCATED>

[40/50] [abbrv] incubator-apex-core git commit: Update README.md from devel-3

Posted by th...@apache.org.
Update README.md from 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/e3e8cf8d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/e3e8cf8d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/e3e8cf8d

Branch: refs/heads/master
Commit: e3e8cf8d740284423ff5c5ec0d5ac3770bffd917
Parents: ce44d23
Author: Thomas Weise <th...@datatorrent.com>
Authored: Thu Oct 22 20:41:22 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Thu Oct 22 20:41:22 2015 -0700

----------------------------------------------------------------------
 README.md | 15 +--------------
 1 file changed, 1 insertion(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/e3e8cf8d/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index baabcd2..a4a2b46 100644
--- a/README.md
+++ b/README.md
@@ -13,22 +13,9 @@ Please visit the [documentation section](http://apex.incubator.apache.org/docs.h
 
 ##Contributing
 
-This project welcomes new contributors.  If you would like to help by adding new features, enhancements or fixing bugs, here is how to do it.
+This project welcomes new contributors.  If you would like to help by adding new features, enhancements or fixing bugs, check out the [contributing guidelines](http://apex.incubator.apache.org/contributing.html).
 
 You acknowledge that your submissions to this repository are made pursuant the terms of the Apache License, Version 2.0 (http://www.apache.org/licenses/LICENSE-2.0.html) and constitute "Contributions," as defined therein, and you represent and warrant that you have the right and authority to do so.
-
-  * Fork your own GitHub repository
-  * Create a topic branch with an appropriate name
-  * Write code, comments, tests in your repository
-  * Create a GitHub pull request from your repository, providing as many details about your changes as possible
-  * After review and acceptance one of the committers will merge the pull request.
-
-When adding **new files**, please include the Apache v2.0 license header. From the top level directory:
-
-Run `mvn license:check -Dlicense.skip=false` to check correct header formatting.
-Run `mvn license:format -Dlicense.skip=false` to automatically add the header when missing.
-
-Thanks for contributing!
  
 ##Building Apex
 


[31/50] [abbrv] incubator-apex-core git commit: APEX-189 Add original copyright owner to NOTICE, rename license header template for clarity.

Posted by th...@apache.org.
APEX-189 Add original copyright owner to NOTICE, rename license header template for clarity.


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/979a0efd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/979a0efd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/979a0efd

Branch: refs/heads/master
Commit: 979a0efd282f3c2b26889767e72b5b25921775ac
Parents: d2bf3e5
Author: Thomas Weise <th...@datatorrent.com>
Authored: Sun Oct 18 22:33:20 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sun Oct 18 22:33:20 2015 -0700

----------------------------------------------------------------------
 NOTICE      |  3 +++
 header.txt  | 16 ++++++++++++++++
 license.txt | 16 ----------------
 pom.xml     |  4 ++--
 4 files changed, 21 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/979a0efd/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index d834235..eef5e86 100644
--- a/NOTICE
+++ b/NOTICE
@@ -4,3 +4,6 @@ Copyright (c) 2015 The Apache Software Foundation
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
 
+The initial developer of the original code is
+DataTorrent, Inc. (http://www.datatorrent.com)
+Copyright (c) 2012 - 2015. All Rights Reserved.

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/979a0efd/header.txt
----------------------------------------------------------------------
diff --git a/header.txt b/header.txt
new file mode 100644
index 0000000..60b675e
--- /dev/null
+++ b/header.txt
@@ -0,0 +1,16 @@
+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.

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/979a0efd/license.txt
----------------------------------------------------------------------
diff --git a/license.txt b/license.txt
deleted file mode 100644
index 60b675e..0000000
--- a/license.txt
+++ /dev/null
@@ -1,16 +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.

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/979a0efd/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6aad8c7..1fdabcf 100644
--- a/pom.xml
+++ b/pom.xml
@@ -33,7 +33,7 @@
   <version>3.2.0-incubating-SNAPSHOT</version>
   <packaging>pom</packaging>
 
-  <name>Apache Apex Core</name>
+  <name>Apache Apex</name>
   <url>http://apex.apache.org</url>
 
   <organization>
@@ -100,7 +100,7 @@
         <artifactId>license-maven-plugin</artifactId>
         <version>2.11</version>
         <configuration>
-          <header>license.txt</header>
+          <header>header.txt</header>
           <properties>
             <owner>Apache Software Foundation</owner>
             <email>dev@apex.incubator.apache.org</email>


[25/50] [abbrv] incubator-apex-core git commit: APEX-186 Enable license header check in Travis CI.

Posted by th...@apache.org.
APEX-186 Enable license header check in Travis CI.


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/4007ec58
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/4007ec58
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/4007ec58

Branch: refs/heads/master
Commit: 4007ec58e5aac2a4fc8ad8d6132a57116ee8e288
Parents: 78dc57d
Author: Thomas Weise <th...@datatorrent.com>
Authored: Fri Oct 9 11:29:21 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Sun Oct 11 20:03:00 2015 -0700

----------------------------------------------------------------------
 .travis.yml | 2 ++
 pom.xml     | 3 +++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4007ec58/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 9b18190..5540704 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -15,6 +15,8 @@
 
 language: java
 
+script: mvn apache-rat:check verify -Dlicense.skip=false
+
 notifications:
   slack:
     secure: GlWD2EjgNC6Lg2DtAfZuKhg2RTHE0FMeyfHH24D7TKmV49gRVTzTTqvExwOdLLYDDIu966eOF5w90/PfiD11A4rpm6+WyjRHDkpOhTyjBKWc2btMqNwiP1hRf2uKEG6A+RgszaQJ4HkGiMxIpDJ3o/jaTpBseOeA399t8Z7Pkd6obXVAEMcnm2XtfUPzIBqGblVkiecS3OzbkzjKWaOG6+nlp+ajVO7MazsOR05JNZ1MXnDOK/Qq/7xLBtweF4r/O8okyg52fST4pGqk4JTTI++bVFoRySpjNWSSJbdrfkWG/7h0sfqMcwMj8TpZqojcCuTvZih+IyPZwX3MP6Ls4bF6OFcD3BvWg049WbstA6ZdKnVW9fOiJoJ+Bx9gNI0tKtYeDt/8nMBfSRVMmzRVEGdTOEEpFs7n79OVVytwbp8qYFU+waqlG1/tMQvCclSaxuY8d236Ybg410KLiMQ0YhQ8ZaTVagHu2l0KrfV16Xq3/CRsolHa8k7PxDzmCxdi07Ao/mrorQLmJWoc7FkxEw6ZrEGHP17HTn5uidoTkANgGak4AVgrJm6zLKdAERxvr1KMnfyuJLT1ZK1x73SV+3EpdzcMLxvq2dMaBpZcfARiYB04EwSHVBdm/D0AIZtRL2s1rG85y1OtCkDMd04ZEvbwdKYZ+fD+HjTn7Zo7AKM=

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/4007ec58/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8f68dcb..2c14ce8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -99,6 +99,9 @@
             <email>dev@apex.incubator.apache.org</email>
           </properties>
           <excludes>
+            <exclude>DISCLAIMER</exclude>
+            <exclude>LICENSE</exclude>
+            <exclude>NOTICE</exclude>
             <exclude>license.txt</exclude>
             <exclude>**/*.md</exclude>
             <exclude>**/*.txt</exclude>


[10/50] [abbrv] incubator-apex-core git commit: APEX-164 #resolve changed log4j level to INFO from DEBUG

Posted by th...@apache.org.
APEX-164 #resolve changed log4j level to INFO from DEBUG


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/7957570e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/7957570e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/7957570e

Branch: refs/heads/master
Commit: 7957570e02edfd4221bfcc015922b5c660518158
Parents: 4adac06
Author: Vlad Rozov <v....@datatorrent.com>
Authored: Mon Sep 28 15:19:50 2015 -0700
Committer: Vlad Rozov <v....@datatorrent.com>
Committed: Mon Sep 28 15:19:50 2015 -0700

----------------------------------------------------------------------
 api/src/test/resources/log4j.properties                        | 2 +-
 bufferserver/src/test/resources/log4j.properties               | 2 +-
 engine/src/test/resources/log4j.properties                     | 6 +++---
 .../testAppPackage/mydtapp/src/test/resources/log4j.properties | 6 +++---
 4 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/7957570e/api/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/api/src/test/resources/log4j.properties b/api/src/test/resources/log4j.properties
index b7516d4..26857d2 100644
--- a/api/src/test/resources/log4j.properties
+++ b/api/src/test/resources/log4j.properties
@@ -37,4 +37,4 @@ log4j.appender.SYSLOG.Facility=LOCAL1
 
 #log4j.logger.org=info
 #log4j.logger.org.apache.commons.beanutils=warn
-log4j.logger.com.datatorrent=debug
+log4j.logger.com.datatorrent=info

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/7957570e/bufferserver/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/bufferserver/src/test/resources/log4j.properties b/bufferserver/src/test/resources/log4j.properties
index b7516d4..26857d2 100644
--- a/bufferserver/src/test/resources/log4j.properties
+++ b/bufferserver/src/test/resources/log4j.properties
@@ -37,4 +37,4 @@ log4j.appender.SYSLOG.Facility=LOCAL1
 
 #log4j.logger.org=info
 #log4j.logger.org.apache.commons.beanutils=warn
-log4j.logger.com.datatorrent=debug
+log4j.logger.com.datatorrent=info

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/7957570e/engine/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/engine/src/test/resources/log4j.properties b/engine/src/test/resources/log4j.properties
index f944776..26857d2 100644
--- a/engine/src/test/resources/log4j.properties
+++ b/engine/src/test/resources/log4j.properties
@@ -17,7 +17,7 @@
 # under the License.
 #
 
-log4j.rootLogger=DEBUG,CONSOLE
+log4j.rootLogger=INFO,CONSOLE
 
 log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
 log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
@@ -35,6 +35,6 @@ log4j.appender.SYSLOG.layout=org.apache.log4j.PatternLayout
 log4j.appender.SYSLOG.layout.conversionPattern=${dt.cid} %-5p [%t] %c{2} %x - %m%n
 log4j.appender.SYSLOG.Facility=LOCAL1
 
-log4j.logger.org=info
+#log4j.logger.org=info
 #log4j.logger.org.apache.commons.beanutils=warn
-log4j.logger.com.datatorrent=debug
+log4j.logger.com.datatorrent=info

http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/7957570e/engine/src/test/resources/testAppPackage/mydtapp/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/engine/src/test/resources/testAppPackage/mydtapp/src/test/resources/log4j.properties b/engine/src/test/resources/testAppPackage/mydtapp/src/test/resources/log4j.properties
index 451cff3..169b57e 100644
--- a/engine/src/test/resources/testAppPackage/mydtapp/src/test/resources/log4j.properties
+++ b/engine/src/test/resources/testAppPackage/mydtapp/src/test/resources/log4j.properties
@@ -17,7 +17,7 @@
 # under the License.
 #
 
-log4j.rootLogger=DEBUG,CONSOLE
+log4j.rootLogger=INFO,CONSOLE
 
 log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
 log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
@@ -35,6 +35,6 @@ log4j.appender.SYSLOG.layout=org.apache.log4j.PatternLayout
 log4j.appender.SYSLOG.layout.conversionPattern=${dt.cid} %-5p [%t] %c{2} %x - %m%n
 log4j.appender.SYSLOG.Facility=LOCAL1
 
-log4j.logger.org=info
+#log4j.logger.org=info
 #log4j.logger.org.apache.commons.beanutils=warn
-log4j.logger.com.datatorrent=debug
+log4j.logger.com.datatorrent=info


[24/50] [abbrv] incubator-apex-core git commit: Update changelog for release 3.2.0

Posted by th...@apache.org.
Update changelog for release 3.2.0


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/78dc57db
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/tree/78dc57db
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-core/diff/78dc57db

Branch: refs/heads/master
Commit: 78dc57dbc63b9f3e8ff69c12ea63afead963b137
Parents: b986f70
Author: Thomas Weise <th...@datatorrent.com>
Authored: Mon Oct 5 20:11:14 2015 -0700
Committer: Thomas Weise <th...@datatorrent.com>
Committed: Mon Oct 5 20:11:14 2015 -0700

----------------------------------------------------------------------
 CHANGELOG.md | 42 ++++++++++++++++++++++++++++++++++++++++--
 1 file changed, 40 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-apex-core/blob/78dc57db/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index af1ca53..7670a49 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -2,19 +2,57 @@ Apex Changelog
 ========================================================================================================================
 
 
-Version 3.1.0
+Version 3.2.0
 ------------------------------------------------------------------------------------------------------------------------
 
+### Bug
+* [APEX-88] - Stray directories getting created under working directory when running tests using StramLocalCluster
+* [APEX-89] - Javascript error when launching an app under CDH 5.2 (hadoop 2.5.0)
+* [APEX-93] - Persist operators need a re-deploy after a sink being persisted is dynamically partitioned
+* [APEX-96] - AsyncFSStorageAgent loses synccheckpoint flag value during serialization/deserialization
+* [APEX-97] - syncCheckpoint property on AsyncFSStorageAgent not working
+* [APEX-98] - WindowGenerator.getWindowMillis loses precisions
+* [APEX-100] - StreamingContainerManagerTest.testAppDataPush uses hardcoded 12345 port
+* [APEX-101] - Negative Memory reported for Application Master
+* [APEX-102] - AppDataPush Agent Not Adding timeBuckets and dimensionsAggregators Correctly To The Schema
+* [APEX-111] - dtcli: show-logical-plan with app package does not list the applications in the app package
+* [APEX-112] - Property change on logical operator converts from null to "null"(string)
+* [APEX-113] - Application Master not setting correct temp location
+* [APEX-114] - Stateful Stream Codec Exception
+* [APEX-117] - When Using Double Max Aggregator Data Stops Being Pushed By AppDataPushAgent
+* [APEX-118] - Sometimes the collection of metric values passed to an AutoMetric aggregator is empty
+* [APEX-120] - AsyncFSStorageAgentTest unit tests fail
+* [APEX-121] - Making sure that state is transferred to client for Statefull Codec
+* [APEX-126] - handleIdleTime Called Outside Of The Space Between beginWindow and endWindow
+* [APEX-149] - In secure mode non-HA setup STRAM web service calls are failing
+* [APEX-156] - checkstyle CustomImportOrder inconsistent behavior
+* [APEX-159] - StramMiniClusterTest.testOperatorFailureRecovery succeeds with unexpected error condition
+* [APEX-173] - StringCodec registered with BeanUtils is throwing null pointer
+
+### Task
+* [APEX-16] - Configure Checkstyle plugin
+* [APEX-124] - Set the default temp location in pom
+
 ### Improvement
-* [APEX-12] - Fix Base Operator To Not Show Name Property In App Builder
+* [APEX-22] - Ability to re-declare ports 
+* [APEX-68] - Buffer server should use a separate thread to spool blocks to disk
+* [APEX-115] - Use containers set temp location
+
+
+
+Version 3.1.0
+------------------------------------------------------------------------------------------------------------------------
 
 ### Bug
+* [APEX-12] - Fix Base Operator To Not Show Name Property In App Builder
 * [APEX-35] - Test exceptions due to missing directory in saveMetaInfo
 * [APEX-36] - FSStorageAgent to account for HDFS lease when writing checkpoint files
 * [APEX-37] - Container and operator json line file in StreamingContainerManager should not be appended from previous app attempt 
 * [APEX-43] - SchemaSupport: TUPLE_CLASS attribute should use Class2String StringCodec
 * [APEX-56] - Controlled plan modification on operator shutdown 
 
+### Improvement
+* [APEX-13] - Unblock operator thread from checkpoint writes
 
 
 Version 3.0.0