You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samoa.apache.org by gd...@apache.org on 2016/03/16 07:12:39 UTC

[01/17] incubator-samoa git commit: SAMOA-48: Fix for VHT [Forced Update!]

Repository: incubator-samoa
Updated Branches:
  refs/heads/master f9db1f271 -> 1bd1012af (forced update)


SAMOA-48: Fix for VHT


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

Branch: refs/heads/master
Commit: af25e7d1f464323dcd0e2bb5729dc70e5ad36887
Parents: d454deb
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Mon Oct 26 14:20:55 2015 +0200
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Mon Oct 26 14:20:55 2015 +0200

----------------------------------------------------------------------
 .../trees/ModelAggregatorProcessor.java         | 47 ++++++++------------
 1 file changed, 19 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/af25e7d1/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
index 1e79f48..846d8e1 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
@@ -123,8 +123,7 @@ final class ModelAggregatorProcessor implements Processor {
       SplittingNodeInfo splittingNode = splittingNodes.get(timedOutSplitId);
       if (splittingNode != null) {
         this.splittingNodes.remove(timedOutSplitId);
-        this.continueAttemptToSplit(splittingNode.activeLearningNode,
-            splittingNode.foundNode);
+        this.continueAttemptToSplit(splittingNode.activeLearningNode, splittingNode.foundNode);
 
       }
 
@@ -168,15 +167,12 @@ final class ModelAggregatorProcessor implements Processor {
         // removed by timeout thread
         ActiveLearningNode activeLearningNode = splittingNodeInfo.activeLearningNode;
 
-        activeLearningNode.addDistributedSuggestions(
-            lrce.getBestSuggestion(),
-            lrce.getSecondBestSuggestion());
+        activeLearningNode.addDistributedSuggestions(lrce.getBestSuggestion(), lrce.getSecondBestSuggestion());
 
         if (activeLearningNode.isAllSuggestionsCollected()) {
           splittingNodeInfo.scheduledFuture.cancel(false);
           this.splittingNodes.remove(lrceSplitId);
-          this.continueAttemptToSplit(activeLearningNode,
-              splittingNodeInfo.foundNode);
+          this.continueAttemptToSplit(activeLearningNode, splittingNodeInfo.foundNode);
         }
       }
     }
@@ -205,8 +201,7 @@ final class ModelAggregatorProcessor implements Processor {
   @Override
   public Processor newProcessor(Processor p) {
     ModelAggregatorProcessor oldProcessor = (ModelAggregatorProcessor) p;
-    ModelAggregatorProcessor newProcessor =
-        new ModelAggregatorProcessor.Builder(oldProcessor).build();
+    ModelAggregatorProcessor newProcessor = new ModelAggregatorProcessor.Builder(oldProcessor).build();
 
     newProcessor.setResultStream(oldProcessor.resultStream);
     newProcessor.setAttributeStream(oldProcessor.attributeStream);
@@ -264,8 +259,13 @@ final class ModelAggregatorProcessor implements Processor {
   }
 
   private ResultContentEvent newResultContentEvent(double[] prediction, Instance inst, InstancesContentEvent inEvent) {
+    boolean isLastEvent = false;
+    if (inEvent.isLastEvent()) {
+      Instance[] tmp = inEvent.getInstances();
+      isLastEvent = inst == tmp[tmp.length - 1]; // only set LastEvent on the last instance in the mini-batch
+    }
     ResultContentEvent rce = new ResultContentEvent(inEvent.getInstanceIndex(), inst, (int) inst.classValue(),
-        prediction, inEvent.isLastEvent());
+        prediction, isLastEvent);
     rce.setClassifierIndex(this.processorId);
     rce.setEvaluationIndex(inEvent.getEvaluationIndex());
     return rce;
@@ -314,8 +314,7 @@ final class ModelAggregatorProcessor implements Processor {
     double[] prediction = null;
     if (isTesting) {
       prediction = getVotesForInstance(inst, false);
-      this.resultStream.put(newResultContentEvent(prediction, inst,
-          instContentEvent));
+      this.resultStream.put(newResultContentEvent(prediction, inst, instContentEvent));
     }
 
     if (isTraining) {
@@ -363,15 +362,13 @@ final class ModelAggregatorProcessor implements Processor {
     return foundList.toArray(new FoundNode[foundList.size()]);
   }
 
-  protected void findNodes(Node node, SplitNode parent,
-      int parentBranch, List<FoundNode> found) {
+  protected void findNodes(Node node, SplitNode parent, int parentBranch, List<FoundNode> found) {
     if (node != null) {
       found.add(new FoundNode(node, parent, parentBranch));
       if (node instanceof SplitNode) {
         SplitNode splitNode = (SplitNode) node;
         for (int i = 0; i < splitNode.numChildren(); i++) {
-          findNodes(splitNode.getChild(i), splitNode, i,
-              found);
+          findNodes(splitNode.getChild(i), splitNode, i, found);
         }
       }
     }
@@ -466,8 +463,7 @@ final class ModelAggregatorProcessor implements Processor {
 
     // Schedule time-out thread
     ScheduledFuture<?> timeOutHandler = this.executor.schedule(new AggregationTimeOutHandler(this.splitId,
-        this.timedOutSplittingNodes),
-        this.timeOut, TimeUnit.SECONDS);
+        this.timedOutSplittingNodes), this.timeOut, TimeUnit.SECONDS);
 
     // Keep track of the splitting node information, so that we can continue the
     // split
@@ -494,10 +490,8 @@ final class ModelAggregatorProcessor implements Processor {
 
     // compare with null split
     double[] preSplitDist = activeLearningNode.getObservedClassDistribution();
-    AttributeSplitSuggestion nullSplit = new AttributeSplitSuggestion(null,
-        new double[0][], this.splitCriterion.getMeritOfSplit(
-            preSplitDist,
-            new double[][] { preSplitDist }));
+    AttributeSplitSuggestion nullSplit = new AttributeSplitSuggestion(null, new double[0][],
+        this.splitCriterion.getMeritOfSplit(preSplitDist, new double[][] { preSplitDist }));
 
     if ((bestSuggestion == null) || (nullSplit.compareTo(bestSuggestion) > 0)) {
       secondBestSuggestion = bestSuggestion;
@@ -514,12 +508,10 @@ final class ModelAggregatorProcessor implements Processor {
       shouldSplit = (bestSuggestion != null);
     } else {
       double hoeffdingBound = computeHoeffdingBound(
-          this.splitCriterion.getRangeOfMerit(activeLearningNode.getObservedClassDistribution()),
-          this.splitConfidence,
+          this.splitCriterion.getRangeOfMerit(activeLearningNode.getObservedClassDistribution()), this.splitConfidence,
           activeLearningNode.getWeightSeen());
 
-      if ((bestSuggestion.merit - secondBestSuggestion.merit > hoeffdingBound)
-          || (hoeffdingBound < tieThreshold)) {
+      if ((bestSuggestion.merit - secondBestSuggestion.merit > hoeffdingBound) || (hoeffdingBound < tieThreshold)) {
         shouldSplit = true;
       }
       // TODO: add poor attributes removal
@@ -597,8 +589,7 @@ final class ModelAggregatorProcessor implements Processor {
   private void setModelContext(InstancesHeader ih) {
     // TODO possibly refactored
     if ((ih != null) && (ih.classIndex() < 0)) {
-      throw new IllegalArgumentException(
-          "Context for a classifier must include a class to learn");
+      throw new IllegalArgumentException("Context for a classifier must include a class to learn");
     }
     // TODO: check flag for checking whether training has started or not
 


[03/17] incubator-samoa git commit: SAMOA-48: Update InstancesContentEvent as a list of InstanceContent objects

Posted by gd...@apache.org.
SAMOA-48: Update InstancesContentEvent as a list of InstanceContent objects


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

Branch: refs/heads/master
Commit: b02882efe056b14ac9596e44e313485725b02353
Parents: 23169b0
Author: Albert Bifet <ab...@gmail.com>
Authored: Fri Nov 6 12:25:12 2015 +0100
Committer: Albert Bifet <ab...@gmail.com>
Committed: Fri Nov 6 12:25:12 2015 +0100

----------------------------------------------------------------------
 .../apache/samoa/learners/InstanceContent.java  | 197 +++++++++++++++++++
 .../samoa/learners/InstanceContentEvent.java    |  61 +++---
 .../samoa/learners/InstancesContentEvent.java   | 110 ++---------
 .../classifiers/trees/FilterProcessor.java      |   8 +-
 .../trees/ModelAggregatorProcessor.java         |  76 +++----
 5 files changed, 271 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b02882ef/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContent.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContent.java b/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContent.java
new file mode 100644
index 0000000..7a49985
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContent.java
@@ -0,0 +1,197 @@
+package org.apache.samoa.learners;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+/**
+ * License
+ */
+
+import net.jcip.annotations.Immutable;
+import org.apache.samoa.core.SerializableInstance;
+import org.apache.samoa.instances.Instance;
+
+import java.io.Serializable;
+
+/**
+ * The Class InstanceContent.
+ */
+@Immutable
+final public class InstanceContent implements Serializable {
+
+  private static final long serialVersionUID = -8620668863064613841L;
+
+  private long instanceIndex;
+  private int classifierIndex;
+  private int evaluationIndex;
+  private SerializableInstance instance;
+  private boolean isTraining;
+  private boolean isTesting;
+  private boolean isLast = false;
+
+  public InstanceContent() {
+
+  }
+
+  /**
+   * Instantiates a new instance event.
+   *
+   * @param index
+   *          the index
+   * @param instance
+   *          the instance
+   * @param isTraining
+   *          the is training
+   */
+  public InstanceContent(long index, Instance instance,
+                         boolean isTraining, boolean isTesting) {
+    if (instance != null) {
+      this.instance = new SerializableInstance(instance);
+    }
+    this.instanceIndex = index;
+    this.isTraining = isTraining;
+    this.isTesting = isTesting;
+  }
+
+  /**
+   * Gets the single instance of InstanceEvent.
+   * 
+   * @return the instance.
+   */
+  public Instance getInstance() {
+    return instance;
+  }
+
+  /**
+   * Gets the instance index.
+   * 
+   * @return the index of the data vector.
+   */
+  public long getInstanceIndex() {
+    return instanceIndex;
+  }
+
+  /**
+   * Gets the class id.
+   * 
+   * @return the true class of the vector.
+   */
+  public int getClassId() {
+    // return classId;
+    return (int) instance.classValue();
+  }
+
+  /**
+   * Checks if is training.
+   * 
+   * @return true if this is training data.
+   */
+  public boolean isTraining() {
+    return isTraining;
+  }
+
+  /**
+   * Set training flag.
+   * 
+   * @param training
+   *          flag.
+   */
+  public void setTraining(boolean training) {
+    this.isTraining = training;
+  }
+
+  /**
+   * Checks if is testing.
+   * 
+   * @return true if this is testing data.
+   */
+  public boolean isTesting() {
+    return isTesting;
+  }
+
+  /**
+   * Set testing flag.
+   * 
+   * @param testing
+   *          flag.
+   */
+  public void setTesting(boolean testing) {
+    this.isTesting = testing;
+  }
+
+  /**
+   * Gets the classifier index.
+   * 
+   * @return the classifier index
+   */
+  public int getClassifierIndex() {
+    return classifierIndex;
+  }
+
+  /**
+   * Sets the classifier index.
+   * 
+   * @param classifierIndex
+   *          the new classifier index
+   */
+  public void setClassifierIndex(int classifierIndex) {
+    this.classifierIndex = classifierIndex;
+  }
+
+  /**
+   * Gets the evaluation index.
+   * 
+   * @return the evaluation index
+   */
+  public int getEvaluationIndex() {
+    return evaluationIndex;
+  }
+
+  /**
+   * Sets the evaluation index.
+   * 
+   * @param evaluationIndex
+   *          the new evaluation index
+   */
+  public void setEvaluationIndex(int evaluationIndex) {
+    this.evaluationIndex = evaluationIndex;
+  }
+
+  /**
+   * Sets the instance index.
+   *
+   * @param instanceIndex
+   *          the new evaluation index
+   */
+  public void setInstanceIndex(long instanceIndex) {
+    this.instanceIndex = instanceIndex;
+  }
+
+  public boolean isLastEvent() {
+    return isLast;
+  }
+
+  public void setLast(boolean isLast) {
+    this.isLast = isLast;
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b02882ef/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContentEvent.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContentEvent.java b/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContentEvent.java
index 69a4428..c1a1de6 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContentEvent.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContentEvent.java
@@ -30,10 +30,8 @@ import org.apache.samoa.instances.Instance;
 
 import net.jcip.annotations.Immutable;
 
-//import weka.core.Instance;
-
 /**
- * The Class InstanceEvent.
+ * The Class InstanceContentEvent.
  */
 @Immutable
 final public class InstanceContentEvent implements ContentEvent {
@@ -42,13 +40,7 @@ final public class InstanceContentEvent implements ContentEvent {
 	 * 
 	 */
   private static final long serialVersionUID = -8620668863064613845L;
-  private long instanceIndex;
-  private int classifierIndex;
-  private int evaluationIndex;
-  private SerializableInstance instance;
-  private boolean isTraining;
-  private boolean isTesting;
-  private boolean isLast = false;
+  private InstanceContent instanceContent;
 
   public InstanceContentEvent() {
 
@@ -66,12 +58,7 @@ final public class InstanceContentEvent implements ContentEvent {
    */
   public InstanceContentEvent(long index, Instance instance,
       boolean isTraining, boolean isTesting) {
-    if (instance != null) {
-      this.instance = new SerializableInstance(instance);
-    }
-    this.instanceIndex = index;
-    this.isTraining = isTraining;
-    this.isTesting = isTesting;
+    this.instanceContent = new InstanceContent(index, instance, isTraining, isTesting);
   }
 
   /**
@@ -80,7 +67,7 @@ final public class InstanceContentEvent implements ContentEvent {
    * @return the instance.
    */
   public Instance getInstance() {
-    return instance;
+    return this.instanceContent.getInstance();
   }
 
   /**
@@ -89,7 +76,7 @@ final public class InstanceContentEvent implements ContentEvent {
    * @return the index of the data vector.
    */
   public long getInstanceIndex() {
-    return instanceIndex;
+    return this.instanceContent.getInstanceIndex();
   }
 
   /**
@@ -97,9 +84,7 @@ final public class InstanceContentEvent implements ContentEvent {
    * 
    * @return the true class of the vector.
    */
-  public int getClassId() {
-    // return classId;
-    return (int) instance.classValue();
+  public int getClassId() {return this.instanceContent.getClassId();
   }
 
   /**
@@ -108,7 +93,7 @@ final public class InstanceContentEvent implements ContentEvent {
    * @return true if this is training data.
    */
   public boolean isTraining() {
-    return isTraining;
+    return this.instanceContent.isTraining();
   }
 
   /**
@@ -117,9 +102,8 @@ final public class InstanceContentEvent implements ContentEvent {
    * @param training
    *          flag.
    */
-  public void setTraining(boolean training) {
-    this.isTraining = training;
-  }
+  public void setTraining(boolean training) {this.instanceContent.setTraining(training);}
+
 
   /**
    * Checks if is testing.
@@ -127,7 +111,7 @@ final public class InstanceContentEvent implements ContentEvent {
    * @return true if this is testing data.
    */
   public boolean isTesting() {
-    return isTesting;
+    return this.instanceContent.isTesting();
   }
 
   /**
@@ -137,7 +121,7 @@ final public class InstanceContentEvent implements ContentEvent {
    *          flag.
    */
   public void setTesting(boolean testing) {
-    this.isTesting = testing;
+    this.instanceContent.setTesting(testing);
   }
 
   /**
@@ -146,7 +130,7 @@ final public class InstanceContentEvent implements ContentEvent {
    * @return the classifier index
    */
   public int getClassifierIndex() {
-    return classifierIndex;
+    return this.instanceContent.getClassifierIndex();
   }
 
   /**
@@ -156,7 +140,7 @@ final public class InstanceContentEvent implements ContentEvent {
    *          the new classifier index
    */
   public void setClassifierIndex(int classifierIndex) {
-    this.classifierIndex = classifierIndex;
+    this.instanceContent.setClassifierIndex(classifierIndex);
   }
 
   /**
@@ -165,7 +149,7 @@ final public class InstanceContentEvent implements ContentEvent {
    * @return the evaluation index
    */
   public int getEvaluationIndex() {
-    return evaluationIndex;
+    return this.instanceContent.getEvaluationIndex();
   }
 
   /**
@@ -175,7 +159,7 @@ final public class InstanceContentEvent implements ContentEvent {
    *          the new evaluation index
    */
   public void setEvaluationIndex(int evaluationIndex) {
-    this.evaluationIndex = evaluationIndex;
+    this.instanceContent.setEvaluationIndex(evaluationIndex);
   }
 
   /*
@@ -200,16 +184,23 @@ final public class InstanceContentEvent implements ContentEvent {
 
   @Override
   public void setKey(String str) {
-    this.instanceIndex = Long.parseLong(str);
+    this.instanceContent.setInstanceIndex(Long.parseLong(str));
   }
 
   @Override
   public boolean isLastEvent() {
-    return isLast;
+    return this.instanceContent.isLastEvent();
   }
 
   public void setLast(boolean isLast) {
-    this.isLast = isLast;
+    this.instanceContent.setLast(isLast);
+  }
+  /**
+   * Gets the Instance Content.
+   *
+   * @return the instance content
+   */
+  public InstanceContent getInstanceContent() {
+    return instanceContent;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b02882ef/samoa-api/src/main/java/org/apache/samoa/learners/InstancesContentEvent.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/InstancesContentEvent.java b/samoa-api/src/main/java/org/apache/samoa/learners/InstancesContentEvent.java
index 2bab8a6..5a0f0a2 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/InstancesContentEvent.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/InstancesContentEvent.java
@@ -30,10 +30,6 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.samoa.core.ContentEvent;
-import org.apache.samoa.core.SerializableInstance;
-import org.apache.samoa.instances.Instance;
-
-//import weka.core.Instance;
 
 /**
  * The Class InstanceEvent.
@@ -45,49 +41,25 @@ final public class InstancesContentEvent implements ContentEvent {
 	 * 
 	 */
   private static final long serialVersionUID = -8620668863064613845L;
-  private long instanceIndex;
-  private int classifierIndex;
-  private int evaluationIndex;
-  // private SerializableInstance instance;
-  private boolean isTraining;
-  private boolean isTesting;
-  private boolean isLast = false;
+
+  protected List<InstanceContent> instanceList = new LinkedList<InstanceContent>();
 
   public InstancesContentEvent() {
 
   }
 
   /**
-   * Instantiates a new instance event.
-   * 
-   * @param index
-   *          the index
-   * @param instance
-   *          the instance
-   * @param isTraining
-   *          the is training
+   * Instantiates a new event with a list of InstanceContent.
+   *
    */
-  public InstancesContentEvent(long index,// Instance instance,
-      boolean isTraining, boolean isTesting) {
-    /*
-     * if (instance != null) { this.instance = new
-     * SerializableInstance(instance); }
-     */
-    this.instanceIndex = index;
-    this.isTraining = isTraining;
-    this.isTesting = isTesting;
-  }
 
   public InstancesContentEvent(InstanceContentEvent event) {
-    this.instanceIndex = event.getInstanceIndex();
-    this.isTraining = event.isTraining();
-    this.isTesting = event.isTesting();
+    this.add(event.getInstanceContent());
   }
 
-  protected List<Instance> instanceList = new LinkedList<Instance>();
 
-  public void add(Instance instance) {
-    instanceList.add(new SerializableInstance(instance));
+  public void add(InstanceContent instance) {
+    instanceList.add(instance);
   }
 
   /**
@@ -95,74 +67,29 @@ final public class InstancesContentEvent implements ContentEvent {
    * 
    * @return the instance.
    */
-  public Instance[] getInstances() {
-    return instanceList.toArray(new Instance[instanceList.size()]);
+  public InstanceContent[] getInstances() {
+    return instanceList.toArray(new InstanceContent[instanceList.size()]);
   }
 
-  /**
-   * Gets the instance index.
-   * 
-   * @return the index of the data vector.
-   */
-  public long getInstanceIndex() {
-    return instanceIndex;
-  }
-
-  /**
-   * Checks if is training.
-   * 
-   * @return true if this is training data.
-   */
-  public boolean isTraining() {
-    return isTraining;
-  }
-
-  /**
-   * Checks if is testing.
-   * 
-   * @return true if this is testing data.
-   */
-  public boolean isTesting() {
-    return isTesting;
-  }
 
   /**
    * Gets the classifier index.
-   * 
+   *
    * @return the classifier index
    */
   public int getClassifierIndex() {
-    return classifierIndex;
-  }
-
-  /**
-   * Sets the classifier index.
-   * 
-   * @param classifierIndex
-   *          the new classifier index
-   */
-  public void setClassifierIndex(int classifierIndex) {
-    this.classifierIndex = classifierIndex;
+    return this.instanceList.get(0).getClassifierIndex();
   }
 
   /**
    * Gets the evaluation index.
-   * 
+   *
    * @return the evaluation index
    */
   public int getEvaluationIndex() {
-    return evaluationIndex;
+    return this.instanceList.get(0).getEvaluationIndex();
   }
 
-  /**
-   * Sets the evaluation index.
-   * 
-   * @param evaluationIndex
-   *          the new evaluation index
-   */
-  public void setEvaluationIndex(int evaluationIndex) {
-    this.evaluationIndex = evaluationIndex;
-  }
 
   /*
    * (non-Javadoc)
@@ -185,17 +112,16 @@ final public class InstancesContentEvent implements ContentEvent {
   }
 
   @Override
-  public void setKey(String str) {
-    this.instanceIndex = Long.parseLong(str);
+  public void setKey(String key) {
+    //No needed
   }
 
   @Override
   public boolean isLastEvent() {
-    return isLast;
+    return this.instanceList.get(this.instanceList.size()-1).isLastEvent();
   }
 
-  public void setLast(boolean isLast) {
-    this.isLast = isLast;
+  public List<InstanceContent> getList() {
+    return this.instanceList;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b02882ef/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java
index 0af8b93..a55b813 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java
@@ -79,16 +79,10 @@ final class FilterProcessor implements Processor {
       if (this.waitingInstances == this.batchSize || instanceContentEvent.isLastEvent()) {
         // Send Instances
         InstancesContentEvent outputEvent = new InstancesContentEvent(instanceContentEvent);
-        boolean isLastEvent = false;
         while (!this.contentEventList.isEmpty()) {
           InstanceContentEvent ice = this.contentEventList.remove(0);
-          Instance inst = ice.getInstance();
-          outputEvent.add(inst);
-          if (!isLastEvent) {
-            isLastEvent = ice.isLastEvent();
-          }
+          outputEvent.add(ice.getInstanceContent());
         }
-        outputEvent.setLast(isLastEvent);
         this.waitingInstances = 0;
         this.outputStream.put(outputEvent);
         if (this.delay > 0) {

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b02882ef/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
index 846d8e1..ce2d4c4 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
@@ -38,6 +38,7 @@ import org.apache.samoa.core.Processor;
 import org.apache.samoa.instances.Instance;
 import org.apache.samoa.instances.Instances;
 import org.apache.samoa.instances.InstancesHeader;
+import org.apache.samoa.learners.InstanceContent;
 import org.apache.samoa.learners.InstanceContentEvent;
 import org.apache.samoa.learners.InstancesContentEvent;
 import org.apache.samoa.learners.ResultContentEvent;
@@ -250,7 +251,7 @@ final class ModelAggregatorProcessor implements Processor {
    *          The associated instance content event
    * @return ResultContentEvent to be sent into Evaluator PI or other destination PI.
    */
-  private ResultContentEvent newResultContentEvent(double[] prediction, InstanceContentEvent inEvent) {
+  private ResultContentEvent newResultContentEvent(double[] prediction, InstanceContent inEvent) {
     ResultContentEvent rce = new ResultContentEvent(inEvent.getInstanceIndex(), inEvent.getInstance(),
         inEvent.getClassId(), prediction, inEvent.isLastEvent());
     rce.setClassifierIndex(this.processorId);
@@ -258,19 +259,6 @@ final class ModelAggregatorProcessor implements Processor {
     return rce;
   }
 
-  private ResultContentEvent newResultContentEvent(double[] prediction, Instance inst, InstancesContentEvent inEvent) {
-    boolean isLastEvent = false;
-    if (inEvent.isLastEvent()) {
-      Instance[] tmp = inEvent.getInstances();
-      isLastEvent = inst == tmp[tmp.length - 1]; // only set LastEvent on the last instance in the mini-batch
-    }
-    ResultContentEvent rce = new ResultContentEvent(inEvent.getInstanceIndex(), inst, (int) inst.classValue(),
-        prediction, isLastEvent);
-    rce.setClassifierIndex(this.processorId);
-    rce.setEvaluationIndex(inEvent.getEvaluationIndex());
-    return rce;
-  }
-
   private List<InstancesContentEvent> contentEventList = new LinkedList<>();
 
   /**
@@ -297,40 +285,34 @@ final class ModelAggregatorProcessor implements Processor {
   private int numBatches = 0;
 
   private void processInstances(InstancesContentEvent instContentEvent) {
+    for (InstanceContent instContent : instContentEvent.getList()) {
+      Instance inst = instContent.getInstance();
+      boolean isTesting = instContent.isTesting();
+      boolean isTraining = instContent.isTraining();
+      inst.setDataset(this.dataset);
+      // Check the instance whether it is used for testing or training
+      // boolean testAndTrain = isTraining; //Train after testing
+      double[] prediction = null;
+      if (isTesting) {
+        prediction = getVotesForInstance(inst, false);
+        this.resultStream.put(newResultContentEvent(prediction, instContent));
+      }
 
-    Instance[] instances = instContentEvent.getInstances();
-    boolean isTesting = instContentEvent.isTesting();
-    boolean isTraining = instContentEvent.isTraining();
-    for (Instance inst : instances) {
-      this.processInstance(inst, instContentEvent, isTesting, isTraining);
-    }
-  }
-
-  private void processInstance(Instance inst, InstancesContentEvent instContentEvent, boolean isTesting,
-      boolean isTraining) {
-    inst.setDataset(this.dataset);
-    // Check the instance whether it is used for testing or training
-    // boolean testAndTrain = isTraining; //Train after testing
-    double[] prediction = null;
-    if (isTesting) {
-      prediction = getVotesForInstance(inst, false);
-      this.resultStream.put(newResultContentEvent(prediction, inst, instContentEvent));
-    }
-
-    if (isTraining) {
-      trainOnInstanceImpl(inst);
-      if (this.changeDetector != null) {
-        if (prediction == null) {
-          prediction = getVotesForInstance(inst);
-        }
-        boolean correctlyClassifies = this.correctlyClassifies(inst, prediction);
-        double oldEstimation = this.changeDetector.getEstimation();
-        this.changeDetector.input(correctlyClassifies ? 0 : 1);
-        if (this.changeDetector.getEstimation() > oldEstimation) {
-          // Start a new classifier
-          logger.info("Change detected, resetting the classifier");
-          this.resetLearning();
-          this.changeDetector.resetLearning();
+      if (isTraining) {
+        trainOnInstanceImpl(inst);
+        if (this.changeDetector != null) {
+          if (prediction == null) {
+            prediction = getVotesForInstance(inst);
+          }
+          boolean correctlyClassifies = this.correctlyClassifies(inst, prediction);
+          double oldEstimation = this.changeDetector.getEstimation();
+          this.changeDetector.input(correctlyClassifies ? 0 : 1);
+          if (this.changeDetector.getEstimation() > oldEstimation) {
+            // Start a new classifier
+            logger.info("Change detected, resetting the classifier");
+            this.resetLearning();
+            this.changeDetector.resetLearning();
+          }
         }
       }
     }


[05/17] incubator-samoa git commit: SAMOA-48: Fix double processing of last even in FilterProcessor

Posted by gd...@apache.org.
SAMOA-48: Fix double processing of last even in FilterProcessor


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

Branch: refs/heads/master
Commit: b84e8ac6b2a191ac24b78082a37a0b18aa51f98f
Parents: dc99c7f
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Mon Nov 9 13:57:42 2015 +0200
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Mon Nov 9 14:41:33 2015 +0200

----------------------------------------------------------------------
 .../java/org/apache/samoa/learners/InstanceContent.java | 12 +++++++++---
 .../ensemble/PredictionCombinerProcessor.java           |  1 -
 .../learners/classifiers/trees/FilterProcessor.java     |  2 +-
 .../classifiers/trees/ModelAggregatorProcessor.java     |  5 ++---
 4 files changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b84e8ac6/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContent.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContent.java b/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContent.java
index 7a49985..c4e6b84 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContent.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/InstanceContent.java
@@ -25,6 +25,7 @@ package org.apache.samoa.learners;
  */
 
 import net.jcip.annotations.Immutable;
+
 import org.apache.samoa.core.SerializableInstance;
 import org.apache.samoa.instances.Instance;
 
@@ -61,7 +62,7 @@ final public class InstanceContent implements Serializable {
    *          the is training
    */
   public InstanceContent(long index, Instance instance,
-                         boolean isTraining, boolean isTesting) {
+      boolean isTraining, boolean isTesting) {
     if (instance != null) {
       this.instance = new SerializableInstance(instance);
     }
@@ -192,6 +193,11 @@ final public class InstanceContent implements Serializable {
     this.isLast = isLast;
   }
 
-
-
+  @Override
+  public String toString() {
+    return String
+        .format(
+            "InstanceContent [instanceIndex=%s, classifierIndex=%s, evaluationIndex=%s, instance=%s, isTraining=%s, isTesting=%s, isLast=%s]",
+            instanceIndex, classifierIndex, evaluationIndex, instance, isTraining, isTesting, isLast);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b84e8ac6/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java
index 8e10094..92a209b 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java
@@ -105,7 +105,6 @@ public class PredictionCombinerProcessor implements Processor {
     int instanceIndex = (int) inEvent.getInstanceIndex();
 
     addStatisticsForInstanceReceived(instanceIndex, inEvent.getClassifierIndex(), prediction, 1);
-
     if (hasAllVotesArrivedInstance(instanceIndex)) {
       DoubleVector combinedVote = this.mapVotesforInstanceReceived.get(instanceIndex);
       if (combinedVote == null) {

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b84e8ac6/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java
index a55b813..2c81fd0 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/FilterProcessor.java
@@ -78,7 +78,7 @@ final class FilterProcessor implements Processor {
       this.waitingInstances++;
       if (this.waitingInstances == this.batchSize || instanceContentEvent.isLastEvent()) {
         // Send Instances
-        InstancesContentEvent outputEvent = new InstancesContentEvent(instanceContentEvent);
+        InstancesContentEvent outputEvent = new InstancesContentEvent();
         while (!this.contentEventList.isEmpty()) {
           InstanceContentEvent ice = this.contentEventList.remove(0);
           outputEvent.add(ice.getInstanceContent());

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b84e8ac6/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
index ce2d4c4..cad40a7 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/trees/ModelAggregatorProcessor.java
@@ -20,6 +20,8 @@ package org.apache.samoa.learners.classifiers.trees;
  * #L%
  */
 
+import static org.apache.samoa.moa.core.Utils.maxIndex;
+
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -39,7 +41,6 @@ import org.apache.samoa.instances.Instance;
 import org.apache.samoa.instances.Instances;
 import org.apache.samoa.instances.InstancesHeader;
 import org.apache.samoa.learners.InstanceContent;
-import org.apache.samoa.learners.InstanceContentEvent;
 import org.apache.samoa.learners.InstancesContentEvent;
 import org.apache.samoa.learners.ResultContentEvent;
 import org.apache.samoa.moa.classifiers.core.AttributeSplitSuggestion;
@@ -50,8 +51,6 @@ import org.apache.samoa.topology.Stream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.samoa.moa.core.Utils.maxIndex;
-
 /**
  * Model Aggegator Processor consists of the decision tree model. It connects to local-statistic PI via attribute stream
  * and control stream. Model-aggregator PI sends the split instances via attribute stream and it sends control messages


[10/17] incubator-samoa git commit: SAMOA-51: Update Flink Module to v0.10.1 Fix #44

Posted by gd...@apache.org.
SAMOA-51: Update Flink Module to v0.10.1
Fix #44


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

Branch: refs/heads/master
Commit: b86ab83d5a8e72fdac92cbe72b7655210f4cac02
Parents: 4375bce
Author: Paris Carbone <pa...@kth.se>
Authored: Fri Nov 20 16:43:40 2015 +0100
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Sun Feb 7 16:34:13 2016 +0300

----------------------------------------------------------------------
 pom.xml                                         |   2 +-
 samoa-flink/pom.xml                             |   2 +-
 .../org/apache/samoa/flink/FlinkDoTask.java     |   7 -
 .../samoa/flink/helpers/CircleDetection.java    |  99 -------
 .../samoa/flink/helpers/CycleDetection.java     |  99 +++++++
 .../org/apache/samoa/flink/helpers/Utils.java   |   4 +-
 .../topology/impl/FlinkProcessingItem.java      |  55 ++--
 .../samoa/flink/topology/impl/FlinkStream.java  |  32 +--
 .../flink/topology/impl/FlinkTopology.java      | 278 ++++++++++---------
 9 files changed, 283 insertions(+), 295 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b86ab83d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 71b131f..af8fe98 100644
--- a/pom.xml
+++ b/pom.xml
@@ -137,7 +137,7 @@
         <miniball.version>1.0.3</miniball.version>
         <s4.version>0.6.0-incubating</s4.version>
         <samza.version>0.7.0</samza.version>
-        <flink.version>0.9.0</flink.version>
+        <flink.version>0.10.1</flink.version>
         <slf4j-log4j12.version>1.7.2</slf4j-log4j12.version>
         <slf4j-simple.version>1.7.5</slf4j-simple.version>
         <maven-surefire-plugin.version>2.18</maven-surefire-plugin.version>

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b86ab83d/samoa-flink/pom.xml
----------------------------------------------------------------------
diff --git a/samoa-flink/pom.xml b/samoa-flink/pom.xml
index f0266fa..5575643 100644
--- a/samoa-flink/pom.xml
+++ b/samoa-flink/pom.xml
@@ -70,7 +70,7 @@
         </dependency>
   		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
+			<artifactId>flink-streaming-java</artifactId>
 			<version>${flink.version}</version>
             <!--<scope>provided</scope>-->
 		</dependency>

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b86ab83d/samoa-flink/src/main/java/org/apache/samoa/flink/FlinkDoTask.java
----------------------------------------------------------------------
diff --git a/samoa-flink/src/main/java/org/apache/samoa/flink/FlinkDoTask.java b/samoa-flink/src/main/java/org/apache/samoa/flink/FlinkDoTask.java
index cd0b82c..7805371 100644
--- a/samoa-flink/src/main/java/org/apache/samoa/flink/FlinkDoTask.java
+++ b/samoa-flink/src/main/java/org/apache/samoa/flink/FlinkDoTask.java
@@ -21,16 +21,9 @@ package org.apache.samoa.flink;
  */
 
 import com.github.javacliparser.ClassOption;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 
-import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.samoa.flink.helpers.CircleDetection;
-import org.apache.samoa.flink.helpers.Utils;
 import org.apache.samoa.flink.topology.impl.FlinkComponentFactory;
-import org.apache.samoa.flink.topology.impl.FlinkProcessingItem;
-import org.apache.samoa.flink.topology.impl.FlinkStream;
 import org.apache.samoa.flink.topology.impl.FlinkTopology;
 import org.apache.samoa.tasks.Task;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b86ab83d/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/CircleDetection.java
----------------------------------------------------------------------
diff --git a/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/CircleDetection.java b/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/CircleDetection.java
deleted file mode 100644
index 400e49c..0000000
--- a/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/CircleDetection.java
+++ /dev/null
@@ -1,99 +0,0 @@
-package org.apache.samoa.flink.helpers;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Stack;
-
-/**
- * This class contains all logic needed in order to mark circles in job graphs explicitly such as 
- * in the case of Apache Flink. A circle is defined as a list of node ids ordered in topological 
- * (DFS) order.
- * 
- */
-public class CircleDetection {
-	private int[] index;
-	private int[] lowLink;
-	private int counter;
-	private Stack<Integer> stack;
-	private List<List<Integer>> scc;
-	List<Integer>[] graph;
-
-
-	public CircleDetection() {
-		stack = new Stack<Integer>();
-		scc = new ArrayList<>();
-	}
-
-	public List<List<Integer>> getCircles(List<Integer>[] adjacencyList) {
-		graph = adjacencyList;
-		index = new int[adjacencyList.length];
-		lowLink = new int[adjacencyList.length];
-		counter = 0;
-
-		//initialize index and lowLink as "undefined"(=-1)
-		for (int j = 0; j < graph.length; j++) {
-			index[j] = -1;
-			lowLink[j] = -1;
-		}
-		for (int v = 0; v < graph.length; v++) {
-			if (index[v] == -1) { //undefined.
-				findSCC(v);
-			}
-		}
-		return scc;
-	}
-
-	private void findSCC(int node) {
-		index[node] = counter;
-		lowLink[node] = counter;
-		counter++;
-		stack.push(node);
-
-		for (int neighbor : graph[node]) {
-			if (index[neighbor] == -1) {
-				findSCC(neighbor);
-				lowLink[node] = Math.min(lowLink[node], lowLink[neighbor]);
-			} else if (stack.contains(neighbor)) { //if neighbor has been already visited
-				lowLink[node] = Math.min(lowLink[node], index[neighbor]);
-				List<Integer> sccComponent = new ArrayList<Integer>();
-				int w;
-				do {
-					w = stack.pop();
-					sccComponent.add(w);
-				} while (neighbor != w);
-				//add neighbor again, just in case it is a member of another circle 
-				stack.add(neighbor); 
-				scc.add(sccComponent);
-			}
-
-		}
-		if (lowLink[node] == index[node]) {
-			int w;
-			do {
-				w = stack.pop();
-			} while (node != w);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b86ab83d/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/CycleDetection.java
----------------------------------------------------------------------
diff --git a/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/CycleDetection.java b/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/CycleDetection.java
new file mode 100644
index 0000000..9aedb25
--- /dev/null
+++ b/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/CycleDetection.java
@@ -0,0 +1,99 @@
+package org.apache.samoa.flink.helpers;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Stack;
+
+/**
+ * This class contains all logic needed in order to mark cycles in job graphs explicitly such as 
+ * in the case of Apache Flink. A cycle is defined as a list of node ids ordered in topological 
+ * (DFS) order.
+ * 
+ */
+public class CycleDetection {
+	private int[] index;
+	private int[] lowLink;
+	private int counter;
+	private Stack<Integer> stack;
+	private List<List<Integer>> scc;
+	List<Integer>[] graph;
+
+
+	public CycleDetection() {
+		stack = new Stack<>();
+		scc = new ArrayList<>();
+	}
+
+	public List<List<Integer>> getCycles(List<Integer>[] adjacencyList) {
+		graph = adjacencyList;
+		index = new int[adjacencyList.length];
+		lowLink = new int[adjacencyList.length];
+		counter = 0;
+
+		//initialize index and lowLink as "undefined"(=-1)
+		for (int j = 0; j < graph.length; j++) {
+			index[j] = -1;
+			lowLink[j] = -1;
+		}
+		for (int v = 0; v < graph.length; v++) {
+			if (index[v] == -1) { //undefined.
+				findSCC(v);
+			}
+		}
+		return scc;
+	}
+
+	private void findSCC(int node) {
+		index[node] = counter;
+		lowLink[node] = counter;
+		counter++;
+		stack.push(node);
+
+		for (int neighbor : graph[node]) {
+			if (index[neighbor] == -1) {
+				findSCC(neighbor);
+				lowLink[node] = Math.min(lowLink[node], lowLink[neighbor]);
+			} else if (stack.contains(neighbor)) { //if neighbor has been already visited
+				lowLink[node] = Math.min(lowLink[node], index[neighbor]);
+				List<Integer> sccComponent = new ArrayList<Integer>();
+				int w;
+				do {
+					w = stack.pop();
+					sccComponent.add(w);
+				} while (neighbor != w);
+				//add neighbor again, just in case it is a member of another cycle 
+				stack.add(neighbor); 
+				scc.add(sccComponent);
+			}
+
+		}
+		if (lowLink[node] == index[node]) {
+			int w;
+			do {
+				w = stack.pop();
+			} while (node != w);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b86ab83d/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/Utils.java
----------------------------------------------------------------------
diff --git a/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/Utils.java b/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/Utils.java
index 38b4bdc..ce01567 100644
--- a/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/Utils.java
+++ b/samoa-flink/src/main/java/org/apache/samoa/flink/helpers/Utils.java
@@ -32,8 +32,6 @@ import org.apache.samoa.core.ContentEvent;
 import org.apache.samoa.flink.topology.impl.SamoaType;
 import org.apache.samoa.utils.PartitioningScheme;
 
-import java.util.List;
-
 import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO;
 
 public class Utils {
@@ -45,7 +43,7 @@ public class Utils {
 			case BROADCAST:
 				return stream.broadcast();
 			case GROUP_BY_KEY:
-				return stream.groupBy(new KeySelector<SamoaType, String>() {
+				return stream.keyBy(new KeySelector<SamoaType, String>() {
 					@Override
 					public String getKey(SamoaType samoaType) throws Exception {
 						return samoaType.f0;

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b86ab83d/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkProcessingItem.java
----------------------------------------------------------------------
diff --git a/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkProcessingItem.java b/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkProcessingItem.java
index 28701df..9e3c880 100644
--- a/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkProcessingItem.java
+++ b/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkProcessingItem.java
@@ -22,20 +22,20 @@ package org.apache.samoa.flink.topology.impl;
 
 
 import com.google.common.collect.Lists;
-
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.samoa.core.ContentEvent;
 import org.apache.samoa.core.Processor;
 import org.apache.samoa.flink.helpers.Utils;
 import org.apache.samoa.topology.ProcessingItem;
 import org.apache.samoa.topology.Stream;
 import org.apache.samoa.utils.PartitioningScheme;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,9 +60,8 @@ public class FlinkProcessingItem extends AbstractUdfStreamOperator<SamoaType, Fl
 	private int parallelism;
 	private static int numberOfPIs = 0;
 	private int piID;
-	private List<Integer> circleId; //check if we can refactor this
+	private List<Integer> cycleId; //check if we can refactor this
 	private boolean onIteration;
-	//private int circleId; //check if we can refactor this
 
 	public FlinkProcessingItem(StreamExecutionEnvironment env, Processor proc) {
 		this(env, proc, 1);
@@ -79,8 +78,8 @@ public class FlinkProcessingItem extends AbstractUdfStreamOperator<SamoaType, Fl
 		this.processor = proc;
 		this.parallelism = parallelism;
 		this.piID = numberOfPIs++;
-		this.circleId = new ArrayList<Integer>() {
-		}; // if size equals 0, then it is part of no circle
+		this.cycleId = new ArrayList<Integer>() {
+		}; // if size equals 0, then it is part of no cycle
 	}
 
 	public Stream createStream() {
@@ -90,12 +89,12 @@ public class FlinkProcessingItem extends AbstractUdfStreamOperator<SamoaType, Fl
 	}
 
 	public void putToStream(ContentEvent data, Stream targetStream) {
-		output.collect(SamoaType.of(data, targetStream.getStreamId()));
+		output.collect(new StreamRecord<>(SamoaType.of(data, targetStream.getStreamId())));
 	}
-
+	
 	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
+	public void open() throws Exception {
+		super.open();
 		this.processor.onCreate(getComponentId());
 	}
 
@@ -148,8 +147,13 @@ public class FlinkProcessingItem extends AbstractUdfStreamOperator<SamoaType, Fl
 	}
 
 	@Override
-	public void processElement(SamoaType samoaType) throws Exception {
-		fun.processEvent(samoaType.f1);
+	public void processElement(StreamRecord<SamoaType> streamRecord) throws Exception {
+		fun.processEvent(streamRecord.getValue().f1);
+	}
+
+	@Override
+	public void processWatermark(Watermark watermark) throws Exception {
+
 	}
 
 	@Override
@@ -175,10 +179,6 @@ public class FlinkProcessingItem extends AbstractUdfStreamOperator<SamoaType, Fl
 		return parallelism;
 	}
 
-	public void setParallelism(int parallelism) {
-		this.parallelism = parallelism;
-	}
-
 	public List<FlinkStream> getOutputStreams() {
 		return outputStreams;
 	}
@@ -187,28 +187,24 @@ public class FlinkProcessingItem extends AbstractUdfStreamOperator<SamoaType, Fl
 		return this.outStream;
 	}
 
-	public void setOutStream(DataStream outStream) {
-		this.outStream = outStream;
-	}
-
 	@Override
 	public int getComponentId() {
 		return piID;
 	}
 
-	public boolean isPartOfCircle() {
-		return this.circleId.size() > 0;
+	public boolean isPartOfCycle() {
+		return this.cycleId.size() > 0;
 	}
 
-	public List<Integer> getCircleIds() {
-		return circleId;
+	public List<Integer> getCycleIds() {
+		return cycleId;
 	}
 
-	public void addPItoLoop(int piId) {
-		this.circleId.add(piId);
+	public void addPItoCycle(int piId) {
+		this.cycleId.add(piId);
 	}
 
-	public DataStream<SamoaType> getInStream() {
+	public DataStream<SamoaType> getDataStream() {
 		return inStream;
 	}
 
@@ -219,6 +215,7 @@ public class FlinkProcessingItem extends AbstractUdfStreamOperator<SamoaType, Fl
 	public void setOnIteration(boolean onIteration) {
 		this.onIteration = onIteration;
 	}
+	
 
 	static class SamoaDelegateFunction implements Function, Serializable {
 		private final Processor proc;

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b86ab83d/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkStream.java
----------------------------------------------------------------------
diff --git a/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkStream.java b/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkStream.java
index 31617a7..286802c 100644
--- a/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkStream.java
+++ b/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkStream.java
@@ -36,46 +36,44 @@ import java.io.Serializable;
 public class FlinkStream extends AbstractStream implements FlinkComponent, Serializable {
 
 	private static int outputCounter = 0;
-	private FlinkComponent procItem;
-	private transient DataStream<SamoaType> dataStream;
-	private int sourcePiId;
-	private String flinkStreamId;
+	private FlinkComponent sourceComponent;
+	private transient DataStream<SamoaType> filteredStream;
+	private String filterID;
 
 	public FlinkStream(FlinkComponent sourcePi) {
-		this.procItem = sourcePi;
-		this.sourcePiId = sourcePi.getComponentId();
+		this.sourceComponent = sourcePi;
 		setStreamId("stream-" + Integer.toString(outputCounter));
-		flinkStreamId = "stream-" + Integer.toString(outputCounter);
+		filterID = "stream-" + Integer.toString(outputCounter);
 		outputCounter++;
 	}
 
 	@Override
 	public void initialise() {
-		if (procItem instanceof FlinkProcessingItem) {
-			dataStream = procItem.getOutStream().filter(Utils.getFilter(getStreamId()))
-			.setParallelism(((FlinkProcessingItem) procItem).getParallelism());
+		if (sourceComponent instanceof FlinkProcessingItem) {
+			filteredStream = sourceComponent.getOutStream().filter(Utils.getFilter(getStreamId()))
+			.setParallelism(((FlinkProcessingItem) sourceComponent).getParallelism());
 		} else
-			dataStream = procItem.getOutStream();
+			filteredStream = sourceComponent.getOutStream();
 	}
 
 	@Override
 	public boolean canBeInitialised() {
-		return procItem.isInitialised();
+		return sourceComponent.isInitialised();
 	}
 
 	@Override
 	public boolean isInitialised() {
-		return dataStream != null;
+		return filteredStream != null;
 	}
 
 	@Override
 	public DataStream getOutStream() {
-		return dataStream;
+		return filteredStream;
 	}
 
 	@Override
 	public void put(ContentEvent event) {
-		((FlinkProcessingItem) procItem).putToStream(event, this);
+		((FlinkProcessingItem) sourceComponent).putToStream(event, this);
 	}
 
 	@Override
@@ -84,11 +82,11 @@ public class FlinkStream extends AbstractStream implements FlinkComponent, Seria
 	}
 
 	public int getSourcePiId() {
-		return sourcePiId;
+		return sourceComponent.getComponentId();
 	}
 
 	@Override
 	public String getStreamId() {
-		return flinkStreamId;
+		return filterID;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/b86ab83d/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkTopology.java
----------------------------------------------------------------------
diff --git a/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkTopology.java b/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkTopology.java
index 65c52c6..a09ba71 100644
--- a/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkTopology.java
+++ b/samoa-flink/src/main/java/org/apache/samoa/flink/topology/impl/FlinkTopology.java
@@ -21,17 +21,15 @@ package org.apache.samoa.flink.topology.impl;
  */
 
 
-
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-
 import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.api.datastream.IterativeDataStream;
+import org.apache.flink.streaming.api.datastream.IterativeStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.samoa.flink.helpers.CircleDetection;
-import org.apache.samoa.flink.helpers.Utils;
+import org.apache.samoa.flink.helpers.CycleDetection;
 import org.apache.samoa.topology.AbstractTopology;
 import org.apache.samoa.topology.EntranceProcessingItem;
 import org.apache.samoa.utils.PartitioningScheme;
@@ -43,144 +41,148 @@ import java.util.List;
 
 /**
  * A SAMOA topology on Apache Flink
- * 
+ * <p/>
  * A Samoa-Flink Streaming Topology is DAG of ProcessingItems encapsulated within custom operators.
- * Streams are tagged and filtered in each operator's output so they can be routed to the right 
+ * Streams are tagged and filtered in each operator's output so they can be routed to the right
  * operator respectively. Building a Flink topology from a Samoa task involves invoking all these
- * stream transformations and finally, marking and initiating loops in the graph. We have to do that
- * since Flink only allows explicit loops in the topology started with 'iterate()' and closed with
- * 'closeWith()'. Thus, when we build a flink topology we have to do it incrementally from the 
- * sources, mark loops and initialize them with explicit iterations.
- * 
+ * stream transformations and finally, marking and initiating cycles in the graph. We have to do that
+ * since Flink only allows explicit cycles in the topology started with 'iterate()' and closed with
+ * 'closeWith()'. Thus, when we build a flink topology we have to do it incrementally from the
+ * sources, mark cycles and initialize them with explicit iterations.
  */
 public class FlinkTopology extends AbstractTopology {
 
-	private static final Logger logger = LoggerFactory.getLogger(FlinkTopology.class);
-	public static StreamExecutionEnvironment env;
-	public List<List<FlinkProcessingItem>> topologyLoops = new ArrayList<>();
-	public  List<Integer> backEdges = new ArrayList<Integer>();
-
-	public FlinkTopology(String name, StreamExecutionEnvironment env) {
-		super(name);
-		this.env = env;
-	}
-
-	public StreamExecutionEnvironment getEnvironment() {
-		return env;
-	}
-	
-	public void build() {
-		markCircles();
-		for (EntranceProcessingItem src : getEntranceProcessingItems()) {
-			((FlinkEntranceProcessingItem) src).initialise();
-		}
-		initComponents(ImmutableList.copyOf(Iterables.filter(getProcessingItems(), FlinkProcessingItem.class)));
-	}
-
-	private void initComponents(ImmutableList<FlinkProcessingItem> flinkComponents) {
-		if (flinkComponents.isEmpty()) return;
-
-		for (FlinkProcessingItem comp : flinkComponents) {
-			if (comp.canBeInitialised() && !comp.isInitialised() && !comp.isPartOfCircle()) {
-				comp.initialise();
-				comp.initialiseStreams();
-
-			}//if component is part of one or more circle
-			else if (comp.isPartOfCircle() && !comp.isInitialised()) {
-				for (Integer circle : comp.getCircleIds()) {
-					//check if circle can be initialized
-					if (checkCircleReady(circle)) {
-						logger.debug("Circle: " + circle + " can be initialised");
-						initialiseCircle(circle);
-					} else {
-						logger.debug("Circle cannot be initialised");
-					}
-				}
-			}
-
-		}
-		initComponents(ImmutableList.copyOf(Iterables.filter(flinkComponents, new Predicate<FlinkProcessingItem>() {
-			@Override
-			public boolean apply(FlinkProcessingItem flinkComponent) {
-				return !flinkComponent.isInitialised();
-			}
-		})));
-	}
-
-	private void markCircles(){
-		List<FlinkProcessingItem> pis = Lists.newArrayList(Iterables.filter(getProcessingItems(), FlinkProcessingItem.class));
-		List<Integer>[] graph = new List[pis.size()];
-		FlinkProcessingItem[] processingItems = new FlinkProcessingItem[pis.size()];
-
-
-		for (int i=0;i<pis.size();i++) {
-			graph[i] = new ArrayList<Integer>();
-		}
-		//construct the graph of the topology for the Processing Items (No entrance pi is included)
-		for (FlinkProcessingItem pi: pis) {
-			processingItems[pi.getComponentId()] = pi;
-			for (Tuple3<FlinkStream, PartitioningScheme, Integer> is : pi.getInputStreams()) {
-				if (is.f2 != -1) graph[is.f2].add(pi.getComponentId());
-			}
-		}
-		for (int g=0;g<graph.length;g++)
-			logger.debug(graph[g].toString());
-
-		CircleDetection detCircles = new CircleDetection();
-		List<List<Integer>> circles = detCircles.getCircles(graph);
-
-		//update PIs, regarding being part of a circle.
-		for (List<Integer> c : circles){
-			List<FlinkProcessingItem> circle = new ArrayList<>();
-			for (Integer it : c){
-				circle.add(processingItems[it]);
-				processingItems[it].addPItoLoop(topologyLoops.size());
-			}
-			topologyLoops.add(circle);
-			backEdges.add(circle.get(0).getComponentId());
-		}
-		logger.debug("Circles detected in the topology: " + circles);
-	}
-	
-
-	private boolean checkCircleReady(int circleId) {
-
-		List<Integer> circleIds = new ArrayList<>();
-
-		for (FlinkProcessingItem pi : topologyLoops.get(circleId)) {
-			circleIds.add(pi.getComponentId());
-		}
-		//check that all incoming to the circle streams are initialised
-		for (FlinkProcessingItem procItem : topologyLoops.get(circleId)) {
-			for (Tuple3<FlinkStream, PartitioningScheme, Integer> inputStream : procItem.getInputStreams()) {
-				//if a inputStream is not initialized AND source of inputStream is not in the circle or a tail of other circle
-				if ((!inputStream.f0.isInitialised()) && (!circleIds.contains(inputStream.f2)) && (!backEdges.contains(inputStream.f2)))
-					return false;
-			}
-		}
-		return true;
-	}
-
-	private void initialiseCircle(int circleId) {
-		//get the head and tail of circle
-		FlinkProcessingItem tail = topologyLoops.get(circleId).get(0);
-		FlinkProcessingItem head = topologyLoops.get(circleId).get(topologyLoops.get(circleId).size() - 1);
-
-		//initialise source stream of the iteration, so as to use it for the iteration starting point
-		if (!head.isInitialised()) {
-			head.setOnIteration(true);
-			head.initialise();
-			head.initialiseStreams();
-		}
-
-		//initialise all nodes after head
-		for (int node = topologyLoops.get(circleId).size() - 2; node >= 0; node--) {
-			topologyLoops.get(circleId).get(node).initialise();
-			topologyLoops.get(circleId).get(node).initialiseStreams();
-		}
-
-		((IterativeDataStream) head.getInStream()).closeWith(head.getInputStreamBySourceID(tail.getComponentId()).getOutStream());
-	}
+    private static final Logger logger = LoggerFactory.getLogger(FlinkTopology.class);
+    public static StreamExecutionEnvironment env;
+    public List<List<FlinkProcessingItem>> cycles = new ArrayList<>();
+    public List<Integer> backEdges = new ArrayList<Integer>(); 
+
+    public FlinkTopology(String name, StreamExecutionEnvironment env) {
+        super(name);
+        this.env = env;
+    }
+
+    public StreamExecutionEnvironment getEnvironment() {
+        return env;
+    }
+
+    public void build() {
+        markCycles();
+        for (EntranceProcessingItem src : getEntranceProcessingItems()) {
+            ((FlinkEntranceProcessingItem) src).initialise();
+        }
+        initComponents(ImmutableList.copyOf(Iterables.filter(getProcessingItems(), FlinkProcessingItem.class)));
+    }
+
+    private void initComponents(ImmutableList<FlinkProcessingItem> flinkComponents) {
+        if (flinkComponents.isEmpty()) return;
+
+        for (FlinkProcessingItem comp : flinkComponents) {
+            if (comp.canBeInitialised() && !comp.isInitialised() && !comp.isPartOfCycle()) {
+                comp.initialise();
+                comp.initialiseStreams();
+
+            }//if component is part of one or more cycle
+            else if (comp.isPartOfCycle() && !comp.isInitialised()) {
+                for (Integer cycle : comp.getCycleIds()) {
+                    //check if cycle can be initialized
+                    if (completenessCheck(cycle)) {
+                        logger.debug("Cycle: " + cycle + " can be initialised");
+                        initializeCycle(cycle);
+                    } else {
+                        logger.debug("Cycle cannot be initialised");
+                    }
+                }
+            }
+        }
+        initComponents(ImmutableList.copyOf(Iterables.filter(flinkComponents, new Predicate<FlinkProcessingItem>() {
+            @Override
+            public boolean apply(FlinkProcessingItem flinkComponent) {
+                return !flinkComponent.isInitialised();
+            }
+        })));
+    }
+
+    /**
+     * Detects and marks all cycles and backedges needed to construct a Flink topology
+     */
+    private void markCycles() {
+        List<FlinkProcessingItem> pis = Lists.newArrayList(Iterables.filter(getProcessingItems(), FlinkProcessingItem.class));
+        List<Integer>[] graph = new List[pis.size()];
+        FlinkProcessingItem[] processingItems = new FlinkProcessingItem[pis.size()];
+
+
+        for (int i = 0; i < pis.size(); i++) {
+            graph[i] = new ArrayList<>();
+        }
+        //construct the graph of the topology for the Processing Items (No entrance pi is included)
+        for (FlinkProcessingItem pi : pis) {
+            processingItems[pi.getComponentId()] = pi;
+            for (Tuple3<FlinkStream, PartitioningScheme, Integer> is : pi.getInputStreams()) {
+                if (is.f2 != -1) graph[is.f2].add(pi.getComponentId());
+            }
+        }
+        for (int g = 0; g < graph.length; g++)
+            logger.debug(graph[g].toString());
+
+        CycleDetection detCycles = new CycleDetection();
+        List<List<Integer>> graphCycles = detCycles.getCycles(graph);
+
+        //update PIs, regarding being part of a cycle.
+        for (List<Integer> c : graphCycles) {
+            List<FlinkProcessingItem> cycle = new ArrayList<>();
+            for (Integer it : c) {
+                cycle.add(processingItems[it]);
+                processingItems[it].addPItoCycle(cycles.size());
+            }
+            cycles.add(cycle);
+            backEdges.add(cycle.get(0).getComponentId());
+        }
+        logger.debug("Cycles detected in the topology: " + graphCycles);
+    }
+
+
+    private boolean completenessCheck(int cycleId) {
+
+        List<Integer> cycleIDs = new ArrayList<>();
+
+        for (FlinkProcessingItem pi : cycles.get(cycleId)) {
+            cycleIDs.add(pi.getComponentId());
+        }
+        //check that all incoming to the cycle streams are initialised
+        for (FlinkProcessingItem procItem : cycles.get(cycleId)) {
+            for (Tuple3<FlinkStream, PartitioningScheme, Integer> inputStream : procItem.getInputStreams()) {
+                //if a inputStream is not initialized AND source of inputStream is not in the cycle or a tail of other cycle
+                if ((!inputStream.f0.isInitialised()) && (!cycleIDs.contains(inputStream.f2)) && (!backEdges.contains(inputStream.f2)))
+                    return false;
+            }
+        }
+        return true;
+    }
+
+    private void initializeCycle(int cycleID) {
+        //get the head and tail of cycle
+        FlinkProcessingItem tail = cycles.get(cycleID).get(0);
+        FlinkProcessingItem head = cycles.get(cycleID).get(cycles.get(cycleID).size() - 1);
+
+        //initialise source stream of the iteration, so as to use it for the iteration starting point
+        if (!head.isInitialised()) {
+            head.setOnIteration(true);
+            head.initialise();
+            head.initialiseStreams();
+        }
+
+        //initialise all nodes after head
+        for (int node = cycles.get(cycleID).size() - 2; node >= 0; node--) {
+            FlinkProcessingItem processingItem = cycles.get(cycleID).get(node);
+            processingItem.initialise();
+            processingItem.initialiseStreams();
+        }
+
+        SingleOutputStreamOperator backedge = (SingleOutputStreamOperator) head.getInputStreamBySourceID(tail.getComponentId()).getOutStream();
+        backedge.setParallelism(head.getParallelism());
+        ((IterativeStream) head.getDataStream()).closeWith(backedge);
+    }
 
 
 }


[17/17] incubator-samoa git commit: SAMOA-14: Move all streams under single package

Posted by gd...@apache.org.
SAMOA-14: Move all streams under single package


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

Branch: refs/heads/master
Commit: 1bd1012afb28a1c5e8607b11212ea14b57681f5f
Parents: dc2b7bc
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Sun Mar 6 15:29:05 2016 +0300
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Sun Mar 13 11:05:50 2016 +0300

----------------------------------------------------------------------
 .../apache/samoa/moa/streams/ExampleStream.java |  76 --
 .../samoa/moa/streams/InstanceStream.java       |  34 -
 .../moa/streams/clustering/ClusterEvent.java    |  49 -
 .../clustering/ClusterEventListener.java        |  29 -
 .../streams/clustering/ClusteringStream.java    |  54 --
 .../clustering/RandomRBFGeneratorEvents.java    | 968 -------------------
 .../streams/generators/HyperplaneGenerator.java | 186 ----
 .../streams/generators/RandomTreeGenerator.java | 267 -----
 .../streams/ClusteringEntranceProcessor.java    |   5 +-
 .../org/apache/samoa/streams/ExampleStream.java |  76 ++
 .../org/apache/samoa/streams/FileStream.java    |   1 -
 .../apache/samoa/streams/InstanceStream.java    |  34 +
 .../streams/PrequentialSourceProcessor.java     |   1 -
 .../org/apache/samoa/streams/StreamSource.java  |   1 -
 .../samoa/streams/StreamSourceProcessor.java    |   1 -
 .../samoa/streams/clustering/ClusterEvent.java  |  49 +
 .../clustering/ClusterEventListener.java        |  29 +
 .../streams/clustering/ClusteringStream.java    |  54 ++
 .../clustering/RandomRBFGeneratorEvents.java    | 968 +++++++++++++++++++
 .../streams/generators/HyperplaneGenerator.java | 186 ++++
 .../streams/generators/RandomTreeGenerator.java | 267 +++++
 .../samoa/tasks/ClusteringEvaluation.java       |   6 +-
 .../samoa/tasks/PrequentialEvaluation.java      |   4 +-
 .../test/java/org/apache/samoa/TestParams.java  |   6 +-
 24 files changed, 1673 insertions(+), 1678 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/moa/streams/ExampleStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/ExampleStream.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/ExampleStream.java
deleted file mode 100644
index c4ab2df..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/ExampleStream.java
+++ /dev/null
@@ -1,76 +0,0 @@
-package org.apache.samoa.moa.streams;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import org.apache.samoa.instances.InstancesHeader;
-import org.apache.samoa.moa.MOAObject;
-import org.apache.samoa.moa.core.Example;
-
-/**
- * Interface representing a data stream of examples.
- * 
- * @author Richard Kirkby (rkirkby@cs.waikato.ac.nz)
- * @version $Revision: 7 $
- */
-public interface ExampleStream<E extends Example> extends MOAObject {
-
-  /**
-   * Gets the header of this stream. This is useful to know attributes and classes. InstancesHeader is an extension of
-   * weka.Instances.
-   * 
-   * @return the header of this stream
-   */
-  public InstancesHeader getHeader();
-
-  /**
-   * Gets the estimated number of remaining instances in this stream
-   * 
-   * @return the estimated number of instances to get from this stream
-   */
-  public long estimatedRemainingInstances();
-
-  /**
-   * Gets whether this stream has more instances to output. This is useful when reading streams from files.
-   * 
-   * @return true if this stream has more instances to output
-   */
-  public boolean hasMoreInstances();
-
-  /**
-   * Gets the next example from this stream.
-   * 
-   * @return the next example of this stream
-   */
-  public E nextInstance();
-
-  /**
-   * Gets whether this stream can restart.
-   * 
-   * @return true if this stream can restart
-   */
-  public boolean isRestartable();
-
-  /**
-   * Restarts this stream. It must be similar to starting a new stream from scratch.
-   * 
-   */
-  public void restart();
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/moa/streams/InstanceStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/InstanceStream.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/InstanceStream.java
deleted file mode 100644
index fc6ed8e..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/InstanceStream.java
+++ /dev/null
@@ -1,34 +0,0 @@
-package org.apache.samoa.moa.streams;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import org.apache.samoa.instances.Instance;
-import org.apache.samoa.moa.core.Example;
-
-/**
- * Interface representing a data stream of instances.
- * 
- * @author Richard Kirkby (rkirkby@cs.waikato.ac.nz)
- * @version $Revision: 7 $
- */
-public interface InstanceStream extends ExampleStream<Example<Instance>> {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusterEvent.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusterEvent.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusterEvent.java
deleted file mode 100644
index 9aa1168..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusterEvent.java
+++ /dev/null
@@ -1,49 +0,0 @@
-package org.apache.samoa.moa.streams.clustering;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.util.EventObject;
-
-public class ClusterEvent extends EventObject {
-
-  private String type;
-  private String message;
-  private long timestamp;
-
-  public ClusterEvent(Object source, long timestamp, String type, String message) {
-    super(source);
-    this.type = type;
-    this.message = message;
-    this.timestamp = timestamp;
-  }
-
-  public String getMessage() {
-    return message;
-  }
-
-  public long getTimestamp() {
-    return timestamp;
-  }
-
-  public String getType() {
-    return type;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusterEventListener.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusterEventListener.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusterEventListener.java
deleted file mode 100644
index af7881b..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusterEventListener.java
+++ /dev/null
@@ -1,29 +0,0 @@
-package org.apache.samoa.moa.streams.clustering;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.util.EventListener;
-
-public interface ClusterEventListener extends EventListener {
-
-  public void changeCluster(ClusterEvent e);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusteringStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusteringStream.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusteringStream.java
deleted file mode 100644
index e51b437..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/ClusteringStream.java
+++ /dev/null
@@ -1,54 +0,0 @@
-package org.apache.samoa.moa.streams.clustering;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import org.apache.samoa.moa.options.AbstractOptionHandler;
-import org.apache.samoa.moa.streams.InstanceStream;
-
-import com.github.javacliparser.FloatOption;
-import com.github.javacliparser.IntOption;
-
-public abstract class ClusteringStream extends AbstractOptionHandler implements InstanceStream {
-  public IntOption decayHorizonOption = new IntOption("decayHorizon", 'h',
-      "Decay horizon", 1000, 0, Integer.MAX_VALUE);
-
-  public FloatOption decayThresholdOption = new FloatOption("decayThreshold", 't',
-      "Decay horizon threshold", 0.01, 0, 1);
-
-  public IntOption evaluationFrequencyOption = new IntOption("evaluationFrequency", 'e',
-      "Evaluation frequency", 1000, 0, Integer.MAX_VALUE);
-
-  public IntOption numAttsOption = new IntOption("numAtts", 'a',
-      "The number of attributes to generate.", 2, 0, Integer.MAX_VALUE);
-
-  public int getDecayHorizon() {
-    return decayHorizonOption.getValue();
-  }
-
-  public double getDecayThreshold() {
-    return decayThresholdOption.getValue();
-  }
-
-  public int getEvaluationFrequency() {
-    return evaluationFrequencyOption.getValue();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/RandomRBFGeneratorEvents.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/RandomRBFGeneratorEvents.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/RandomRBFGeneratorEvents.java
deleted file mode 100644
index c83f688..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/clustering/RandomRBFGeneratorEvents.java
+++ /dev/null
@@ -1,968 +0,0 @@
-package org.apache.samoa.moa.streams.clustering;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Enumeration;
-import java.util.LinkedList;
-import java.util.Random;
-import java.util.Vector;
-
-import org.apache.samoa.instances.Attribute;
-import org.apache.samoa.instances.DenseInstance;
-import org.apache.samoa.instances.Instance;
-import org.apache.samoa.instances.Instances;
-import org.apache.samoa.instances.InstancesHeader;
-import org.apache.samoa.moa.cluster.Clustering;
-import org.apache.samoa.moa.cluster.SphereCluster;
-import org.apache.samoa.moa.core.AutoExpandVector;
-import org.apache.samoa.moa.core.DataPoint;
-import org.apache.samoa.moa.core.FastVector;
-import org.apache.samoa.moa.core.InstanceExample;
-import org.apache.samoa.moa.core.ObjectRepository;
-import org.apache.samoa.moa.streams.InstanceStream;
-import org.apache.samoa.moa.tasks.TaskMonitor;
-
-import com.github.javacliparser.FlagOption;
-import com.github.javacliparser.FloatOption;
-import com.github.javacliparser.IntOption;
-
-public class RandomRBFGeneratorEvents extends ClusteringStream {
-  private transient Vector listeners;
-
-  private static final long serialVersionUID = 1L;
-
-  public IntOption modelRandomSeedOption = new IntOption("modelRandomSeed",
-      'm', "Seed for random generation of model.", 1);
-
-  public IntOption instanceRandomSeedOption = new IntOption(
-      "instanceRandomSeed", 'i',
-      "Seed for random generation of instances.", 5);
-
-  public IntOption numClusterOption = new IntOption("numCluster", 'K',
-      "The average number of centroids in the model.", 5, 1, Integer.MAX_VALUE);
-
-  public IntOption numClusterRangeOption = new IntOption("numClusterRange", 'k',
-      "Deviation of the number of centroids in the model.", 3, 0, Integer.MAX_VALUE);
-
-  public FloatOption kernelRadiiOption = new FloatOption("kernelRadius", 'R',
-      "The average radii of the centroids in the model.", 0.07, 0, 1);
-
-  public FloatOption kernelRadiiRangeOption = new FloatOption("kernelRadiusRange", 'r',
-      "Deviation of average radii of the centroids in the model.", 0, 0, 1);
-
-  public FloatOption densityRangeOption = new FloatOption("densityRange", 'd',
-      "Offset of the average weight a cluster has. Value of 0 means all cluster " +
-          "contain the same amount of points.", 0, 0, 1);
-
-  public IntOption speedOption = new IntOption("speed", 'V',
-      "Kernels move a predefined distance of 0.01 every X points", 500, 1, Integer.MAX_VALUE);
-
-  public IntOption speedRangeOption = new IntOption("speedRange", 'v',
-      "Speed/Velocity point offset", 0, 0, Integer.MAX_VALUE);
-
-  public FloatOption noiseLevelOption = new FloatOption("noiseLevel", 'N',
-      "Noise level", 0.1, 0, 1);
-
-  public FlagOption noiseInClusterOption = new FlagOption("noiseInCluster", 'n',
-      "Allow noise to be placed within a cluster");
-
-  public IntOption eventFrequencyOption = new IntOption("eventFrequency", 'E',
-      "Event frequency. Enable at least one of the events below and set numClusterRange!", 30000, 0, Integer.MAX_VALUE);
-
-  public FlagOption eventMergeSplitOption = new FlagOption("eventMergeSplitOption", 'M',
-      "Enable merging and splitting of clusters. Set eventFrequency and numClusterRange!");
-
-  public FlagOption eventDeleteCreateOption = new FlagOption("eventDeleteCreate", 'C',
-      "Enable emering and disapperaing of clusters. Set eventFrequency and numClusterRange!");
-
-  private double merge_threshold = 0.7;
-  private int kernelMovePointFrequency = 10;
-  private double maxDistanceMoveThresholdByStep = 0.01;
-  private int maxOverlapFitRuns = 50;
-  private double eventFrequencyRange = 0;
-
-  private boolean debug = false;
-
-  private AutoExpandVector<GeneratorCluster> kernels;
-  protected Random instanceRandom;
-  protected InstancesHeader streamHeader;
-  private int numGeneratedInstances;
-  private int numActiveKernels;
-  private int nextEventCounter;
-  private int nextEventChoice = -1;
-  private int clusterIdCounter;
-  private GeneratorCluster mergeClusterA;
-  private GeneratorCluster mergeClusterB;
-  private boolean mergeKernelsOverlapping = false;
-
-  private class GeneratorCluster implements Serializable {
-    // TODO: points is redundant to microclusterpoints, we need to come
-    // up with a good strategy that microclusters get updated and
-    // rebuild if needed. Idea: Sort microclusterpoints by timestamp and let
-    // microclusterdecay hold the timestamp for when the last point in a
-    // microcluster gets kicked, then we rebuild... or maybe not... could be
-    // same as searching for point to be kicked. more likely is we rebuild
-    // fewer times then insert.
-
-    private static final long serialVersionUID = -6301649898961112942L;
-
-    SphereCluster generator;
-    int kill = -1;
-    boolean merging = false;
-    double[] moveVector;
-    int totalMovementSteps;
-    int currentMovementSteps;
-    boolean isSplitting = false;
-
-    LinkedList<DataPoint> points = new LinkedList<DataPoint>();
-    ArrayList<SphereCluster> microClusters = new ArrayList<SphereCluster>();
-    ArrayList<ArrayList<DataPoint>> microClustersPoints = new ArrayList();
-    ArrayList<Integer> microClustersDecay = new ArrayList();
-
-    public GeneratorCluster(int label) {
-      boolean outofbounds = true;
-      int tryCounter = 0;
-      while (outofbounds && tryCounter < maxOverlapFitRuns) {
-        tryCounter++;
-        outofbounds = false;
-        double[] center = new double[numAttsOption.getValue()];
-        double radius = kernelRadiiOption.getValue() + (instanceRandom.nextBoolean() ? -1 : 1)
-            * kernelRadiiRangeOption.getValue() * instanceRandom.nextDouble();
-        while (radius <= 0) {
-          radius = kernelRadiiOption.getValue() + (instanceRandom.nextBoolean() ? -1 : 1)
-              * kernelRadiiRangeOption.getValue() * instanceRandom.nextDouble();
-        }
-        for (int j = 0; j < numAttsOption.getValue(); j++) {
-          center[j] = instanceRandom.nextDouble();
-          if (center[j] - radius < 0 || center[j] + radius > 1) {
-            outofbounds = true;
-            break;
-          }
-        }
-        generator = new SphereCluster(center, radius);
-      }
-      if (tryCounter < maxOverlapFitRuns) {
-        generator.setId(label);
-        double avgWeight = 1.0 / numClusterOption.getValue();
-        double weight = avgWeight + (instanceRandom.nextBoolean() ? -1 : 1) * avgWeight * densityRangeOption.getValue()
-            * instanceRandom.nextDouble();
-        generator.setWeight(weight);
-        setDesitnation(null);
-      }
-      else {
-        generator = null;
-        kill = 0;
-        System.out.println("Tried " + maxOverlapFitRuns + " times to create kernel. Reduce average radii.");
-      }
-    }
-
-    public GeneratorCluster(int label, SphereCluster cluster) {
-      this.generator = cluster;
-      cluster.setId(label);
-      setDesitnation(null);
-    }
-
-    public int getWorkID() {
-      for (int c = 0; c < kernels.size(); c++) {
-        if (kernels.get(c).equals(this))
-          return c;
-      }
-      return -1;
-    }
-
-    private void updateKernel() {
-      if (kill == 0) {
-        kernels.remove(this);
-      }
-      if (kill > 0) {
-        kill--;
-      }
-      // we could be lot more precise if we would keep track of timestamps of
-      // points
-      // then we could remove all old points and rebuild the cluster on up to
-      // date point base
-      // BUT worse the effort??? so far we just want to avoid overlap with this,
-      // so its more
-      // konservative as needed. Only needs to change when we need a thighter
-      // representation
-      for (int m = 0; m < microClusters.size(); m++) {
-        if (numGeneratedInstances - microClustersDecay.get(m) > decayHorizonOption.getValue()) {
-          microClusters.remove(m);
-          microClustersPoints.remove(m);
-          microClustersDecay.remove(m);
-        }
-      }
-
-      if (!points.isEmpty()
-          && numGeneratedInstances - points.getFirst().getTimestamp() >= decayHorizonOption.getValue()) {
-        // if(debug)
-        // System.out.println("Cleaning up macro cluster "+generator.getId());
-        points.removeFirst();
-      }
-
-    }
-
-    private void addInstance(Instance instance) {
-      DataPoint point = new DataPoint(instance, numGeneratedInstances);
-      points.add(point);
-
-      int minMicroIndex = -1;
-      double minHullDist = Double.MAX_VALUE;
-      boolean inserted = false;
-      // we favour more recently build clusters so we can remove earlier cluster
-      // sooner
-      for (int m = microClusters.size() - 1; m >= 0; m--) {
-        SphereCluster micro = microClusters.get(m);
-        double hulldist = micro.getCenterDistance(point) - micro.getRadius();
-        // point fits into existing cluster
-        if (hulldist <= 0) {
-          microClustersPoints.get(m).add(point);
-          microClustersDecay.set(m, numGeneratedInstances);
-          inserted = true;
-          break;
-        }
-        // if not, check if its at least the closest cluster
-        else {
-          if (hulldist < minHullDist) {
-            minMicroIndex = m;
-            minHullDist = hulldist;
-          }
-        }
-      }
-      // Reseting index choice for alternative cluster building
-      int alt = 1;
-      if (alt == 1)
-        minMicroIndex = -1;
-      if (!inserted) {
-        // add to closest cluster and expand cluster
-        if (minMicroIndex != -1) {
-          microClustersPoints.get(minMicroIndex).add(point);
-          // we should keep the miniball instances and just check in
-          // new points instead of rebuilding the whole thing
-          SphereCluster s = new SphereCluster(microClustersPoints.get(minMicroIndex), numAttsOption.getValue());
-          // check if current microcluster is bigger then generating cluster
-          if (s.getRadius() > generator.getRadius()) {
-            // remove previously added point
-            microClustersPoints.get(minMicroIndex).remove(microClustersPoints.get(minMicroIndex).size() - 1);
-            minMicroIndex = -1;
-          }
-          else {
-            microClusters.set(minMicroIndex, s);
-            microClustersDecay.set(minMicroIndex, numGeneratedInstances);
-          }
-        }
-        // minMicroIndex might have been reset above
-        // create new micro cluster
-        if (minMicroIndex == -1) {
-          ArrayList<DataPoint> microPoints = new ArrayList<DataPoint>();
-          microPoints.add(point);
-          SphereCluster s;
-          if (alt == 0)
-            s = new SphereCluster(microPoints, numAttsOption.getValue());
-          else
-            s = new SphereCluster(generator.getCenter(), generator.getRadius(), 1);
-
-          microClusters.add(s);
-          microClustersPoints.add(microPoints);
-          microClustersDecay.add(numGeneratedInstances);
-          int id = 0;
-          while (id < kernels.size()) {
-            if (kernels.get(id) == this)
-              break;
-            id++;
-          }
-          s.setGroundTruth(id);
-        }
-      }
-
-    }
-
-    private void move() {
-      if (currentMovementSteps < totalMovementSteps) {
-        currentMovementSteps++;
-        if (moveVector == null) {
-          return;
-        }
-        else {
-          double[] center = generator.getCenter();
-          boolean outofbounds = true;
-          while (outofbounds) {
-            double radius = generator.getRadius();
-            outofbounds = false;
-            center = generator.getCenter();
-            for (int d = 0; d < center.length; d++) {
-              center[d] += moveVector[d];
-              if (center[d] - radius < 0 || center[d] + radius > 1) {
-                outofbounds = true;
-                setDesitnation(null);
-                break;
-              }
-            }
-          }
-          generator.setCenter(center);
-        }
-      }
-      else {
-        if (!merging) {
-          setDesitnation(null);
-          isSplitting = false;
-        }
-      }
-    }
-
-    void setDesitnation(double[] destination) {
-
-      if (destination == null) {
-        destination = new double[numAttsOption.getValue()];
-        for (int j = 0; j < numAttsOption.getValue(); j++) {
-          destination[j] = instanceRandom.nextDouble();
-        }
-      }
-      double[] center = generator.getCenter();
-      int dim = center.length;
-
-      double[] v = new double[dim];
-
-      for (int d = 0; d < dim; d++) {
-        v[d] = destination[d] - center[d];
-      }
-      setMoveVector(v);
-    }
-
-    void setMoveVector(double[] vector) {
-      // we are ignoring the steps, otherwise we have to change
-      // speed of the kernels, do we want that?
-      moveVector = vector;
-      int speedInPoints = speedOption.getValue();
-      if (speedRangeOption.getValue() > 0)
-        speedInPoints += (instanceRandom.nextBoolean() ? -1 : 1) * instanceRandom.nextInt(speedRangeOption.getValue());
-      if (speedInPoints < 1)
-        speedInPoints = speedOption.getValue();
-
-      double length = 0;
-      for (int d = 0; d < moveVector.length; d++) {
-        length += Math.pow(vector[d], 2);
-      }
-      length = Math.sqrt(length);
-
-      totalMovementSteps = (int) (length / (maxDistanceMoveThresholdByStep * kernelMovePointFrequency) * speedInPoints);
-      for (int d = 0; d < moveVector.length; d++) {
-        moveVector[d] /= (double) totalMovementSteps;
-      }
-
-      currentMovementSteps = 0;
-      // if(debug){
-      // System.out.println("Setting new direction for C"+generator.getId()+": distance "
-      // +length+" in "+totalMovementSteps+" steps");
-      // }
-    }
-
-    private String tryMerging(GeneratorCluster merge) {
-      String message = "";
-      double overlapDegree = generator.overlapRadiusDegree(merge.generator);
-      if (overlapDegree > merge_threshold) {
-        SphereCluster mcluster = merge.generator;
-        double radius = Math.max(generator.getRadius(), mcluster.getRadius());
-        generator.combine(mcluster);
-
-        // //adjust radius, get bigger and bigger with high dim data
-        generator.setRadius(radius);
-        // double[] center = generator.getCenter();
-        // double[] mcenter = mcluster.getCenter();
-        // double weight = generator.getWeight();
-        // double mweight = generator.getWeight();
-        // // for (int i = 0; i < center.length; i++) {
-        // // center[i] = (center[i] * weight + mcenter[i] * mweight) / (mweight
-        // + weight);
-        // // }
-        // generator.setWeight(weight + mweight);
-        message = "Clusters merging: " + mergeClusterB.generator.getId() + " into " + mergeClusterA.generator.getId();
-
-        // clean up and restet merging stuff
-        // mark kernel so it gets killed when it doesn't contain any more
-        // instances
-        merge.kill = decayHorizonOption.getValue();
-        // set weight to 0 so no new instances will be created in the cluster
-        mcluster.setWeight(0.0);
-        normalizeWeights();
-        numActiveKernels--;
-        mergeClusterB = mergeClusterA = null;
-        merging = false;
-        mergeKernelsOverlapping = false;
-      }
-      else {
-        if (overlapDegree > 0 && !mergeKernelsOverlapping) {
-          mergeKernelsOverlapping = true;
-          message = "Merge overlapping started";
-        }
-      }
-      return message;
-    }
-
-    private String splitKernel() {
-      isSplitting = true;
-      // todo radius range
-      double radius = kernelRadiiOption.getValue();
-      double avgWeight = 1.0 / numClusterOption.getValue();
-      double weight = avgWeight + avgWeight * densityRangeOption.getValue() * instanceRandom.nextDouble();
-      SphereCluster spcluster = null;
-
-      double[] center = generator.getCenter();
-      spcluster = new SphereCluster(center, radius, weight);
-
-      if (spcluster != null) {
-        GeneratorCluster gc = new GeneratorCluster(clusterIdCounter++, spcluster);
-        gc.isSplitting = true;
-        kernels.add(gc);
-        normalizeWeights();
-        numActiveKernels++;
-        return "Split from Kernel " + generator.getId();
-      }
-      else {
-        System.out.println("Tried to split new kernel from C" + generator.getId() +
-            ". Not enough room for new cluster, decrease average radii, number of clusters or enable overlap.");
-        return "";
-      }
-    }
-
-    private String fadeOut() {
-      kill = decayHorizonOption.getValue();
-      generator.setWeight(0.0);
-      numActiveKernels--;
-      normalizeWeights();
-      return "Fading out C" + generator.getId();
-    }
-
-  }
-
-  public RandomRBFGeneratorEvents() {
-    noiseInClusterOption.set();
-    // eventDeleteCreateOption.set();
-    // eventMergeSplitOption.set();
-  }
-
-  public InstancesHeader getHeader() {
-    return streamHeader;
-  }
-
-  public long estimatedRemainingInstances() {
-    return -1;
-  }
-
-  public boolean hasMoreInstances() {
-    return true;
-  }
-
-  public boolean isRestartable() {
-    return true;
-  }
-
-  @Override
-  public void prepareForUseImpl(TaskMonitor monitor, ObjectRepository repository) {
-    monitor.setCurrentActivity("Preparing random RBF...", -1.0);
-    generateHeader();
-    restart();
-  }
-
-  public void restart() {
-    instanceRandom = new Random(instanceRandomSeedOption.getValue());
-    nextEventCounter = eventFrequencyOption.getValue();
-    nextEventChoice = getNextEvent();
-    numActiveKernels = 0;
-    numGeneratedInstances = 0;
-    clusterIdCounter = 0;
-    mergeClusterA = mergeClusterB = null;
-    kernels = new AutoExpandVector<GeneratorCluster>();
-
-    initKernels();
-  }
-
-  protected void generateHeader() { // 2013/06/02: Noise label
-    ArrayList<Attribute> attributes = new ArrayList<Attribute>();
-    for (int i = 0; i < this.numAttsOption.getValue(); i++) {
-      attributes.add(new Attribute("att" + (i + 1)));
-    }
-
-    ArrayList<String> classLabels = new ArrayList<String>();
-    for (int i = 0; i < this.numClusterOption.getValue(); i++) {
-      classLabels.add("class" + (i + 1));
-    }
-    if (noiseLevelOption.getValue() > 0)
-      classLabels.add("noise"); // The last label = "noise"
-
-    attributes.add(new Attribute("class", classLabels));
-    streamHeader = new InstancesHeader(new Instances(getCLICreationString(InstanceStream.class), attributes, 0));
-    streamHeader.setClassIndex(streamHeader.numAttributes() - 1);
-  }
-
-  protected void initKernels() {
-    for (int i = 0; i < numClusterOption.getValue(); i++) {
-      kernels.add(new GeneratorCluster(clusterIdCounter));
-      numActiveKernels++;
-      clusterIdCounter++;
-    }
-    normalizeWeights();
-  }
-
-  public InstanceExample nextInstance() {
-    numGeneratedInstances++;
-    eventScheduler();
-
-    // make room for the classlabel
-    double[] values_new = new double[numAttsOption.getValue()]; // +1
-    double[] values = null;
-    int clusterChoice = -1;
-
-    if (instanceRandom.nextDouble() > noiseLevelOption.getValue()) {
-      clusterChoice = chooseWeightedElement();
-      values = kernels.get(clusterChoice).generator.sample(instanceRandom).toDoubleArray();
-    }
-    else {
-      // get ranodm noise point
-      values = getNoisePoint();
-    }
-
-    if (Double.isNaN(values[0])) {
-      System.out.println("Instance corrupted:" + numGeneratedInstances);
-    }
-    System.arraycopy(values, 0, values_new, 0, values.length);
-
-    Instance inst = new DenseInstance(1.0, values_new);
-    inst.setDataset(getHeader());
-    if (clusterChoice == -1) {
-      // 2013/06/02 (Yunsu Kim)
-      // Noise instance has the last class value instead of "-1"
-      // Preventing ArrayIndexOutOfBoundsException in WriteStreamToARFFFile
-      inst.setClassValue(numClusterOption.getValue());
-    }
-    else {
-      inst.setClassValue(kernels.get(clusterChoice).generator.getId());
-      // Do we need micro cluster representation if have overlapping clusters?
-      // if(!overlappingOption.isSet())
-      kernels.get(clusterChoice).addInstance(inst);
-    }
-    // System.out.println(numGeneratedInstances+": Overlap is"+updateOverlaps());
-
-    return new InstanceExample(inst);
-  }
-
-  public Clustering getGeneratingClusters() {
-    Clustering clustering = new Clustering();
-    for (int c = 0; c < kernels.size(); c++) {
-      clustering.add(kernels.get(c).generator);
-    }
-    return clustering;
-  }
-
-  public Clustering getMicroClustering() {
-    Clustering clustering = new Clustering();
-    int id = 0;
-
-    for (int c = 0; c < kernels.size(); c++) {
-      for (int m = 0; m < kernels.get(c).microClusters.size(); m++) {
-        kernels.get(c).microClusters.get(m).setId(id);
-        kernels.get(c).microClusters.get(m).setGroundTruth(kernels.get(c).generator.getId());
-        clustering.add(kernels.get(c).microClusters.get(m));
-        id++;
-      }
-    }
-
-    // System.out.println("numMicroKernels "+clustering.size());
-    return clustering;
-  }
-
-  /**************************** EVENTS ******************************************/
-  private void eventScheduler() {
-
-    for (int i = 0; i < kernels.size(); i++) {
-      kernels.get(i).updateKernel();
-    }
-
-    nextEventCounter--;
-    // only move kernels every 10 points, performance reasons????
-    // should this be randomized as well???
-    if (nextEventCounter % kernelMovePointFrequency == 0) {
-      // move kernels
-      for (int i = 0; i < kernels.size(); i++) {
-        kernels.get(i).move();
-        // overlapControl();
-      }
-    }
-
-    if (eventFrequencyOption.getValue() == 0) {
-      return;
-    }
-
-    String type = "";
-    String message = "";
-    boolean eventFinished = false;
-    switch (nextEventChoice) {
-    case 0:
-      if (numActiveKernels > 1 && numActiveKernels > numClusterOption.getValue() - numClusterRangeOption.getValue()) {
-        message = mergeKernels(nextEventCounter);
-        type = "Merge";
-      }
-      if (mergeClusterA == null && mergeClusterB == null && message.startsWith("Clusters merging")) {
-        eventFinished = true;
-      }
-      break;
-    case 1:
-      if (nextEventCounter <= 0) {
-        if (numActiveKernels < numClusterOption.getValue() + numClusterRangeOption.getValue()) {
-          type = "Split";
-          message = splitKernel();
-        }
-        eventFinished = true;
-      }
-      break;
-    case 2:
-      if (nextEventCounter <= 0) {
-        if (numActiveKernels > 1 && numActiveKernels > numClusterOption.getValue() - numClusterRangeOption.getValue()) {
-          message = fadeOut();
-          type = "Delete";
-        }
-        eventFinished = true;
-      }
-      break;
-    case 3:
-      if (nextEventCounter <= 0) {
-        if (numActiveKernels < numClusterOption.getValue() + numClusterRangeOption.getValue()) {
-          message = fadeIn();
-          type = "Create";
-        }
-        eventFinished = true;
-      }
-      break;
-
-    }
-    if (eventFinished) {
-      nextEventCounter = (int) (eventFrequencyOption.getValue() + (instanceRandom.nextBoolean() ? -1 : 1)
-          * eventFrequencyOption.getValue() * eventFrequencyRange * instanceRandom.nextDouble());
-      nextEventChoice = getNextEvent();
-      // System.out.println("Next event choice: "+nextEventChoice);
-    }
-    if (!message.isEmpty()) {
-      message += " (numKernels = " + numActiveKernels + " at " + numGeneratedInstances + ")";
-      if (!type.equals("Merge") || message.startsWith("Clusters merging"))
-        fireClusterChange(numGeneratedInstances, type, message);
-    }
-  }
-
-  private int getNextEvent() {
-    int choice = -1;
-    boolean lowerLimit = numActiveKernels <= numClusterOption.getValue() - numClusterRangeOption.getValue();
-    boolean upperLimit = numActiveKernels >= numClusterOption.getValue() + numClusterRangeOption.getValue();
-
-    if (!lowerLimit || !upperLimit) {
-      int mode = -1;
-      if (eventDeleteCreateOption.isSet() && eventMergeSplitOption.isSet()) {
-        mode = instanceRandom.nextInt(2);
-      }
-
-      if (mode == 0 || (mode == -1 && eventMergeSplitOption.isSet())) {
-        // have we reached a limit? if not free choice
-        if (!lowerLimit && !upperLimit)
-          choice = instanceRandom.nextInt(2);
-        else
-        // we have a limit. if lower limit, choose split
-        if (lowerLimit)
-          choice = 1;
-        // otherwise we reached upper level, choose merge
-        else
-          choice = 0;
-      }
-
-      if (mode == 1 || (mode == -1 && eventDeleteCreateOption.isSet())) {
-        // have we reached a limit? if not free choice
-        if (!lowerLimit && !upperLimit)
-          choice = instanceRandom.nextInt(2) + 2;
-        else
-        // we have a limit. if lower limit, choose create
-        if (lowerLimit)
-          choice = 3;
-        // otherwise we reached upper level, choose delete
-        else
-          choice = 2;
-      }
-    }
-
-    return choice;
-  }
-
-  private String fadeOut() {
-    int id = instanceRandom.nextInt(kernels.size());
-    while (kernels.get(id).kill != -1)
-      id = instanceRandom.nextInt(kernels.size());
-
-    String message = kernels.get(id).fadeOut();
-    return message;
-  }
-
-  private String fadeIn() {
-    GeneratorCluster gc = new GeneratorCluster(clusterIdCounter++);
-    kernels.add(gc);
-    numActiveKernels++;
-    normalizeWeights();
-    return "Creating new cluster";
-  }
-
-  private String changeWeight(boolean increase) {
-    double changeRate = 0.1;
-    int id = instanceRandom.nextInt(kernels.size());
-    while (kernels.get(id).kill != -1)
-      id = instanceRandom.nextInt(kernels.size());
-
-    int sign = 1;
-    if (!increase)
-      sign = -1;
-    double weight_old = kernels.get(id).generator.getWeight();
-    double delta = sign * numActiveKernels * instanceRandom.nextDouble() * changeRate;
-    kernels.get(id).generator.setWeight(weight_old + delta);
-
-    normalizeWeights();
-
-    String message;
-    if (increase)
-      message = "Increase ";
-    else
-      message = "Decrease ";
-    message += " weight on Cluster " + id + " from " + weight_old + " to " + (weight_old + delta);
-    return message;
-
-  }
-
-  private String changeRadius(boolean increase) {
-    double maxChangeRate = 0.1;
-    int id = instanceRandom.nextInt(kernels.size());
-    while (kernels.get(id).kill != -1)
-      id = instanceRandom.nextInt(kernels.size());
-
-    int sign = 1;
-    if (!increase)
-      sign = -1;
-
-    double r_old = kernels.get(id).generator.getRadius();
-    double r_new = r_old + sign * r_old * instanceRandom.nextDouble() * maxChangeRate;
-    if (r_new >= 0.5)
-      return "Radius to big";
-    kernels.get(id).generator.setRadius(r_new);
-
-    String message;
-    if (increase)
-      message = "Increase ";
-    else
-      message = "Decrease ";
-    message += " radius on Cluster " + id + " from " + r_old + " to " + r_new;
-    return message;
-  }
-
-  private String splitKernel() {
-    int id = instanceRandom.nextInt(kernels.size());
-    while (kernels.get(id).kill != -1)
-      id = instanceRandom.nextInt(kernels.size());
-
-    String message = kernels.get(id).splitKernel();
-
-    return message;
-  }
-
-  private String mergeKernels(int steps) {
-    if (numActiveKernels > 1 && ((mergeClusterA == null && mergeClusterB == null))) {
-
-      // choose clusters to merge
-      double diseredDist = steps / speedOption.getValue() * maxDistanceMoveThresholdByStep;
-      double minDist = Double.MAX_VALUE;
-      // System.out.println("DisredDist:"+(2*diseredDist));
-      for (int i = 0; i < kernels.size(); i++) {
-        for (int j = 0; j < i; j++) {
-          if (kernels.get(i).kill != -1 || kernels.get(j).kill != -1) {
-            continue;
-          }
-          else {
-            double kernelDist = kernels.get(i).generator.getCenterDistance(kernels.get(j).generator);
-            double d = kernelDist - 2 * diseredDist;
-            // System.out.println("Dist:"+i+" / "+j+" "+d);
-            if (Math.abs(d) < minDist &&
-                (minDist != Double.MAX_VALUE || d > 0 || Math.abs(d) < 0.001)) {
-              minDist = Math.abs(d);
-              mergeClusterA = kernels.get(i);
-              mergeClusterB = kernels.get(j);
-            }
-          }
-        }
-      }
-
-      if (mergeClusterA != null && mergeClusterB != null) {
-        double[] merge_point = mergeClusterA.generator.getCenter();
-        double[] v = mergeClusterA.generator.getDistanceVector(mergeClusterB.generator);
-        for (int i = 0; i < v.length; i++) {
-          merge_point[i] = merge_point[i] + v[i] * 0.5;
-        }
-
-        mergeClusterA.merging = true;
-        mergeClusterB.merging = true;
-        mergeClusterA.setDesitnation(merge_point);
-        mergeClusterB.setDesitnation(merge_point);
-
-        if (debug) {
-          System.out.println("Center1" + Arrays.toString(mergeClusterA.generator.getCenter()));
-          System.out.println("Center2" + Arrays.toString(mergeClusterB.generator.getCenter()));
-          System.out.println("Vector" + Arrays.toString(v));
-
-          System.out.println("Try to merge cluster " + mergeClusterA.generator.getId() +
-              " into " + mergeClusterB.generator.getId() +
-              " at " + Arrays.toString(merge_point) +
-              " time " + numGeneratedInstances);
-        }
-        return "Init merge";
-      }
-    }
-
-    if (mergeClusterA != null && mergeClusterB != null) {
-
-      // movekernels will move the kernels close to each other,
-      // we just need to check and merge here if they are close enough
-      return mergeClusterA.tryMerging(mergeClusterB);
-    }
-
-    return "";
-  }
-
-  /************************* TOOLS **************************************/
-
-  public void getDescription(StringBuilder sb, int indent) {
-
-  }
-
-  private double[] getNoisePoint() {
-    double[] sample = new double[numAttsOption.getValue()];
-    boolean incluster = true;
-    int counter = 20;
-    while (incluster) {
-      for (int j = 0; j < numAttsOption.getValue(); j++) {
-        sample[j] = instanceRandom.nextDouble();
-      }
-      incluster = false;
-      if (!noiseInClusterOption.isSet() && counter > 0) {
-        counter--;
-        for (int c = 0; c < kernels.size(); c++) {
-          for (int m = 0; m < kernels.get(c).microClusters.size(); m++) {
-            Instance inst = new DenseInstance(1, sample);
-            if (kernels.get(c).microClusters.get(m).getInclusionProbability(inst) > 0) {
-              incluster = true;
-              break;
-            }
-          }
-          if (incluster)
-            break;
-        }
-      }
-    }
-
-    // double [] sample = new double [numAttsOption.getValue()];
-    // for (int j = 0; j < numAttsOption.getValue(); j++) {
-    // sample[j] = instanceRandom.nextDouble();
-    // }
-
-    return sample;
-  }
-
-  private int chooseWeightedElement() {
-    double r = instanceRandom.nextDouble();
-
-    // Determine index of choosen element
-    int i = 0;
-    while (r > 0.0) {
-      r -= kernels.get(i).generator.getWeight();
-      i++;
-    }
-    --i; // Overcounted once
-    // System.out.println(i);
-    return i;
-  }
-
-  private void normalizeWeights() {
-    double sumWeights = 0.0;
-    for (int i = 0; i < kernels.size(); i++) {
-      sumWeights += kernels.get(i).generator.getWeight();
-    }
-    for (int i = 0; i < kernels.size(); i++) {
-      kernels.get(i).generator.setWeight(kernels.get(i).generator.getWeight() / sumWeights);
-    }
-  }
-
-  /*************** EVENT Listener *********************/
-  // should go into the superclass of the generator, create new one for cluster
-  // streams?
-
-  /** Add a listener */
-  synchronized public void addClusterChangeListener(ClusterEventListener l) {
-    if (listeners == null)
-      listeners = new Vector();
-    listeners.addElement(l);
-  }
-
-  /** Remove a listener */
-  synchronized public void removeClusterChangeListener(ClusterEventListener l) {
-    if (listeners == null)
-      listeners = new Vector();
-    listeners.removeElement(l);
-  }
-
-  /** Fire a ClusterChangeEvent to all registered listeners */
-  protected void fireClusterChange(long timestamp, String type, String message) {
-    // if we have no listeners, do nothing...
-    if (listeners != null && !listeners.isEmpty()) {
-      // create the event object to send
-      ClusterEvent event =
-          new ClusterEvent(this, timestamp, type, message);
-
-      // make a copy of the listener list in case
-      // anyone adds/removes listeners
-      Vector targets;
-      synchronized (this) {
-        targets = (Vector) listeners.clone();
-      }
-
-      // walk through the listener list and
-      // call the sunMoved method in each
-      Enumeration e = targets.elements();
-      while (e.hasMoreElements()) {
-        ClusterEventListener l = (ClusterEventListener) e.nextElement();
-        l.changeCluster(event);
-
-      }
-    }
-  }
-
-  @Override
-  public String getPurposeString() {
-    return "Generates a random radial basis function stream.";
-  }
-
-  public String getParameterString() {
-    return "";
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/moa/streams/generators/HyperplaneGenerator.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/generators/HyperplaneGenerator.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/generators/HyperplaneGenerator.java
deleted file mode 100644
index b05fa1b..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/generators/HyperplaneGenerator.java
+++ /dev/null
@@ -1,186 +0,0 @@
-package org.apache.samoa.moa.streams.generators;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.util.Random;
-
-import org.apache.samoa.instances.Attribute;
-import org.apache.samoa.instances.DenseInstance;
-import org.apache.samoa.instances.Instance;
-import org.apache.samoa.instances.Instances;
-import org.apache.samoa.instances.InstancesHeader;
-import org.apache.samoa.moa.core.Example;
-import org.apache.samoa.moa.core.FastVector;
-import org.apache.samoa.moa.core.InstanceExample;
-import org.apache.samoa.moa.core.ObjectRepository;
-import org.apache.samoa.moa.options.AbstractOptionHandler;
-import org.apache.samoa.moa.streams.InstanceStream;
-import org.apache.samoa.moa.tasks.TaskMonitor;
-
-import com.github.javacliparser.FloatOption;
-import com.github.javacliparser.IntOption;
-
-/**
- * Stream generator for Hyperplane data stream.
- * 
- * @author Albert Bifet (abifet at cs dot waikato dot ac dot nz)
- * @version $Revision: 7 $
- */
-public class HyperplaneGenerator extends AbstractOptionHandler implements InstanceStream {
-
-  @Override
-  public String getPurposeString() {
-    return "Generates a problem of predicting class of a rotating hyperplane.";
-  }
-
-  private static final long serialVersionUID = 1L;
-
-  public IntOption instanceRandomSeedOption = new IntOption("instanceRandomSeed", 'i',
-      "Seed for random generation of instances.", 1);
-
-  public IntOption numClassesOption = new IntOption("numClasses", 'c', "The number of classes to generate.", 2, 2,
-      Integer.MAX_VALUE);
-
-  public IntOption numAttsOption = new IntOption("numAtts", 'a', "The number of attributes to generate.", 10, 0,
-      Integer.MAX_VALUE);
-
-  public IntOption numDriftAttsOption = new IntOption("numDriftAtts", 'k', "The number of attributes with drift.", 2,
-      0, Integer.MAX_VALUE);
-
-  public FloatOption magChangeOption = new FloatOption("magChange", 't', "Magnitude of the change for every example",
-      0.0, 0.0, 1.0);
-
-  public IntOption noisePercentageOption = new IntOption("noisePercentage", 'n',
-      "Percentage of noise to add to the data.", 5, 0, 100);
-
-  public IntOption sigmaPercentageOption = new IntOption("sigmaPercentage", 's',
-      "Percentage of probability that the direction of change is reversed.", 10,
-      0, 100);
-
-  protected InstancesHeader streamHeader;
-
-  protected Random instanceRandom;
-
-  protected double[] weights;
-
-  protected int[] sigma;
-
-  public int numberInstance;
-
-  @Override
-  protected void prepareForUseImpl(TaskMonitor monitor, ObjectRepository repository) {
-    monitor.setCurrentActivity("Preparing hyperplane...", -1.0);
-    generateHeader();
-    restart();
-  }
-
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  protected void generateHeader() {
-    FastVector attributes = new FastVector();
-    for (int i = 0; i < this.numAttsOption.getValue(); i++) {
-      attributes.addElement(new Attribute("att" + (i + 1)));
-    }
-
-    FastVector classLabels = new FastVector();
-    for (int i = 0; i < this.numClassesOption.getValue(); i++) {
-      classLabels.addElement("class" + (i + 1));
-    }
-    attributes.addElement(new Attribute("class", classLabels));
-    this.streamHeader = new InstancesHeader(new Instances(getCLICreationString(InstanceStream.class), attributes, 0));
-    this.streamHeader.setClassIndex(this.streamHeader.numAttributes() - 1);
-  }
-
-  @Override
-  public long estimatedRemainingInstances() {
-    return -1;
-  }
-
-  @Override
-  public InstancesHeader getHeader() {
-    return this.streamHeader;
-  }
-
-  @Override
-  public boolean hasMoreInstances() {
-    return true;
-  }
-
-  @Override
-  public boolean isRestartable() {
-    return true;
-  }
-
-  @Override
-  public Example<Instance> nextInstance() {
-
-    int numAtts = this.numAttsOption.getValue();
-    double[] attVals = new double[numAtts + 1];
-    double sum = 0.0;
-    double sumWeights = 0.0;
-    for (int i = 0; i < numAtts; i++) {
-      attVals[i] = this.instanceRandom.nextDouble();
-      sum += this.weights[i] * attVals[i];
-      sumWeights += this.weights[i];
-    }
-    int classLabel;
-    if (sum >= sumWeights * 0.5) {
-      classLabel = 1;
-    } else {
-      classLabel = 0;
-    }
-    // Add Noise
-    if ((1 + (this.instanceRandom.nextInt(100))) <= this.noisePercentageOption.getValue()) {
-      classLabel = (classLabel == 0 ? 1 : 0);
-    }
-
-    Instance inst = new DenseInstance(1.0, attVals);
-    inst.setDataset(getHeader());
-    inst.setClassValue(classLabel);
-    addDrift();
-    return new InstanceExample(inst);
-  }
-
-  private void addDrift() {
-    for (int i = 0; i < this.numDriftAttsOption.getValue(); i++) {
-      this.weights[i] += (double) ((double) sigma[i]) * ((double) this.magChangeOption.getValue());
-      if (// this.weights[i] >= 1.0 || this.weights[i] <= 0.0 ||
-      (1 + (this.instanceRandom.nextInt(100))) <= this.sigmaPercentageOption.getValue()) {
-        this.sigma[i] *= -1;
-      }
-    }
-  }
-
-  @Override
-  public void restart() {
-    this.instanceRandom = new Random(this.instanceRandomSeedOption.getValue());
-    this.weights = new double[this.numAttsOption.getValue()];
-    this.sigma = new int[this.numAttsOption.getValue()];
-    for (int i = 0; i < this.numAttsOption.getValue(); i++) {
-      this.weights[i] = this.instanceRandom.nextDouble();
-      this.sigma[i] = (i < this.numDriftAttsOption.getValue() ? 1 : 0);
-    }
-  }
-
-  @Override
-  public void getDescription(StringBuilder sb, int indent) {
-    // TODO Auto-generated method stub
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/moa/streams/generators/RandomTreeGenerator.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/generators/RandomTreeGenerator.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/generators/RandomTreeGenerator.java
deleted file mode 100644
index 69100aa..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/generators/RandomTreeGenerator.java
+++ /dev/null
@@ -1,267 +0,0 @@
-package org.apache.samoa.moa.streams.generators;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Random;
-
-import org.apache.samoa.instances.Attribute;
-import org.apache.samoa.instances.DenseInstance;
-import org.apache.samoa.instances.Instance;
-import org.apache.samoa.instances.Instances;
-import org.apache.samoa.instances.InstancesHeader;
-import org.apache.samoa.moa.core.FastVector;
-import org.apache.samoa.moa.core.InstanceExample;
-import org.apache.samoa.moa.core.ObjectRepository;
-import org.apache.samoa.moa.options.AbstractOptionHandler;
-import org.apache.samoa.moa.streams.InstanceStream;
-import org.apache.samoa.moa.tasks.TaskMonitor;
-
-import com.github.javacliparser.FloatOption;
-import com.github.javacliparser.IntOption;
-
-/**
- * Stream generator for a stream based on a randomly generated tree..
- * 
- * @author Richard Kirkby (rkirkby@cs.waikato.ac.nz)
- * @version $Revision: 7 $
- */
-public class RandomTreeGenerator extends AbstractOptionHandler implements InstanceStream {
-
-  @Override
-  public String getPurposeString() {
-    return "Generates a stream based on a randomly generated tree.";
-  }
-
-  private static final long serialVersionUID = 1L;
-
-  public IntOption treeRandomSeedOption = new IntOption("treeRandomSeed",
-      'r', "Seed for random generation of tree.", 1);
-
-  public IntOption instanceRandomSeedOption = new IntOption(
-      "instanceRandomSeed", 'i',
-      "Seed for random generation of instances.", 1);
-
-  public IntOption numClassesOption = new IntOption("numClasses", 'c',
-      "The number of classes to generate.", 2, 2, Integer.MAX_VALUE);
-
-  public IntOption numNominalsOption = new IntOption("numNominals", 'o',
-      "The number of nominal attributes to generate.", 5, 0,
-      Integer.MAX_VALUE);
-
-  public IntOption numNumericsOption = new IntOption("numNumerics", 'u',
-      "The number of numeric attributes to generate.", 5, 0,
-      Integer.MAX_VALUE);
-
-  public IntOption numValsPerNominalOption = new IntOption(
-      "numValsPerNominal", 'v',
-      "The number of values to generate per nominal attribute.", 5, 2,
-      Integer.MAX_VALUE);
-
-  public IntOption maxTreeDepthOption = new IntOption("maxTreeDepth", 'd',
-      "The maximum depth of the tree concept.", 5, 0, Integer.MAX_VALUE);
-
-  public IntOption firstLeafLevelOption = new IntOption(
-      "firstLeafLevel",
-      'l',
-      "The first level of the tree above maxTreeDepth that can have leaves.",
-      3, 0, Integer.MAX_VALUE);
-
-  public FloatOption leafFractionOption = new FloatOption("leafFraction",
-      'f',
-      "The fraction of leaves per level from firstLeafLevel onwards.",
-      0.15, 0.0, 1.0);
-
-  protected static class Node implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    public int classLabel;
-
-    public int splitAttIndex;
-
-    public double splitAttValue;
-
-    public Node[] children;
-  }
-
-  protected Node treeRoot;
-
-  protected InstancesHeader streamHeader;
-
-  protected Random instanceRandom;
-
-  @Override
-  public void prepareForUseImpl(TaskMonitor monitor,
-      ObjectRepository repository) {
-    monitor.setCurrentActivity("Preparing random tree...", -1.0);
-    generateHeader();
-    generateRandomTree();
-    restart();
-  }
-
-  @Override
-  public long estimatedRemainingInstances() {
-    return -1;
-  }
-
-  @Override
-  public boolean isRestartable() {
-    return true;
-  }
-
-  @Override
-  public void restart() {
-    this.instanceRandom = new Random(this.instanceRandomSeedOption.getValue());
-  }
-
-  @Override
-  public InstancesHeader getHeader() {
-    return this.streamHeader;
-  }
-
-  @Override
-  public boolean hasMoreInstances() {
-    return true;
-  }
-
-  @Override
-  public InstanceExample nextInstance() {
-    double[] attVals = new double[this.numNominalsOption.getValue()
-        + this.numNumericsOption.getValue()];
-    InstancesHeader header = getHeader();
-    Instance inst = new DenseInstance(header.numAttributes());
-    for (int i = 0; i < attVals.length; i++) {
-      attVals[i] = i < this.numNominalsOption.getValue() ? this.instanceRandom.nextInt(this.numValsPerNominalOption
-          .getValue())
-          : this.instanceRandom.nextDouble();
-      inst.setValue(i, attVals[i]);
-    }
-    inst.setDataset(header);
-    inst.setClassValue(classifyInstance(this.treeRoot, attVals));
-    return new InstanceExample(inst);
-  }
-
-  protected int classifyInstance(Node node, double[] attVals) {
-    if (node.children == null) {
-      return node.classLabel;
-    }
-    if (node.splitAttIndex < this.numNominalsOption.getValue()) {
-      return classifyInstance(
-          node.children[(int) attVals[node.splitAttIndex]], attVals);
-    }
-    return classifyInstance(
-        node.children[attVals[node.splitAttIndex] < node.splitAttValue ? 0
-            : 1], attVals);
-  }
-
-  protected void generateHeader() {
-    FastVector<Attribute> attributes = new FastVector<>();
-    FastVector<String> nominalAttVals = new FastVector<>();
-    for (int i = 0; i < this.numValsPerNominalOption.getValue(); i++) {
-      nominalAttVals.addElement("value" + (i + 1));
-    }
-    for (int i = 0; i < this.numNominalsOption.getValue(); i++) {
-      attributes.addElement(new Attribute("nominal" + (i + 1),
-          nominalAttVals));
-    }
-    for (int i = 0; i < this.numNumericsOption.getValue(); i++) {
-      attributes.addElement(new Attribute("numeric" + (i + 1)));
-    }
-    FastVector<String> classLabels = new FastVector<>();
-    for (int i = 0; i < this.numClassesOption.getValue(); i++) {
-      classLabels.addElement("class" + (i + 1));
-    }
-    attributes.addElement(new Attribute("class", classLabels));
-    this.streamHeader = new InstancesHeader(new Instances(
-        getCLICreationString(InstanceStream.class), attributes, 0));
-    this.streamHeader.setClassIndex(this.streamHeader.numAttributes() - 1);
-  }
-
-  protected void generateRandomTree() {
-    Random treeRand = new Random(this.treeRandomSeedOption.getValue());
-    ArrayList<Integer> nominalAttCandidates = new ArrayList<>(
-        this.numNominalsOption.getValue());
-    for (int i = 0; i < this.numNominalsOption.getValue(); i++) {
-      nominalAttCandidates.add(i);
-    }
-    double[] minNumericVals = new double[this.numNumericsOption.getValue()];
-    double[] maxNumericVals = new double[this.numNumericsOption.getValue()];
-    for (int i = 0; i < this.numNumericsOption.getValue(); i++) {
-      minNumericVals[i] = 0.0;
-      maxNumericVals[i] = 1.0;
-    }
-    this.treeRoot = generateRandomTreeNode(0, nominalAttCandidates,
-        minNumericVals, maxNumericVals, treeRand);
-  }
-
-  protected Node generateRandomTreeNode(int currentDepth,
-      ArrayList<Integer> nominalAttCandidates, double[] minNumericVals,
-      double[] maxNumericVals, Random treeRand) {
-    if ((currentDepth >= this.maxTreeDepthOption.getValue())
-        || ((currentDepth >= this.firstLeafLevelOption.getValue()) && (this.leafFractionOption.getValue() >= (1.0 - treeRand
-            .nextDouble())))) {
-      Node leaf = new Node();
-      leaf.classLabel = treeRand.nextInt(this.numClassesOption.getValue());
-      return leaf;
-    }
-    Node node = new Node();
-    int chosenAtt = treeRand.nextInt(nominalAttCandidates.size()
-        + this.numNumericsOption.getValue());
-    if (chosenAtt < nominalAttCandidates.size()) {
-      node.splitAttIndex = nominalAttCandidates.get(chosenAtt);
-      node.children = new Node[this.numValsPerNominalOption.getValue()];
-      ArrayList<Integer> newNominalCandidates = new ArrayList<>(
-          nominalAttCandidates);
-      newNominalCandidates.remove(new Integer(node.splitAttIndex));
-      newNominalCandidates.trimToSize();
-      for (int i = 0; i < node.children.length; i++) {
-        node.children[i] = generateRandomTreeNode(currentDepth + 1,
-            newNominalCandidates, minNumericVals, maxNumericVals,
-            treeRand);
-      }
-    } else {
-      int numericIndex = chosenAtt - nominalAttCandidates.size();
-      node.splitAttIndex = this.numNominalsOption.getValue()
-          + numericIndex;
-      double minVal = minNumericVals[numericIndex];
-      double maxVal = maxNumericVals[numericIndex];
-      node.splitAttValue = ((maxVal - minVal) * treeRand.nextDouble())
-          + minVal;
-      node.children = new Node[2];
-      double[] newMaxVals = maxNumericVals.clone();
-      newMaxVals[numericIndex] = node.splitAttValue;
-      node.children[0] = generateRandomTreeNode(currentDepth + 1,
-          nominalAttCandidates, minNumericVals, newMaxVals, treeRand);
-      double[] newMinVals = minNumericVals.clone();
-      newMinVals[numericIndex] = node.splitAttValue;
-      node.children[1] = generateRandomTreeNode(currentDepth + 1,
-          nominalAttCandidates, newMinVals, maxNumericVals, treeRand);
-    }
-    return node;
-  }
-
-  @Override
-  public void getDescription(StringBuilder sb, int indent) {
-    // TODO Auto-generated method stub
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/ClusteringEntranceProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/ClusteringEntranceProcessor.java b/samoa-api/src/main/java/org/apache/samoa/streams/ClusteringEntranceProcessor.java
index db6f698..3c0d76e 100644
--- a/samoa-api/src/main/java/org/apache/samoa/streams/ClusteringEntranceProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/ClusteringEntranceProcessor.java
@@ -32,9 +32,8 @@ import org.apache.samoa.learners.clusterers.ClusteringContentEvent;
 import org.apache.samoa.moa.cluster.Clustering;
 import org.apache.samoa.moa.core.DataPoint;
 import org.apache.samoa.moa.options.AbstractOptionHandler;
-import org.apache.samoa.moa.streams.InstanceStream;
-import org.apache.samoa.moa.streams.clustering.ClusteringStream;
-import org.apache.samoa.moa.streams.clustering.RandomRBFGeneratorEvents;
+import org.apache.samoa.streams.clustering.ClusteringStream;
+import org.apache.samoa.streams.clustering.RandomRBFGeneratorEvents;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/ExampleStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/ExampleStream.java b/samoa-api/src/main/java/org/apache/samoa/streams/ExampleStream.java
new file mode 100644
index 0000000..b0cbe11
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/ExampleStream.java
@@ -0,0 +1,76 @@
+package org.apache.samoa.streams;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import org.apache.samoa.instances.InstancesHeader;
+import org.apache.samoa.moa.MOAObject;
+import org.apache.samoa.moa.core.Example;
+
+/**
+ * Interface representing a data stream of examples.
+ * 
+ * @author Richard Kirkby (rkirkby@cs.waikato.ac.nz)
+ * @version $Revision: 7 $
+ */
+public interface ExampleStream<E extends Example> extends MOAObject {
+
+  /**
+   * Gets the header of this stream. This is useful to know attributes and classes. InstancesHeader is an extension of
+   * weka.Instances.
+   * 
+   * @return the header of this stream
+   */
+  public InstancesHeader getHeader();
+
+  /**
+   * Gets the estimated number of remaining instances in this stream
+   * 
+   * @return the estimated number of instances to get from this stream
+   */
+  public long estimatedRemainingInstances();
+
+  /**
+   * Gets whether this stream has more instances to output. This is useful when reading streams from files.
+   * 
+   * @return true if this stream has more instances to output
+   */
+  public boolean hasMoreInstances();
+
+  /**
+   * Gets the next example from this stream.
+   * 
+   * @return the next example of this stream
+   */
+  public E nextInstance();
+
+  /**
+   * Gets whether this stream can restart.
+   * 
+   * @return true if this stream can restart
+   */
+  public boolean isRestartable();
+
+  /**
+   * Restarts this stream. It must be similar to starting a new stream from scratch.
+   * 
+   */
+  public void restart();
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/FileStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/FileStream.java b/samoa-api/src/main/java/org/apache/samoa/streams/FileStream.java
index 4f07ed2..eece541 100644
--- a/samoa-api/src/main/java/org/apache/samoa/streams/FileStream.java
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/FileStream.java
@@ -31,7 +31,6 @@ import org.apache.samoa.instances.InstancesHeader;
 import org.apache.samoa.moa.core.InstanceExample;
 import org.apache.samoa.moa.core.ObjectRepository;
 import org.apache.samoa.moa.options.AbstractOptionHandler;
-import org.apache.samoa.moa.streams.InstanceStream;
 import org.apache.samoa.moa.tasks.TaskMonitor;
 import org.apache.samoa.streams.fs.FileStreamSource;
 

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/InstanceStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/InstanceStream.java b/samoa-api/src/main/java/org/apache/samoa/streams/InstanceStream.java
new file mode 100644
index 0000000..bf9e7af
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/InstanceStream.java
@@ -0,0 +1,34 @@
+package org.apache.samoa.streams;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import org.apache.samoa.instances.Instance;
+import org.apache.samoa.moa.core.Example;
+
+/**
+ * Interface representing a data stream of instances.
+ * 
+ * @author Richard Kirkby (rkirkby@cs.waikato.ac.nz)
+ * @version $Revision: 7 $
+ */
+public interface InstanceStream extends ExampleStream<Example<Instance>> {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/PrequentialSourceProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/PrequentialSourceProcessor.java b/samoa-api/src/main/java/org/apache/samoa/streams/PrequentialSourceProcessor.java
index b947b2f..6e1598f 100644
--- a/samoa-api/src/main/java/org/apache/samoa/streams/PrequentialSourceProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/PrequentialSourceProcessor.java
@@ -32,7 +32,6 @@ import org.apache.samoa.instances.Instance;
 import org.apache.samoa.instances.Instances;
 import org.apache.samoa.learners.InstanceContentEvent;
 import org.apache.samoa.moa.options.AbstractOptionHandler;
-import org.apache.samoa.moa.streams.InstanceStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/StreamSource.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/StreamSource.java b/samoa-api/src/main/java/org/apache/samoa/streams/StreamSource.java
index f4dba63..14684db 100644
--- a/samoa-api/src/main/java/org/apache/samoa/streams/StreamSource.java
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/StreamSource.java
@@ -26,7 +26,6 @@ package org.apache.samoa.streams;
 
 import org.apache.samoa.instances.Instance;
 import org.apache.samoa.moa.core.Example;
-import org.apache.samoa.moa.streams.InstanceStream;
 
 /**
  * The Class StreamSource.

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/StreamSourceProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/StreamSourceProcessor.java b/samoa-api/src/main/java/org/apache/samoa/streams/StreamSourceProcessor.java
index d34f701..8238134 100644
--- a/samoa-api/src/main/java/org/apache/samoa/streams/StreamSourceProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/StreamSourceProcessor.java
@@ -29,7 +29,6 @@ import org.apache.samoa.core.Processor;
 import org.apache.samoa.instances.Instance;
 import org.apache.samoa.instances.Instances;
 import org.apache.samoa.learners.InstanceContentEvent;
-import org.apache.samoa.moa.streams.InstanceStream;
 import org.apache.samoa.topology.Stream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusterEvent.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusterEvent.java b/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusterEvent.java
new file mode 100644
index 0000000..80d222c
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusterEvent.java
@@ -0,0 +1,49 @@
+package org.apache.samoa.streams.clustering;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import java.util.EventObject;
+
+public class ClusterEvent extends EventObject {
+
+  private String type;
+  private String message;
+  private long timestamp;
+
+  public ClusterEvent(Object source, long timestamp, String type, String message) {
+    super(source);
+    this.type = type;
+    this.message = message;
+    this.timestamp = timestamp;
+  }
+
+  public String getMessage() {
+    return message;
+  }
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public String getType() {
+    return type;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusterEventListener.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusterEventListener.java b/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusterEventListener.java
new file mode 100644
index 0000000..3412549
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusterEventListener.java
@@ -0,0 +1,29 @@
+package org.apache.samoa.streams.clustering;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import java.util.EventListener;
+
+public interface ClusterEventListener extends EventListener {
+
+  public void changeCluster(ClusterEvent e);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusteringStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusteringStream.java b/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusteringStream.java
new file mode 100644
index 0000000..1eb5607
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/clustering/ClusteringStream.java
@@ -0,0 +1,54 @@
+package org.apache.samoa.streams.clustering;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import org.apache.samoa.moa.options.AbstractOptionHandler;
+import org.apache.samoa.streams.InstanceStream;
+
+import com.github.javacliparser.FloatOption;
+import com.github.javacliparser.IntOption;
+
+public abstract class ClusteringStream extends AbstractOptionHandler implements InstanceStream {
+  public IntOption decayHorizonOption = new IntOption("decayHorizon", 'h',
+      "Decay horizon", 1000, 0, Integer.MAX_VALUE);
+
+  public FloatOption decayThresholdOption = new FloatOption("decayThreshold", 't',
+      "Decay horizon threshold", 0.01, 0, 1);
+
+  public IntOption evaluationFrequencyOption = new IntOption("evaluationFrequency", 'e',
+      "Evaluation frequency", 1000, 0, Integer.MAX_VALUE);
+
+  public IntOption numAttsOption = new IntOption("numAtts", 'a',
+      "The number of attributes to generate.", 2, 0, Integer.MAX_VALUE);
+
+  public int getDecayHorizon() {
+    return decayHorizonOption.getValue();
+  }
+
+  public double getDecayThreshold() {
+    return decayThresholdOption.getValue();
+  }
+
+  public int getEvaluationFrequency() {
+    return evaluationFrequencyOption.getValue();
+  }
+
+}


[15/17] incubator-samoa git commit: SAMOA-14: Consolidate ARFFFileStream

Posted by gd...@apache.org.
SAMOA-14: Consolidate ARFFFileStream


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

Branch: refs/heads/master
Commit: dc2b7bc30f05fd2d3899827d3adb3818960084a8
Parents: 1bf3c02
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Sun Mar 6 15:22:37 2016 +0300
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Sun Mar 13 11:05:50 2016 +0300

----------------------------------------------------------------------
 .../samoa/moa/streams/ArffFileStream.java       | 200 -------------------
 .../samoa/moa/streams/AvroFileStream.java       | 167 ----------------
 .../apache/samoa/streams/ArffFileStream.java    |   1 -
 .../apache/samoa/streams/AvroFileStream.java    | 166 +++++++++++++++
 .../samoa/streams/fs/HDFSFileStreamSource.java  |  14 +-
 5 files changed, 174 insertions(+), 374 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/dc2b7bc3/samoa-api/src/main/java/org/apache/samoa/moa/streams/ArffFileStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/ArffFileStream.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/ArffFileStream.java
deleted file mode 100644
index 3a17d61..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/ArffFileStream.java
+++ /dev/null
@@ -1,200 +0,0 @@
-package org.apache.samoa.moa.streams;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.io.BufferedReader;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.Reader;
-
-import org.apache.samoa.instances.Instances;
-import org.apache.samoa.instances.InstancesHeader;
-import org.apache.samoa.moa.core.InputStreamProgressMonitor;
-import org.apache.samoa.moa.core.InstanceExample;
-import org.apache.samoa.moa.core.ObjectRepository;
-import org.apache.samoa.moa.options.AbstractOptionHandler;
-import org.apache.samoa.moa.tasks.TaskMonitor;
-
-import com.github.javacliparser.FileOption;
-import com.github.javacliparser.IntOption;
-
-/**
- * Stream reader of ARFF files.
- * 
- * @author Richard Kirkby (rkirkby@cs.waikato.ac.nz)
- * @version $Revision: 7 $
- */
-public class ArffFileStream extends AbstractOptionHandler implements InstanceStream {
-
-  @Override
-  public String getPurposeString() {
-    return "A stream read from an ARFF file.";
-  }
-
-  private static final long serialVersionUID = 1L;
-
-  public FileOption arffFileOption = new FileOption("arffFile", 'f',
-      "ARFF file to load.", null, "arff", false);
-
-  public IntOption classIndexOption = new IntOption(
-      "classIndex",
-      'c',
-      "Class index of data. 0 for none or -1 for last attribute in file.",
-      -1, -1, Integer.MAX_VALUE);
-
-  protected Instances instances;
-
-  transient protected Reader fileReader;
-
-  protected boolean hitEndOfFile;
-
-  protected InstanceExample lastInstanceRead;
-
-  protected int numInstancesRead;
-
-  transient protected InputStreamProgressMonitor fileProgressMonitor;
-
-  protected boolean hasStarted;
-
-  public ArffFileStream() {
-  }
-
-  public ArffFileStream(String arffFileName, int classIndex) {
-    this.arffFileOption.setValue(arffFileName);
-    this.classIndexOption.setValue(classIndex);
-    this.hasStarted = false;
-    restart();
-  }
-
-  @Override
-  public void prepareForUseImpl(TaskMonitor monitor,
-      ObjectRepository repository) {
-    // restart();
-    this.hasStarted = false;
-    this.lastInstanceRead = null;
-  }
-
-  @Override
-  public InstancesHeader getHeader() {
-    return new InstancesHeader(this.instances);
-  }
-
-  @Override
-  public long estimatedRemainingInstances() {
-    double progressFraction = this.fileProgressMonitor.getProgressFraction();
-    if ((progressFraction > 0.0) && (this.numInstancesRead > 0)) {
-      return (long) ((this.numInstancesRead / progressFraction) - this.numInstancesRead);
-    }
-    return -1;
-  }
-
-  @Override
-  public boolean hasMoreInstances() {
-    return !this.hitEndOfFile;
-  }
-
-  @Override
-  public InstanceExample nextInstance() {
-    if (this.lastInstanceRead == null) {
-      readNextInstanceFromFile();
-    }
-    InstanceExample prevInstance = this.lastInstanceRead;
-    this.hitEndOfFile = !readNextInstanceFromFile();
-    return prevInstance;
-  }
-
-  @Override
-  public boolean isRestartable() {
-    return true;
-  }
-
-  @Override
-  public void restart() {
-    try {
-      reset();
-      // this.hitEndOfFile = !readNextInstanceFromFile();
-    } catch (IOException ioe) {
-      throw new RuntimeException("ArffFileStream restart failed.", ioe);
-    }
-  }
-
-  protected boolean readNextInstanceFromFile() {
-    boolean ret;
-    if (!this.hasStarted) {
-      try {
-        reset();
-        ret = getNextInstanceFromFile();
-        this.hitEndOfFile = !ret;
-      } catch (IOException ioe) {
-        throw new RuntimeException("ArffFileStream restart failed.", ioe);
-      }
-      this.hasStarted = true;
-    } else {
-      ret = getNextInstanceFromFile();
-    }
-    return ret;
-  }
-
-  @Override
-  public void getDescription(StringBuilder sb, int indent) {
-    // TODO Auto-generated method stub
-  }
-
-  private void reset() throws IOException {
-    if (this.fileReader != null) {
-      this.fileReader.close();
-    }
-    InputStream fileStream = new FileInputStream(this.arffFileOption.getFile());
-    this.fileProgressMonitor = new InputStreamProgressMonitor(
-        fileStream);
-    this.fileReader = new BufferedReader(new InputStreamReader(
-        this.fileProgressMonitor));
-    this.instances = new Instances(this.fileReader, 1, this.classIndexOption.getValue());
-    if (this.classIndexOption.getValue() < 0) {
-      this.instances.setClassIndex(this.instances.numAttributes() - 1);
-    } else if (this.classIndexOption.getValue() > 0) {
-      this.instances.setClassIndex(this.classIndexOption.getValue() - 1);
-    }
-    this.numInstancesRead = 0;
-    this.lastInstanceRead = null;
-  }
-
-  private boolean getNextInstanceFromFile() throws RuntimeException {
-    try {
-      if (this.instances.readInstance(this.fileReader)) {
-        this.lastInstanceRead = new InstanceExample(this.instances.instance(0));
-        this.instances.delete(); // keep instances clean
-        this.numInstancesRead++;
-        return true;
-      }
-      if (this.fileReader != null) {
-        this.fileReader.close();
-        this.fileReader = null;
-      }
-      return false;
-    } catch (IOException ioe) {
-      throw new RuntimeException(
-          "ArffFileStream failed to read instance from stream.", ioe);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/dc2b7bc3/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
deleted file mode 100644
index 74b31dd..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
+++ /dev/null
@@ -1,167 +0,0 @@
-package org.apache.samoa.moa.streams;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.samoa.instances.Instances;
-import org.apache.samoa.moa.core.InstanceExample;
-import org.apache.samoa.moa.core.ObjectRepository;
-import org.apache.samoa.moa.tasks.TaskMonitor;
-import org.apache.samoa.streams.FileStream;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.github.javacliparser.FileOption;
-import com.github.javacliparser.IntOption;
-import com.github.javacliparser.StringOption;
-
-/**
- * InstanceStream implementation to handle Apache Avro Files. Handles both JSON & Binary encoded streams
- * 
- *
- */
-public class AvroFileStream extends FileStream {
-
-  private static final long serialVersionUID = 1L;
-  private static final Logger logger = LoggerFactory.getLogger(AvroFileStream.class);
-
-  public FileOption avroFileOption = new FileOption("avroFile", 'f', "Avro File(s) to load.", null, null, false);
-  public IntOption classIndexOption = new IntOption("classIndex", 'c',
-      "Class index of data. 0 for none or -1 for last attribute in file.", -1, -1, Integer.MAX_VALUE);
-  public StringOption encodingFormatOption = new StringOption("encodingFormatOption", 'e',
-      "Encoding format for Avro Files. Can be JSON/AVRO", "BINARY");
-
-  /** Represents the last read Instance **/
-  protected InstanceExample lastInstanceRead;
-
-  /** Represents the binary input stream of avro data **/
-  protected transient InputStream inputStream = null;
-
-  /** The extension to be considered for the files **/
-  private static final String AVRO_FILE_EXTENSION = "avro";
-
-  /* (non-Javadoc)
-   * @see org.apache.samoa.streams.FileStream#reset()
-   * Reset the BINARY encoded Avro Stream & Close the file source
-   */
-  @Override
-  protected void reset() {
-
-    try {
-      if (this.inputStream != null)
-        this.inputStream.close();
-
-      fileSource.reset();
-    } catch (IOException ioException) {
-      logger.error(AVRO_STREAM_FAILED_RESTART_ERROR + " : {}", ioException);
-      throw new RuntimeException(AVRO_STREAM_FAILED_RESTART_ERROR, ioException);
-    }
-
-    if (!getNextFileStream()) {
-      hitEndOfStream = true;
-      throw new RuntimeException(AVRO_STREAM_EMPTY_STREAM_ERROR);
-    }
-  }
-
-  /**
-   * Get next File Stream & set the class index read from the command line option
-   * 
-   * @return
-   */
-  protected boolean getNextFileStream() {
-    if (this.inputStream != null)
-      try {
-        this.inputStream.close();
-      } catch (IOException ioException) {
-        logger.error(AVRO_STREAM_FAILED_READ_ERROR + " : {}", ioException);
-        throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
-      }
-
-    this.inputStream = this.fileSource.getNextInputStream();
-
-    if (this.inputStream == null)
-      return false;
-
-    this.instances = new Instances(this.inputStream, classIndexOption.getValue(), encodingFormatOption.getValue());
-
-    if (this.classIndexOption.getValue() < 0) {
-      this.instances.setClassIndex(this.instances.numAttributes() - 1);
-    } else if (this.classIndexOption.getValue() > 0) {
-      this.instances.setClassIndex(this.classIndexOption.getValue() - 1);
-    }
-    return true;
-  }
-
-  /* (non-Javadoc)
-   * @see org.apache.samoa.streams.FileStream#readNextInstanceFromFile()
-   * Read next Instance from File. Return false if unable to read next Instance
-   */
-  @Override
-  protected boolean readNextInstanceFromFile() {
-    try {
-      if (this.instances.readInstance()) {
-        this.lastInstanceRead = new InstanceExample(this.instances.instance(0));
-        this.instances.delete();
-        return true;
-      }
-      if (this.inputStream != null) {
-        this.inputStream.close();
-        this.inputStream = null;
-      }
-      return false;
-    } catch (IOException ioException) {
-      logger.error(AVRO_STREAM_FAILED_READ_ERROR + " : {}", ioException);
-      throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
-    }
-
-  }
-
-  @Override
-  public void prepareForUseImpl(TaskMonitor monitor, ObjectRepository repository) {
-    super.prepareForUseImpl(monitor, repository);
-    String filePath = this.avroFileOption.getFile().getAbsolutePath();
-    this.fileSource.init(filePath, AvroFileStream.AVRO_FILE_EXTENSION);
-    this.lastInstanceRead = null;
-  }
-
-  /* (non-Javadoc)
-   * @see org.apache.samoa.streams.FileStream#getLastInstanceRead()
-   * Return the last read Instance
-   */
-  @Override
-  protected InstanceExample getLastInstanceRead() {
-    return this.lastInstanceRead;
-  }
-
-  @Override
-  public void getDescription(StringBuilder sb, int indent) {
-    throw new UnsupportedOperationException(AVRO_STREAM_UNSUPPORTED_METHOD);
-  }
-
-  /** Error Messages to for all types of Avro File Streams */
-  protected static final String AVRO_STREAM_FAILED_RESTART_ERROR = "Avro FileStream restart failed.";
-  protected static final String AVRO_STREAM_EMPTY_STREAM_ERROR = "Avro FileStream is empty.";
-  protected static final String AVRO_STREAM_FAILED_READ_ERROR = "Failed to read next instance from Avro File Stream.";
-  protected static final String AVRO_STREAM_UNSUPPORTED_METHOD = "This method is not supported for AvroFileStream yet.";
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/dc2b7bc3/samoa-api/src/main/java/org/apache/samoa/streams/ArffFileStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/ArffFileStream.java b/samoa-api/src/main/java/org/apache/samoa/streams/ArffFileStream.java
index 099f639..9f8a322 100644
--- a/samoa-api/src/main/java/org/apache/samoa/streams/ArffFileStream.java
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/ArffFileStream.java
@@ -33,7 +33,6 @@ import com.github.javacliparser.IntOption;
 /**
  * InstanceStream for ARFF file
  * 
- * @author Casey
  */
 public class ArffFileStream extends FileStream {
 

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/dc2b7bc3/samoa-api/src/main/java/org/apache/samoa/streams/AvroFileStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/AvroFileStream.java b/samoa-api/src/main/java/org/apache/samoa/streams/AvroFileStream.java
new file mode 100644
index 0000000..15229a4
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/AvroFileStream.java
@@ -0,0 +1,166 @@
+package org.apache.samoa.streams;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.samoa.instances.Instances;
+import org.apache.samoa.moa.core.InstanceExample;
+import org.apache.samoa.moa.core.ObjectRepository;
+import org.apache.samoa.moa.tasks.TaskMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.javacliparser.FileOption;
+import com.github.javacliparser.IntOption;
+import com.github.javacliparser.StringOption;
+
+/**
+ * InstanceStream implementation to handle Apache Avro Files. Handles both JSON & Binary encoded streams
+ * 
+ *
+ */
+public class AvroFileStream extends FileStream {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger logger = LoggerFactory.getLogger(AvroFileStream.class);
+
+  public FileOption avroFileOption = new FileOption("avroFile", 'f', "Avro File(s) to load.", null, null, false);
+  public IntOption classIndexOption = new IntOption("classIndex", 'c',
+      "Class index of data. 0 for none or -1 for last attribute in file.", -1, -1, Integer.MAX_VALUE);
+  public StringOption encodingFormatOption = new StringOption("encodingFormatOption", 'e',
+      "Encoding format for Avro Files. Can be JSON/AVRO", "BINARY");
+
+  /** Represents the last read Instance **/
+  protected InstanceExample lastInstanceRead;
+
+  /** Represents the binary input stream of avro data **/
+  protected transient InputStream inputStream = null;
+
+  /** The extension to be considered for the files **/
+  private static final String AVRO_FILE_EXTENSION = "avro";
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.streams.FileStream#reset()
+   * Reset the BINARY encoded Avro Stream & Close the file source
+   */
+  @Override
+  protected void reset() {
+
+    try {
+      if (this.inputStream != null)
+        this.inputStream.close();
+
+      fileSource.reset();
+    } catch (IOException ioException) {
+      logger.error(AVRO_STREAM_FAILED_RESTART_ERROR + " : {}", ioException);
+      throw new RuntimeException(AVRO_STREAM_FAILED_RESTART_ERROR, ioException);
+    }
+
+    if (!getNextFileStream()) {
+      hitEndOfStream = true;
+      throw new RuntimeException(AVRO_STREAM_EMPTY_STREAM_ERROR);
+    }
+  }
+
+  /**
+   * Get next File Stream & set the class index read from the command line option
+   * 
+   * @return
+   */
+  protected boolean getNextFileStream() {
+    if (this.inputStream != null)
+      try {
+        this.inputStream.close();
+      } catch (IOException ioException) {
+        logger.error(AVRO_STREAM_FAILED_READ_ERROR + " : {}", ioException);
+        throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
+      }
+
+    this.inputStream = this.fileSource.getNextInputStream();
+
+    if (this.inputStream == null)
+      return false;
+
+    this.instances = new Instances(this.inputStream, classIndexOption.getValue(), encodingFormatOption.getValue());
+
+    if (this.classIndexOption.getValue() < 0) {
+      this.instances.setClassIndex(this.instances.numAttributes() - 1);
+    } else if (this.classIndexOption.getValue() > 0) {
+      this.instances.setClassIndex(this.classIndexOption.getValue() - 1);
+    }
+    return true;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.streams.FileStream#readNextInstanceFromFile()
+   * Read next Instance from File. Return false if unable to read next Instance
+   */
+  @Override
+  protected boolean readNextInstanceFromFile() {
+    try {
+      if (this.instances.readInstance()) {
+        this.lastInstanceRead = new InstanceExample(this.instances.instance(0));
+        this.instances.delete();
+        return true;
+      }
+      if (this.inputStream != null) {
+        this.inputStream.close();
+        this.inputStream = null;
+      }
+      return false;
+    } catch (IOException ioException) {
+      logger.error(AVRO_STREAM_FAILED_READ_ERROR + " : {}", ioException);
+      throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
+    }
+
+  }
+
+  @Override
+  public void prepareForUseImpl(TaskMonitor monitor, ObjectRepository repository) {
+    super.prepareForUseImpl(monitor, repository);
+    String filePath = this.avroFileOption.getFile().getAbsolutePath();
+    this.fileSource.init(filePath, AvroFileStream.AVRO_FILE_EXTENSION);
+    this.lastInstanceRead = null;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.streams.FileStream#getLastInstanceRead()
+   * Return the last read Instance
+   */
+  @Override
+  protected InstanceExample getLastInstanceRead() {
+    return this.lastInstanceRead;
+  }
+
+  @Override
+  public void getDescription(StringBuilder sb, int indent) {
+    throw new UnsupportedOperationException(AVRO_STREAM_UNSUPPORTED_METHOD);
+  }
+
+  /** Error Messages to for all types of Avro File Streams */
+  protected static final String AVRO_STREAM_FAILED_RESTART_ERROR = "Avro FileStream restart failed.";
+  protected static final String AVRO_STREAM_EMPTY_STREAM_ERROR = "Avro FileStream is empty.";
+  protected static final String AVRO_STREAM_FAILED_READ_ERROR = "Failed to read next instance from Avro File Stream.";
+  protected static final String AVRO_STREAM_UNSUPPORTED_METHOD = "This method is not supported for AvroFileStream yet.";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/dc2b7bc3/samoa-api/src/main/java/org/apache/samoa/streams/fs/HDFSFileStreamSource.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/fs/HDFSFileStreamSource.java b/samoa-api/src/main/java/org/apache/samoa/streams/fs/HDFSFileStreamSource.java
index 00abd1a..67d5e79 100644
--- a/samoa-api/src/main/java/org/apache/samoa/streams/fs/HDFSFileStreamSource.java
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/fs/HDFSFileStreamSource.java
@@ -40,8 +40,8 @@ import org.apache.hadoop.io.IOUtils;
 public class HDFSFileStreamSource implements FileStreamSource {
 
   /**
-	 * 
-	 */
+   * 
+   */
   private static final long serialVersionUID = -3887354805787167400L;
 
   private transient InputStream fileStream;
@@ -59,6 +59,10 @@ public class HDFSFileStreamSource implements FileStreamSource {
 
   public void init(Configuration config, String path, String ext) {
     this.config = config;
+    config.set("fs.hdfs.impl",
+        org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
+    config.set("fs.file.impl",
+        org.apache.hadoop.fs.LocalFileSystem.class.getName());
     this.filePaths = new ArrayList<String>();
     Path hdfsPath = new Path(path);
     FileSystem fs;
@@ -69,8 +73,7 @@ public class HDFSFileStreamSource implements FileStreamSource {
         Path filterPath = hdfsPath;
         if (ext != null) {
           filterPath = new Path(path.toString(), "*." + ext);
-        }
-        else {
+        } else {
           filterPath = new Path(path.toString(), "*");
         }
         FileStatus[] filesInDir = fs.globStatus(filterPath);
@@ -79,8 +82,7 @@ public class HDFSFileStreamSource implements FileStreamSource {
             filePaths.add(filesInDir[i].getPath().toString());
           }
         }
-      }
-      else {
+      } else {
         this.filePaths.add(path);
       }
     } catch (IOException ioe) {


[02/17] incubator-samoa git commit: SAMOA-48: Fix for ensembles (race condition?)

Posted by gd...@apache.org.
SAMOA-48: Fix for ensembles (race condition?)


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

Branch: refs/heads/master
Commit: 23169b0605005bf9b0b5f2dacb36a16b6b5b3b5d
Parents: af25e7d
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Mon Oct 26 15:07:16 2015 +0200
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Mon Oct 26 15:07:16 2015 +0200

----------------------------------------------------------------------
 .../classifiers/ensemble/PredictionCombinerProcessor.java    | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/23169b06/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java
index 76e84f8..8e10094 100644
--- a/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java
+++ b/samoa-api/src/main/java/org/apache/samoa/learners/classifiers/ensemble/PredictionCombinerProcessor.java
@@ -106,14 +106,13 @@ public class PredictionCombinerProcessor implements Processor {
 
     addStatisticsForInstanceReceived(instanceIndex, inEvent.getClassifierIndex(), prediction, 1);
 
-    if (inEvent.isLastEvent() || hasAllVotesArrivedInstance(instanceIndex)) {
+    if (hasAllVotesArrivedInstance(instanceIndex)) {
       DoubleVector combinedVote = this.mapVotesforInstanceReceived.get(instanceIndex);
       if (combinedVote == null) {
         combinedVote = new DoubleVector(new double[inEvent.getInstance().numClasses()]);
       }
-      ResultContentEvent outContentEvent = new ResultContentEvent(inEvent.getInstanceIndex(),
-          inEvent.getInstance(), inEvent.getClassId(),
-          combinedVote.getArrayCopy(), inEvent.isLastEvent());
+      ResultContentEvent outContentEvent = new ResultContentEvent(inEvent.getInstanceIndex(), inEvent.getInstance(),
+          inEvent.getClassId(), combinedVote.getArrayCopy(), inEvent.isLastEvent());
       outContentEvent.setEvaluationIndex(inEvent.getEvaluationIndex());
       outputStream.put(outContentEvent);
       clearStatisticsInstance(instanceIndex);
@@ -133,7 +132,6 @@ public class PredictionCombinerProcessor implements Processor {
 
   /*
    * (non-Javadoc)
-   * 
    * @see samoa.core.Processor#newProcessor(samoa.core.Processor)
    */
   @Override


[16/17] incubator-samoa git commit: SAMOA-14: Move all streams under single package

Posted by gd...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/clustering/RandomRBFGeneratorEvents.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/clustering/RandomRBFGeneratorEvents.java b/samoa-api/src/main/java/org/apache/samoa/streams/clustering/RandomRBFGeneratorEvents.java
new file mode 100644
index 0000000..139b8f7
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/clustering/RandomRBFGeneratorEvents.java
@@ -0,0 +1,968 @@
+package org.apache.samoa.streams.clustering;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Enumeration;
+import java.util.LinkedList;
+import java.util.Random;
+import java.util.Vector;
+
+import org.apache.samoa.instances.Attribute;
+import org.apache.samoa.instances.DenseInstance;
+import org.apache.samoa.instances.Instance;
+import org.apache.samoa.instances.Instances;
+import org.apache.samoa.instances.InstancesHeader;
+import org.apache.samoa.moa.cluster.Clustering;
+import org.apache.samoa.moa.cluster.SphereCluster;
+import org.apache.samoa.moa.core.AutoExpandVector;
+import org.apache.samoa.moa.core.DataPoint;
+import org.apache.samoa.moa.core.FastVector;
+import org.apache.samoa.moa.core.InstanceExample;
+import org.apache.samoa.moa.core.ObjectRepository;
+import org.apache.samoa.moa.tasks.TaskMonitor;
+import org.apache.samoa.streams.InstanceStream;
+
+import com.github.javacliparser.FlagOption;
+import com.github.javacliparser.FloatOption;
+import com.github.javacliparser.IntOption;
+
+public class RandomRBFGeneratorEvents extends ClusteringStream {
+  private transient Vector listeners;
+
+  private static final long serialVersionUID = 1L;
+
+  public IntOption modelRandomSeedOption = new IntOption("modelRandomSeed",
+      'm', "Seed for random generation of model.", 1);
+
+  public IntOption instanceRandomSeedOption = new IntOption(
+      "instanceRandomSeed", 'i',
+      "Seed for random generation of instances.", 5);
+
+  public IntOption numClusterOption = new IntOption("numCluster", 'K',
+      "The average number of centroids in the model.", 5, 1, Integer.MAX_VALUE);
+
+  public IntOption numClusterRangeOption = new IntOption("numClusterRange", 'k',
+      "Deviation of the number of centroids in the model.", 3, 0, Integer.MAX_VALUE);
+
+  public FloatOption kernelRadiiOption = new FloatOption("kernelRadius", 'R',
+      "The average radii of the centroids in the model.", 0.07, 0, 1);
+
+  public FloatOption kernelRadiiRangeOption = new FloatOption("kernelRadiusRange", 'r',
+      "Deviation of average radii of the centroids in the model.", 0, 0, 1);
+
+  public FloatOption densityRangeOption = new FloatOption("densityRange", 'd',
+      "Offset of the average weight a cluster has. Value of 0 means all cluster " +
+          "contain the same amount of points.", 0, 0, 1);
+
+  public IntOption speedOption = new IntOption("speed", 'V',
+      "Kernels move a predefined distance of 0.01 every X points", 500, 1, Integer.MAX_VALUE);
+
+  public IntOption speedRangeOption = new IntOption("speedRange", 'v',
+      "Speed/Velocity point offset", 0, 0, Integer.MAX_VALUE);
+
+  public FloatOption noiseLevelOption = new FloatOption("noiseLevel", 'N',
+      "Noise level", 0.1, 0, 1);
+
+  public FlagOption noiseInClusterOption = new FlagOption("noiseInCluster", 'n',
+      "Allow noise to be placed within a cluster");
+
+  public IntOption eventFrequencyOption = new IntOption("eventFrequency", 'E',
+      "Event frequency. Enable at least one of the events below and set numClusterRange!", 30000, 0, Integer.MAX_VALUE);
+
+  public FlagOption eventMergeSplitOption = new FlagOption("eventMergeSplitOption", 'M',
+      "Enable merging and splitting of clusters. Set eventFrequency and numClusterRange!");
+
+  public FlagOption eventDeleteCreateOption = new FlagOption("eventDeleteCreate", 'C',
+      "Enable emering and disapperaing of clusters. Set eventFrequency and numClusterRange!");
+
+  private double merge_threshold = 0.7;
+  private int kernelMovePointFrequency = 10;
+  private double maxDistanceMoveThresholdByStep = 0.01;
+  private int maxOverlapFitRuns = 50;
+  private double eventFrequencyRange = 0;
+
+  private boolean debug = false;
+
+  private AutoExpandVector<GeneratorCluster> kernels;
+  protected Random instanceRandom;
+  protected InstancesHeader streamHeader;
+  private int numGeneratedInstances;
+  private int numActiveKernels;
+  private int nextEventCounter;
+  private int nextEventChoice = -1;
+  private int clusterIdCounter;
+  private GeneratorCluster mergeClusterA;
+  private GeneratorCluster mergeClusterB;
+  private boolean mergeKernelsOverlapping = false;
+
+  private class GeneratorCluster implements Serializable {
+    // TODO: points is redundant to microclusterpoints, we need to come
+    // up with a good strategy that microclusters get updated and
+    // rebuild if needed. Idea: Sort microclusterpoints by timestamp and let
+    // microclusterdecay hold the timestamp for when the last point in a
+    // microcluster gets kicked, then we rebuild... or maybe not... could be
+    // same as searching for point to be kicked. more likely is we rebuild
+    // fewer times then insert.
+
+    private static final long serialVersionUID = -6301649898961112942L;
+
+    SphereCluster generator;
+    int kill = -1;
+    boolean merging = false;
+    double[] moveVector;
+    int totalMovementSteps;
+    int currentMovementSteps;
+    boolean isSplitting = false;
+
+    LinkedList<DataPoint> points = new LinkedList<DataPoint>();
+    ArrayList<SphereCluster> microClusters = new ArrayList<SphereCluster>();
+    ArrayList<ArrayList<DataPoint>> microClustersPoints = new ArrayList();
+    ArrayList<Integer> microClustersDecay = new ArrayList();
+
+    public GeneratorCluster(int label) {
+      boolean outofbounds = true;
+      int tryCounter = 0;
+      while (outofbounds && tryCounter < maxOverlapFitRuns) {
+        tryCounter++;
+        outofbounds = false;
+        double[] center = new double[numAttsOption.getValue()];
+        double radius = kernelRadiiOption.getValue() + (instanceRandom.nextBoolean() ? -1 : 1)
+            * kernelRadiiRangeOption.getValue() * instanceRandom.nextDouble();
+        while (radius <= 0) {
+          radius = kernelRadiiOption.getValue() + (instanceRandom.nextBoolean() ? -1 : 1)
+              * kernelRadiiRangeOption.getValue() * instanceRandom.nextDouble();
+        }
+        for (int j = 0; j < numAttsOption.getValue(); j++) {
+          center[j] = instanceRandom.nextDouble();
+          if (center[j] - radius < 0 || center[j] + radius > 1) {
+            outofbounds = true;
+            break;
+          }
+        }
+        generator = new SphereCluster(center, radius);
+      }
+      if (tryCounter < maxOverlapFitRuns) {
+        generator.setId(label);
+        double avgWeight = 1.0 / numClusterOption.getValue();
+        double weight = avgWeight + (instanceRandom.nextBoolean() ? -1 : 1) * avgWeight * densityRangeOption.getValue()
+            * instanceRandom.nextDouble();
+        generator.setWeight(weight);
+        setDesitnation(null);
+      }
+      else {
+        generator = null;
+        kill = 0;
+        System.out.println("Tried " + maxOverlapFitRuns + " times to create kernel. Reduce average radii.");
+      }
+    }
+
+    public GeneratorCluster(int label, SphereCluster cluster) {
+      this.generator = cluster;
+      cluster.setId(label);
+      setDesitnation(null);
+    }
+
+    public int getWorkID() {
+      for (int c = 0; c < kernels.size(); c++) {
+        if (kernels.get(c).equals(this))
+          return c;
+      }
+      return -1;
+    }
+
+    private void updateKernel() {
+      if (kill == 0) {
+        kernels.remove(this);
+      }
+      if (kill > 0) {
+        kill--;
+      }
+      // we could be lot more precise if we would keep track of timestamps of
+      // points
+      // then we could remove all old points and rebuild the cluster on up to
+      // date point base
+      // BUT worse the effort??? so far we just want to avoid overlap with this,
+      // so its more
+      // konservative as needed. Only needs to change when we need a thighter
+      // representation
+      for (int m = 0; m < microClusters.size(); m++) {
+        if (numGeneratedInstances - microClustersDecay.get(m) > decayHorizonOption.getValue()) {
+          microClusters.remove(m);
+          microClustersPoints.remove(m);
+          microClustersDecay.remove(m);
+        }
+      }
+
+      if (!points.isEmpty()
+          && numGeneratedInstances - points.getFirst().getTimestamp() >= decayHorizonOption.getValue()) {
+        // if(debug)
+        // System.out.println("Cleaning up macro cluster "+generator.getId());
+        points.removeFirst();
+      }
+
+    }
+
+    private void addInstance(Instance instance) {
+      DataPoint point = new DataPoint(instance, numGeneratedInstances);
+      points.add(point);
+
+      int minMicroIndex = -1;
+      double minHullDist = Double.MAX_VALUE;
+      boolean inserted = false;
+      // we favour more recently build clusters so we can remove earlier cluster
+      // sooner
+      for (int m = microClusters.size() - 1; m >= 0; m--) {
+        SphereCluster micro = microClusters.get(m);
+        double hulldist = micro.getCenterDistance(point) - micro.getRadius();
+        // point fits into existing cluster
+        if (hulldist <= 0) {
+          microClustersPoints.get(m).add(point);
+          microClustersDecay.set(m, numGeneratedInstances);
+          inserted = true;
+          break;
+        }
+        // if not, check if its at least the closest cluster
+        else {
+          if (hulldist < minHullDist) {
+            minMicroIndex = m;
+            minHullDist = hulldist;
+          }
+        }
+      }
+      // Reseting index choice for alternative cluster building
+      int alt = 1;
+      if (alt == 1)
+        minMicroIndex = -1;
+      if (!inserted) {
+        // add to closest cluster and expand cluster
+        if (minMicroIndex != -1) {
+          microClustersPoints.get(minMicroIndex).add(point);
+          // we should keep the miniball instances and just check in
+          // new points instead of rebuilding the whole thing
+          SphereCluster s = new SphereCluster(microClustersPoints.get(minMicroIndex), numAttsOption.getValue());
+          // check if current microcluster is bigger then generating cluster
+          if (s.getRadius() > generator.getRadius()) {
+            // remove previously added point
+            microClustersPoints.get(minMicroIndex).remove(microClustersPoints.get(minMicroIndex).size() - 1);
+            minMicroIndex = -1;
+          }
+          else {
+            microClusters.set(minMicroIndex, s);
+            microClustersDecay.set(minMicroIndex, numGeneratedInstances);
+          }
+        }
+        // minMicroIndex might have been reset above
+        // create new micro cluster
+        if (minMicroIndex == -1) {
+          ArrayList<DataPoint> microPoints = new ArrayList<DataPoint>();
+          microPoints.add(point);
+          SphereCluster s;
+          if (alt == 0)
+            s = new SphereCluster(microPoints, numAttsOption.getValue());
+          else
+            s = new SphereCluster(generator.getCenter(), generator.getRadius(), 1);
+
+          microClusters.add(s);
+          microClustersPoints.add(microPoints);
+          microClustersDecay.add(numGeneratedInstances);
+          int id = 0;
+          while (id < kernels.size()) {
+            if (kernels.get(id) == this)
+              break;
+            id++;
+          }
+          s.setGroundTruth(id);
+        }
+      }
+
+    }
+
+    private void move() {
+      if (currentMovementSteps < totalMovementSteps) {
+        currentMovementSteps++;
+        if (moveVector == null) {
+          return;
+        }
+        else {
+          double[] center = generator.getCenter();
+          boolean outofbounds = true;
+          while (outofbounds) {
+            double radius = generator.getRadius();
+            outofbounds = false;
+            center = generator.getCenter();
+            for (int d = 0; d < center.length; d++) {
+              center[d] += moveVector[d];
+              if (center[d] - radius < 0 || center[d] + radius > 1) {
+                outofbounds = true;
+                setDesitnation(null);
+                break;
+              }
+            }
+          }
+          generator.setCenter(center);
+        }
+      }
+      else {
+        if (!merging) {
+          setDesitnation(null);
+          isSplitting = false;
+        }
+      }
+    }
+
+    void setDesitnation(double[] destination) {
+
+      if (destination == null) {
+        destination = new double[numAttsOption.getValue()];
+        for (int j = 0; j < numAttsOption.getValue(); j++) {
+          destination[j] = instanceRandom.nextDouble();
+        }
+      }
+      double[] center = generator.getCenter();
+      int dim = center.length;
+
+      double[] v = new double[dim];
+
+      for (int d = 0; d < dim; d++) {
+        v[d] = destination[d] - center[d];
+      }
+      setMoveVector(v);
+    }
+
+    void setMoveVector(double[] vector) {
+      // we are ignoring the steps, otherwise we have to change
+      // speed of the kernels, do we want that?
+      moveVector = vector;
+      int speedInPoints = speedOption.getValue();
+      if (speedRangeOption.getValue() > 0)
+        speedInPoints += (instanceRandom.nextBoolean() ? -1 : 1) * instanceRandom.nextInt(speedRangeOption.getValue());
+      if (speedInPoints < 1)
+        speedInPoints = speedOption.getValue();
+
+      double length = 0;
+      for (int d = 0; d < moveVector.length; d++) {
+        length += Math.pow(vector[d], 2);
+      }
+      length = Math.sqrt(length);
+
+      totalMovementSteps = (int) (length / (maxDistanceMoveThresholdByStep * kernelMovePointFrequency) * speedInPoints);
+      for (int d = 0; d < moveVector.length; d++) {
+        moveVector[d] /= (double) totalMovementSteps;
+      }
+
+      currentMovementSteps = 0;
+      // if(debug){
+      // System.out.println("Setting new direction for C"+generator.getId()+": distance "
+      // +length+" in "+totalMovementSteps+" steps");
+      // }
+    }
+
+    private String tryMerging(GeneratorCluster merge) {
+      String message = "";
+      double overlapDegree = generator.overlapRadiusDegree(merge.generator);
+      if (overlapDegree > merge_threshold) {
+        SphereCluster mcluster = merge.generator;
+        double radius = Math.max(generator.getRadius(), mcluster.getRadius());
+        generator.combine(mcluster);
+
+        // //adjust radius, get bigger and bigger with high dim data
+        generator.setRadius(radius);
+        // double[] center = generator.getCenter();
+        // double[] mcenter = mcluster.getCenter();
+        // double weight = generator.getWeight();
+        // double mweight = generator.getWeight();
+        // // for (int i = 0; i < center.length; i++) {
+        // // center[i] = (center[i] * weight + mcenter[i] * mweight) / (mweight
+        // + weight);
+        // // }
+        // generator.setWeight(weight + mweight);
+        message = "Clusters merging: " + mergeClusterB.generator.getId() + " into " + mergeClusterA.generator.getId();
+
+        // clean up and restet merging stuff
+        // mark kernel so it gets killed when it doesn't contain any more
+        // instances
+        merge.kill = decayHorizonOption.getValue();
+        // set weight to 0 so no new instances will be created in the cluster
+        mcluster.setWeight(0.0);
+        normalizeWeights();
+        numActiveKernels--;
+        mergeClusterB = mergeClusterA = null;
+        merging = false;
+        mergeKernelsOverlapping = false;
+      }
+      else {
+        if (overlapDegree > 0 && !mergeKernelsOverlapping) {
+          mergeKernelsOverlapping = true;
+          message = "Merge overlapping started";
+        }
+      }
+      return message;
+    }
+
+    private String splitKernel() {
+      isSplitting = true;
+      // todo radius range
+      double radius = kernelRadiiOption.getValue();
+      double avgWeight = 1.0 / numClusterOption.getValue();
+      double weight = avgWeight + avgWeight * densityRangeOption.getValue() * instanceRandom.nextDouble();
+      SphereCluster spcluster = null;
+
+      double[] center = generator.getCenter();
+      spcluster = new SphereCluster(center, radius, weight);
+
+      if (spcluster != null) {
+        GeneratorCluster gc = new GeneratorCluster(clusterIdCounter++, spcluster);
+        gc.isSplitting = true;
+        kernels.add(gc);
+        normalizeWeights();
+        numActiveKernels++;
+        return "Split from Kernel " + generator.getId();
+      }
+      else {
+        System.out.println("Tried to split new kernel from C" + generator.getId() +
+            ". Not enough room for new cluster, decrease average radii, number of clusters or enable overlap.");
+        return "";
+      }
+    }
+
+    private String fadeOut() {
+      kill = decayHorizonOption.getValue();
+      generator.setWeight(0.0);
+      numActiveKernels--;
+      normalizeWeights();
+      return "Fading out C" + generator.getId();
+    }
+
+  }
+
+  public RandomRBFGeneratorEvents() {
+    noiseInClusterOption.set();
+    // eventDeleteCreateOption.set();
+    // eventMergeSplitOption.set();
+  }
+
+  public InstancesHeader getHeader() {
+    return streamHeader;
+  }
+
+  public long estimatedRemainingInstances() {
+    return -1;
+  }
+
+  public boolean hasMoreInstances() {
+    return true;
+  }
+
+  public boolean isRestartable() {
+    return true;
+  }
+
+  @Override
+  public void prepareForUseImpl(TaskMonitor monitor, ObjectRepository repository) {
+    monitor.setCurrentActivity("Preparing random RBF...", -1.0);
+    generateHeader();
+    restart();
+  }
+
+  public void restart() {
+    instanceRandom = new Random(instanceRandomSeedOption.getValue());
+    nextEventCounter = eventFrequencyOption.getValue();
+    nextEventChoice = getNextEvent();
+    numActiveKernels = 0;
+    numGeneratedInstances = 0;
+    clusterIdCounter = 0;
+    mergeClusterA = mergeClusterB = null;
+    kernels = new AutoExpandVector<GeneratorCluster>();
+
+    initKernels();
+  }
+
+  protected void generateHeader() { // 2013/06/02: Noise label
+    ArrayList<Attribute> attributes = new ArrayList<Attribute>();
+    for (int i = 0; i < this.numAttsOption.getValue(); i++) {
+      attributes.add(new Attribute("att" + (i + 1)));
+    }
+
+    ArrayList<String> classLabels = new ArrayList<String>();
+    for (int i = 0; i < this.numClusterOption.getValue(); i++) {
+      classLabels.add("class" + (i + 1));
+    }
+    if (noiseLevelOption.getValue() > 0)
+      classLabels.add("noise"); // The last label = "noise"
+
+    attributes.add(new Attribute("class", classLabels));
+    streamHeader = new InstancesHeader(new Instances(getCLICreationString(InstanceStream.class), attributes, 0));
+    streamHeader.setClassIndex(streamHeader.numAttributes() - 1);
+  }
+
+  protected void initKernels() {
+    for (int i = 0; i < numClusterOption.getValue(); i++) {
+      kernels.add(new GeneratorCluster(clusterIdCounter));
+      numActiveKernels++;
+      clusterIdCounter++;
+    }
+    normalizeWeights();
+  }
+
+  public InstanceExample nextInstance() {
+    numGeneratedInstances++;
+    eventScheduler();
+
+    // make room for the classlabel
+    double[] values_new = new double[numAttsOption.getValue()]; // +1
+    double[] values = null;
+    int clusterChoice = -1;
+
+    if (instanceRandom.nextDouble() > noiseLevelOption.getValue()) {
+      clusterChoice = chooseWeightedElement();
+      values = kernels.get(clusterChoice).generator.sample(instanceRandom).toDoubleArray();
+    }
+    else {
+      // get ranodm noise point
+      values = getNoisePoint();
+    }
+
+    if (Double.isNaN(values[0])) {
+      System.out.println("Instance corrupted:" + numGeneratedInstances);
+    }
+    System.arraycopy(values, 0, values_new, 0, values.length);
+
+    Instance inst = new DenseInstance(1.0, values_new);
+    inst.setDataset(getHeader());
+    if (clusterChoice == -1) {
+      // 2013/06/02 (Yunsu Kim)
+      // Noise instance has the last class value instead of "-1"
+      // Preventing ArrayIndexOutOfBoundsException in WriteStreamToARFFFile
+      inst.setClassValue(numClusterOption.getValue());
+    }
+    else {
+      inst.setClassValue(kernels.get(clusterChoice).generator.getId());
+      // Do we need micro cluster representation if have overlapping clusters?
+      // if(!overlappingOption.isSet())
+      kernels.get(clusterChoice).addInstance(inst);
+    }
+    // System.out.println(numGeneratedInstances+": Overlap is"+updateOverlaps());
+
+    return new InstanceExample(inst);
+  }
+
+  public Clustering getGeneratingClusters() {
+    Clustering clustering = new Clustering();
+    for (int c = 0; c < kernels.size(); c++) {
+      clustering.add(kernels.get(c).generator);
+    }
+    return clustering;
+  }
+
+  public Clustering getMicroClustering() {
+    Clustering clustering = new Clustering();
+    int id = 0;
+
+    for (int c = 0; c < kernels.size(); c++) {
+      for (int m = 0; m < kernels.get(c).microClusters.size(); m++) {
+        kernels.get(c).microClusters.get(m).setId(id);
+        kernels.get(c).microClusters.get(m).setGroundTruth(kernels.get(c).generator.getId());
+        clustering.add(kernels.get(c).microClusters.get(m));
+        id++;
+      }
+    }
+
+    // System.out.println("numMicroKernels "+clustering.size());
+    return clustering;
+  }
+
+  /**************************** EVENTS ******************************************/
+  private void eventScheduler() {
+
+    for (int i = 0; i < kernels.size(); i++) {
+      kernels.get(i).updateKernel();
+    }
+
+    nextEventCounter--;
+    // only move kernels every 10 points, performance reasons????
+    // should this be randomized as well???
+    if (nextEventCounter % kernelMovePointFrequency == 0) {
+      // move kernels
+      for (int i = 0; i < kernels.size(); i++) {
+        kernels.get(i).move();
+        // overlapControl();
+      }
+    }
+
+    if (eventFrequencyOption.getValue() == 0) {
+      return;
+    }
+
+    String type = "";
+    String message = "";
+    boolean eventFinished = false;
+    switch (nextEventChoice) {
+    case 0:
+      if (numActiveKernels > 1 && numActiveKernels > numClusterOption.getValue() - numClusterRangeOption.getValue()) {
+        message = mergeKernels(nextEventCounter);
+        type = "Merge";
+      }
+      if (mergeClusterA == null && mergeClusterB == null && message.startsWith("Clusters merging")) {
+        eventFinished = true;
+      }
+      break;
+    case 1:
+      if (nextEventCounter <= 0) {
+        if (numActiveKernels < numClusterOption.getValue() + numClusterRangeOption.getValue()) {
+          type = "Split";
+          message = splitKernel();
+        }
+        eventFinished = true;
+      }
+      break;
+    case 2:
+      if (nextEventCounter <= 0) {
+        if (numActiveKernels > 1 && numActiveKernels > numClusterOption.getValue() - numClusterRangeOption.getValue()) {
+          message = fadeOut();
+          type = "Delete";
+        }
+        eventFinished = true;
+      }
+      break;
+    case 3:
+      if (nextEventCounter <= 0) {
+        if (numActiveKernels < numClusterOption.getValue() + numClusterRangeOption.getValue()) {
+          message = fadeIn();
+          type = "Create";
+        }
+        eventFinished = true;
+      }
+      break;
+
+    }
+    if (eventFinished) {
+      nextEventCounter = (int) (eventFrequencyOption.getValue() + (instanceRandom.nextBoolean() ? -1 : 1)
+          * eventFrequencyOption.getValue() * eventFrequencyRange * instanceRandom.nextDouble());
+      nextEventChoice = getNextEvent();
+      // System.out.println("Next event choice: "+nextEventChoice);
+    }
+    if (!message.isEmpty()) {
+      message += " (numKernels = " + numActiveKernels + " at " + numGeneratedInstances + ")";
+      if (!type.equals("Merge") || message.startsWith("Clusters merging"))
+        fireClusterChange(numGeneratedInstances, type, message);
+    }
+  }
+
+  private int getNextEvent() {
+    int choice = -1;
+    boolean lowerLimit = numActiveKernels <= numClusterOption.getValue() - numClusterRangeOption.getValue();
+    boolean upperLimit = numActiveKernels >= numClusterOption.getValue() + numClusterRangeOption.getValue();
+
+    if (!lowerLimit || !upperLimit) {
+      int mode = -1;
+      if (eventDeleteCreateOption.isSet() && eventMergeSplitOption.isSet()) {
+        mode = instanceRandom.nextInt(2);
+      }
+
+      if (mode == 0 || (mode == -1 && eventMergeSplitOption.isSet())) {
+        // have we reached a limit? if not free choice
+        if (!lowerLimit && !upperLimit)
+          choice = instanceRandom.nextInt(2);
+        else
+        // we have a limit. if lower limit, choose split
+        if (lowerLimit)
+          choice = 1;
+        // otherwise we reached upper level, choose merge
+        else
+          choice = 0;
+      }
+
+      if (mode == 1 || (mode == -1 && eventDeleteCreateOption.isSet())) {
+        // have we reached a limit? if not free choice
+        if (!lowerLimit && !upperLimit)
+          choice = instanceRandom.nextInt(2) + 2;
+        else
+        // we have a limit. if lower limit, choose create
+        if (lowerLimit)
+          choice = 3;
+        // otherwise we reached upper level, choose delete
+        else
+          choice = 2;
+      }
+    }
+
+    return choice;
+  }
+
+  private String fadeOut() {
+    int id = instanceRandom.nextInt(kernels.size());
+    while (kernels.get(id).kill != -1)
+      id = instanceRandom.nextInt(kernels.size());
+
+    String message = kernels.get(id).fadeOut();
+    return message;
+  }
+
+  private String fadeIn() {
+    GeneratorCluster gc = new GeneratorCluster(clusterIdCounter++);
+    kernels.add(gc);
+    numActiveKernels++;
+    normalizeWeights();
+    return "Creating new cluster";
+  }
+
+  private String changeWeight(boolean increase) {
+    double changeRate = 0.1;
+    int id = instanceRandom.nextInt(kernels.size());
+    while (kernels.get(id).kill != -1)
+      id = instanceRandom.nextInt(kernels.size());
+
+    int sign = 1;
+    if (!increase)
+      sign = -1;
+    double weight_old = kernels.get(id).generator.getWeight();
+    double delta = sign * numActiveKernels * instanceRandom.nextDouble() * changeRate;
+    kernels.get(id).generator.setWeight(weight_old + delta);
+
+    normalizeWeights();
+
+    String message;
+    if (increase)
+      message = "Increase ";
+    else
+      message = "Decrease ";
+    message += " weight on Cluster " + id + " from " + weight_old + " to " + (weight_old + delta);
+    return message;
+
+  }
+
+  private String changeRadius(boolean increase) {
+    double maxChangeRate = 0.1;
+    int id = instanceRandom.nextInt(kernels.size());
+    while (kernels.get(id).kill != -1)
+      id = instanceRandom.nextInt(kernels.size());
+
+    int sign = 1;
+    if (!increase)
+      sign = -1;
+
+    double r_old = kernels.get(id).generator.getRadius();
+    double r_new = r_old + sign * r_old * instanceRandom.nextDouble() * maxChangeRate;
+    if (r_new >= 0.5)
+      return "Radius to big";
+    kernels.get(id).generator.setRadius(r_new);
+
+    String message;
+    if (increase)
+      message = "Increase ";
+    else
+      message = "Decrease ";
+    message += " radius on Cluster " + id + " from " + r_old + " to " + r_new;
+    return message;
+  }
+
+  private String splitKernel() {
+    int id = instanceRandom.nextInt(kernels.size());
+    while (kernels.get(id).kill != -1)
+      id = instanceRandom.nextInt(kernels.size());
+
+    String message = kernels.get(id).splitKernel();
+
+    return message;
+  }
+
+  private String mergeKernels(int steps) {
+    if (numActiveKernels > 1 && ((mergeClusterA == null && mergeClusterB == null))) {
+
+      // choose clusters to merge
+      double diseredDist = steps / speedOption.getValue() * maxDistanceMoveThresholdByStep;
+      double minDist = Double.MAX_VALUE;
+      // System.out.println("DisredDist:"+(2*diseredDist));
+      for (int i = 0; i < kernels.size(); i++) {
+        for (int j = 0; j < i; j++) {
+          if (kernels.get(i).kill != -1 || kernels.get(j).kill != -1) {
+            continue;
+          }
+          else {
+            double kernelDist = kernels.get(i).generator.getCenterDistance(kernels.get(j).generator);
+            double d = kernelDist - 2 * diseredDist;
+            // System.out.println("Dist:"+i+" / "+j+" "+d);
+            if (Math.abs(d) < minDist &&
+                (minDist != Double.MAX_VALUE || d > 0 || Math.abs(d) < 0.001)) {
+              minDist = Math.abs(d);
+              mergeClusterA = kernels.get(i);
+              mergeClusterB = kernels.get(j);
+            }
+          }
+        }
+      }
+
+      if (mergeClusterA != null && mergeClusterB != null) {
+        double[] merge_point = mergeClusterA.generator.getCenter();
+        double[] v = mergeClusterA.generator.getDistanceVector(mergeClusterB.generator);
+        for (int i = 0; i < v.length; i++) {
+          merge_point[i] = merge_point[i] + v[i] * 0.5;
+        }
+
+        mergeClusterA.merging = true;
+        mergeClusterB.merging = true;
+        mergeClusterA.setDesitnation(merge_point);
+        mergeClusterB.setDesitnation(merge_point);
+
+        if (debug) {
+          System.out.println("Center1" + Arrays.toString(mergeClusterA.generator.getCenter()));
+          System.out.println("Center2" + Arrays.toString(mergeClusterB.generator.getCenter()));
+          System.out.println("Vector" + Arrays.toString(v));
+
+          System.out.println("Try to merge cluster " + mergeClusterA.generator.getId() +
+              " into " + mergeClusterB.generator.getId() +
+              " at " + Arrays.toString(merge_point) +
+              " time " + numGeneratedInstances);
+        }
+        return "Init merge";
+      }
+    }
+
+    if (mergeClusterA != null && mergeClusterB != null) {
+
+      // movekernels will move the kernels close to each other,
+      // we just need to check and merge here if they are close enough
+      return mergeClusterA.tryMerging(mergeClusterB);
+    }
+
+    return "";
+  }
+
+  /************************* TOOLS **************************************/
+
+  public void getDescription(StringBuilder sb, int indent) {
+
+  }
+
+  private double[] getNoisePoint() {
+    double[] sample = new double[numAttsOption.getValue()];
+    boolean incluster = true;
+    int counter = 20;
+    while (incluster) {
+      for (int j = 0; j < numAttsOption.getValue(); j++) {
+        sample[j] = instanceRandom.nextDouble();
+      }
+      incluster = false;
+      if (!noiseInClusterOption.isSet() && counter > 0) {
+        counter--;
+        for (int c = 0; c < kernels.size(); c++) {
+          for (int m = 0; m < kernels.get(c).microClusters.size(); m++) {
+            Instance inst = new DenseInstance(1, sample);
+            if (kernels.get(c).microClusters.get(m).getInclusionProbability(inst) > 0) {
+              incluster = true;
+              break;
+            }
+          }
+          if (incluster)
+            break;
+        }
+      }
+    }
+
+    // double [] sample = new double [numAttsOption.getValue()];
+    // for (int j = 0; j < numAttsOption.getValue(); j++) {
+    // sample[j] = instanceRandom.nextDouble();
+    // }
+
+    return sample;
+  }
+
+  private int chooseWeightedElement() {
+    double r = instanceRandom.nextDouble();
+
+    // Determine index of choosen element
+    int i = 0;
+    while (r > 0.0) {
+      r -= kernels.get(i).generator.getWeight();
+      i++;
+    }
+    --i; // Overcounted once
+    // System.out.println(i);
+    return i;
+  }
+
+  private void normalizeWeights() {
+    double sumWeights = 0.0;
+    for (int i = 0; i < kernels.size(); i++) {
+      sumWeights += kernels.get(i).generator.getWeight();
+    }
+    for (int i = 0; i < kernels.size(); i++) {
+      kernels.get(i).generator.setWeight(kernels.get(i).generator.getWeight() / sumWeights);
+    }
+  }
+
+  /*************** EVENT Listener *********************/
+  // should go into the superclass of the generator, create new one for cluster
+  // streams?
+
+  /** Add a listener */
+  synchronized public void addClusterChangeListener(ClusterEventListener l) {
+    if (listeners == null)
+      listeners = new Vector();
+    listeners.addElement(l);
+  }
+
+  /** Remove a listener */
+  synchronized public void removeClusterChangeListener(ClusterEventListener l) {
+    if (listeners == null)
+      listeners = new Vector();
+    listeners.removeElement(l);
+  }
+
+  /** Fire a ClusterChangeEvent to all registered listeners */
+  protected void fireClusterChange(long timestamp, String type, String message) {
+    // if we have no listeners, do nothing...
+    if (listeners != null && !listeners.isEmpty()) {
+      // create the event object to send
+      ClusterEvent event =
+          new ClusterEvent(this, timestamp, type, message);
+
+      // make a copy of the listener list in case
+      // anyone adds/removes listeners
+      Vector targets;
+      synchronized (this) {
+        targets = (Vector) listeners.clone();
+      }
+
+      // walk through the listener list and
+      // call the sunMoved method in each
+      Enumeration e = targets.elements();
+      while (e.hasMoreElements()) {
+        ClusterEventListener l = (ClusterEventListener) e.nextElement();
+        l.changeCluster(event);
+
+      }
+    }
+  }
+
+  @Override
+  public String getPurposeString() {
+    return "Generates a random radial basis function stream.";
+  }
+
+  public String getParameterString() {
+    return "";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/generators/HyperplaneGenerator.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/generators/HyperplaneGenerator.java b/samoa-api/src/main/java/org/apache/samoa/streams/generators/HyperplaneGenerator.java
new file mode 100644
index 0000000..8274040
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/generators/HyperplaneGenerator.java
@@ -0,0 +1,186 @@
+package org.apache.samoa.streams.generators;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import java.util.Random;
+
+import org.apache.samoa.instances.Attribute;
+import org.apache.samoa.instances.DenseInstance;
+import org.apache.samoa.instances.Instance;
+import org.apache.samoa.instances.Instances;
+import org.apache.samoa.instances.InstancesHeader;
+import org.apache.samoa.moa.core.Example;
+import org.apache.samoa.moa.core.FastVector;
+import org.apache.samoa.moa.core.InstanceExample;
+import org.apache.samoa.moa.core.ObjectRepository;
+import org.apache.samoa.moa.options.AbstractOptionHandler;
+import org.apache.samoa.moa.tasks.TaskMonitor;
+import org.apache.samoa.streams.InstanceStream;
+
+import com.github.javacliparser.FloatOption;
+import com.github.javacliparser.IntOption;
+
+/**
+ * Stream generator for Hyperplane data stream.
+ * 
+ * @author Albert Bifet (abifet at cs dot waikato dot ac dot nz)
+ * @version $Revision: 7 $
+ */
+public class HyperplaneGenerator extends AbstractOptionHandler implements InstanceStream {
+
+  @Override
+  public String getPurposeString() {
+    return "Generates a problem of predicting class of a rotating hyperplane.";
+  }
+
+  private static final long serialVersionUID = 1L;
+
+  public IntOption instanceRandomSeedOption = new IntOption("instanceRandomSeed", 'i',
+      "Seed for random generation of instances.", 1);
+
+  public IntOption numClassesOption = new IntOption("numClasses", 'c', "The number of classes to generate.", 2, 2,
+      Integer.MAX_VALUE);
+
+  public IntOption numAttsOption = new IntOption("numAtts", 'a', "The number of attributes to generate.", 10, 0,
+      Integer.MAX_VALUE);
+
+  public IntOption numDriftAttsOption = new IntOption("numDriftAtts", 'k', "The number of attributes with drift.", 2,
+      0, Integer.MAX_VALUE);
+
+  public FloatOption magChangeOption = new FloatOption("magChange", 't', "Magnitude of the change for every example",
+      0.0, 0.0, 1.0);
+
+  public IntOption noisePercentageOption = new IntOption("noisePercentage", 'n',
+      "Percentage of noise to add to the data.", 5, 0, 100);
+
+  public IntOption sigmaPercentageOption = new IntOption("sigmaPercentage", 's',
+      "Percentage of probability that the direction of change is reversed.", 10,
+      0, 100);
+
+  protected InstancesHeader streamHeader;
+
+  protected Random instanceRandom;
+
+  protected double[] weights;
+
+  protected int[] sigma;
+
+  public int numberInstance;
+
+  @Override
+  protected void prepareForUseImpl(TaskMonitor monitor, ObjectRepository repository) {
+    monitor.setCurrentActivity("Preparing hyperplane...", -1.0);
+    generateHeader();
+    restart();
+  }
+
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  protected void generateHeader() {
+    FastVector attributes = new FastVector();
+    for (int i = 0; i < this.numAttsOption.getValue(); i++) {
+      attributes.addElement(new Attribute("att" + (i + 1)));
+    }
+
+    FastVector classLabels = new FastVector();
+    for (int i = 0; i < this.numClassesOption.getValue(); i++) {
+      classLabels.addElement("class" + (i + 1));
+    }
+    attributes.addElement(new Attribute("class", classLabels));
+    this.streamHeader = new InstancesHeader(new Instances(getCLICreationString(InstanceStream.class), attributes, 0));
+    this.streamHeader.setClassIndex(this.streamHeader.numAttributes() - 1);
+  }
+
+  @Override
+  public long estimatedRemainingInstances() {
+    return -1;
+  }
+
+  @Override
+  public InstancesHeader getHeader() {
+    return this.streamHeader;
+  }
+
+  @Override
+  public boolean hasMoreInstances() {
+    return true;
+  }
+
+  @Override
+  public boolean isRestartable() {
+    return true;
+  }
+
+  @Override
+  public Example<Instance> nextInstance() {
+
+    int numAtts = this.numAttsOption.getValue();
+    double[] attVals = new double[numAtts + 1];
+    double sum = 0.0;
+    double sumWeights = 0.0;
+    for (int i = 0; i < numAtts; i++) {
+      attVals[i] = this.instanceRandom.nextDouble();
+      sum += this.weights[i] * attVals[i];
+      sumWeights += this.weights[i];
+    }
+    int classLabel;
+    if (sum >= sumWeights * 0.5) {
+      classLabel = 1;
+    } else {
+      classLabel = 0;
+    }
+    // Add Noise
+    if ((1 + (this.instanceRandom.nextInt(100))) <= this.noisePercentageOption.getValue()) {
+      classLabel = (classLabel == 0 ? 1 : 0);
+    }
+
+    Instance inst = new DenseInstance(1.0, attVals);
+    inst.setDataset(getHeader());
+    inst.setClassValue(classLabel);
+    addDrift();
+    return new InstanceExample(inst);
+  }
+
+  private void addDrift() {
+    for (int i = 0; i < this.numDriftAttsOption.getValue(); i++) {
+      this.weights[i] += (double) ((double) sigma[i]) * ((double) this.magChangeOption.getValue());
+      if (// this.weights[i] >= 1.0 || this.weights[i] <= 0.0 ||
+      (1 + (this.instanceRandom.nextInt(100))) <= this.sigmaPercentageOption.getValue()) {
+        this.sigma[i] *= -1;
+      }
+    }
+  }
+
+  @Override
+  public void restart() {
+    this.instanceRandom = new Random(this.instanceRandomSeedOption.getValue());
+    this.weights = new double[this.numAttsOption.getValue()];
+    this.sigma = new int[this.numAttsOption.getValue()];
+    for (int i = 0; i < this.numAttsOption.getValue(); i++) {
+      this.weights[i] = this.instanceRandom.nextDouble();
+      this.sigma[i] = (i < this.numDriftAttsOption.getValue() ? 1 : 0);
+    }
+  }
+
+  @Override
+  public void getDescription(StringBuilder sb, int indent) {
+    // TODO Auto-generated method stub
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/streams/generators/RandomTreeGenerator.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/streams/generators/RandomTreeGenerator.java b/samoa-api/src/main/java/org/apache/samoa/streams/generators/RandomTreeGenerator.java
new file mode 100644
index 0000000..70dc4f8
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/streams/generators/RandomTreeGenerator.java
@@ -0,0 +1,267 @@
+package org.apache.samoa.streams.generators;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Random;
+
+import org.apache.samoa.instances.Attribute;
+import org.apache.samoa.instances.DenseInstance;
+import org.apache.samoa.instances.Instance;
+import org.apache.samoa.instances.Instances;
+import org.apache.samoa.instances.InstancesHeader;
+import org.apache.samoa.moa.core.FastVector;
+import org.apache.samoa.moa.core.InstanceExample;
+import org.apache.samoa.moa.core.ObjectRepository;
+import org.apache.samoa.moa.options.AbstractOptionHandler;
+import org.apache.samoa.moa.tasks.TaskMonitor;
+import org.apache.samoa.streams.InstanceStream;
+
+import com.github.javacliparser.FloatOption;
+import com.github.javacliparser.IntOption;
+
+/**
+ * Stream generator for a stream based on a randomly generated tree..
+ * 
+ * @author Richard Kirkby (rkirkby@cs.waikato.ac.nz)
+ * @version $Revision: 7 $
+ */
+public class RandomTreeGenerator extends AbstractOptionHandler implements InstanceStream {
+
+  @Override
+  public String getPurposeString() {
+    return "Generates a stream based on a randomly generated tree.";
+  }
+
+  private static final long serialVersionUID = 1L;
+
+  public IntOption treeRandomSeedOption = new IntOption("treeRandomSeed",
+      'r', "Seed for random generation of tree.", 1);
+
+  public IntOption instanceRandomSeedOption = new IntOption(
+      "instanceRandomSeed", 'i',
+      "Seed for random generation of instances.", 1);
+
+  public IntOption numClassesOption = new IntOption("numClasses", 'c',
+      "The number of classes to generate.", 2, 2, Integer.MAX_VALUE);
+
+  public IntOption numNominalsOption = new IntOption("numNominals", 'o',
+      "The number of nominal attributes to generate.", 5, 0,
+      Integer.MAX_VALUE);
+
+  public IntOption numNumericsOption = new IntOption("numNumerics", 'u',
+      "The number of numeric attributes to generate.", 5, 0,
+      Integer.MAX_VALUE);
+
+  public IntOption numValsPerNominalOption = new IntOption(
+      "numValsPerNominal", 'v',
+      "The number of values to generate per nominal attribute.", 5, 2,
+      Integer.MAX_VALUE);
+
+  public IntOption maxTreeDepthOption = new IntOption("maxTreeDepth", 'd',
+      "The maximum depth of the tree concept.", 5, 0, Integer.MAX_VALUE);
+
+  public IntOption firstLeafLevelOption = new IntOption(
+      "firstLeafLevel",
+      'l',
+      "The first level of the tree above maxTreeDepth that can have leaves.",
+      3, 0, Integer.MAX_VALUE);
+
+  public FloatOption leafFractionOption = new FloatOption("leafFraction",
+      'f',
+      "The fraction of leaves per level from firstLeafLevel onwards.",
+      0.15, 0.0, 1.0);
+
+  protected static class Node implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    public int classLabel;
+
+    public int splitAttIndex;
+
+    public double splitAttValue;
+
+    public Node[] children;
+  }
+
+  protected Node treeRoot;
+
+  protected InstancesHeader streamHeader;
+
+  protected Random instanceRandom;
+
+  @Override
+  public void prepareForUseImpl(TaskMonitor monitor,
+      ObjectRepository repository) {
+    monitor.setCurrentActivity("Preparing random tree...", -1.0);
+    generateHeader();
+    generateRandomTree();
+    restart();
+  }
+
+  @Override
+  public long estimatedRemainingInstances() {
+    return -1;
+  }
+
+  @Override
+  public boolean isRestartable() {
+    return true;
+  }
+
+  @Override
+  public void restart() {
+    this.instanceRandom = new Random(this.instanceRandomSeedOption.getValue());
+  }
+
+  @Override
+  public InstancesHeader getHeader() {
+    return this.streamHeader;
+  }
+
+  @Override
+  public boolean hasMoreInstances() {
+    return true;
+  }
+
+  @Override
+  public InstanceExample nextInstance() {
+    double[] attVals = new double[this.numNominalsOption.getValue()
+        + this.numNumericsOption.getValue()];
+    InstancesHeader header = getHeader();
+    Instance inst = new DenseInstance(header.numAttributes());
+    for (int i = 0; i < attVals.length; i++) {
+      attVals[i] = i < this.numNominalsOption.getValue() ? this.instanceRandom.nextInt(this.numValsPerNominalOption
+          .getValue())
+          : this.instanceRandom.nextDouble();
+      inst.setValue(i, attVals[i]);
+    }
+    inst.setDataset(header);
+    inst.setClassValue(classifyInstance(this.treeRoot, attVals));
+    return new InstanceExample(inst);
+  }
+
+  protected int classifyInstance(Node node, double[] attVals) {
+    if (node.children == null) {
+      return node.classLabel;
+    }
+    if (node.splitAttIndex < this.numNominalsOption.getValue()) {
+      return classifyInstance(
+          node.children[(int) attVals[node.splitAttIndex]], attVals);
+    }
+    return classifyInstance(
+        node.children[attVals[node.splitAttIndex] < node.splitAttValue ? 0
+            : 1], attVals);
+  }
+
+  protected void generateHeader() {
+    FastVector<Attribute> attributes = new FastVector<>();
+    FastVector<String> nominalAttVals = new FastVector<>();
+    for (int i = 0; i < this.numValsPerNominalOption.getValue(); i++) {
+      nominalAttVals.addElement("value" + (i + 1));
+    }
+    for (int i = 0; i < this.numNominalsOption.getValue(); i++) {
+      attributes.addElement(new Attribute("nominal" + (i + 1),
+          nominalAttVals));
+    }
+    for (int i = 0; i < this.numNumericsOption.getValue(); i++) {
+      attributes.addElement(new Attribute("numeric" + (i + 1)));
+    }
+    FastVector<String> classLabels = new FastVector<>();
+    for (int i = 0; i < this.numClassesOption.getValue(); i++) {
+      classLabels.addElement("class" + (i + 1));
+    }
+    attributes.addElement(new Attribute("class", classLabels));
+    this.streamHeader = new InstancesHeader(new Instances(
+        getCLICreationString(InstanceStream.class), attributes, 0));
+    this.streamHeader.setClassIndex(this.streamHeader.numAttributes() - 1);
+  }
+
+  protected void generateRandomTree() {
+    Random treeRand = new Random(this.treeRandomSeedOption.getValue());
+    ArrayList<Integer> nominalAttCandidates = new ArrayList<>(
+        this.numNominalsOption.getValue());
+    for (int i = 0; i < this.numNominalsOption.getValue(); i++) {
+      nominalAttCandidates.add(i);
+    }
+    double[] minNumericVals = new double[this.numNumericsOption.getValue()];
+    double[] maxNumericVals = new double[this.numNumericsOption.getValue()];
+    for (int i = 0; i < this.numNumericsOption.getValue(); i++) {
+      minNumericVals[i] = 0.0;
+      maxNumericVals[i] = 1.0;
+    }
+    this.treeRoot = generateRandomTreeNode(0, nominalAttCandidates,
+        minNumericVals, maxNumericVals, treeRand);
+  }
+
+  protected Node generateRandomTreeNode(int currentDepth,
+      ArrayList<Integer> nominalAttCandidates, double[] minNumericVals,
+      double[] maxNumericVals, Random treeRand) {
+    if ((currentDepth >= this.maxTreeDepthOption.getValue())
+        || ((currentDepth >= this.firstLeafLevelOption.getValue()) && (this.leafFractionOption.getValue() >= (1.0 - treeRand
+            .nextDouble())))) {
+      Node leaf = new Node();
+      leaf.classLabel = treeRand.nextInt(this.numClassesOption.getValue());
+      return leaf;
+    }
+    Node node = new Node();
+    int chosenAtt = treeRand.nextInt(nominalAttCandidates.size()
+        + this.numNumericsOption.getValue());
+    if (chosenAtt < nominalAttCandidates.size()) {
+      node.splitAttIndex = nominalAttCandidates.get(chosenAtt);
+      node.children = new Node[this.numValsPerNominalOption.getValue()];
+      ArrayList<Integer> newNominalCandidates = new ArrayList<>(
+          nominalAttCandidates);
+      newNominalCandidates.remove(new Integer(node.splitAttIndex));
+      newNominalCandidates.trimToSize();
+      for (int i = 0; i < node.children.length; i++) {
+        node.children[i] = generateRandomTreeNode(currentDepth + 1,
+            newNominalCandidates, minNumericVals, maxNumericVals,
+            treeRand);
+      }
+    } else {
+      int numericIndex = chosenAtt - nominalAttCandidates.size();
+      node.splitAttIndex = this.numNominalsOption.getValue()
+          + numericIndex;
+      double minVal = minNumericVals[numericIndex];
+      double maxVal = maxNumericVals[numericIndex];
+      node.splitAttValue = ((maxVal - minVal) * treeRand.nextDouble())
+          + minVal;
+      node.children = new Node[2];
+      double[] newMaxVals = maxNumericVals.clone();
+      newMaxVals[numericIndex] = node.splitAttValue;
+      node.children[0] = generateRandomTreeNode(currentDepth + 1,
+          nominalAttCandidates, minNumericVals, newMaxVals, treeRand);
+      double[] newMinVals = minNumericVals.clone();
+      newMinVals[numericIndex] = node.splitAttValue;
+      node.children[1] = generateRandomTreeNode(currentDepth + 1,
+          nominalAttCandidates, newMinVals, maxNumericVals, treeRand);
+    }
+    return node;
+  }
+
+  @Override
+  public void getDescription(StringBuilder sb, int indent) {
+    // TODO Auto-generated method stub
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/tasks/ClusteringEvaluation.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/tasks/ClusteringEvaluation.java b/samoa-api/src/main/java/org/apache/samoa/tasks/ClusteringEvaluation.java
index a5c0ef1..327b1ad 100644
--- a/samoa-api/src/main/java/org/apache/samoa/tasks/ClusteringEvaluation.java
+++ b/samoa-api/src/main/java/org/apache/samoa/tasks/ClusteringEvaluation.java
@@ -26,10 +26,10 @@ import org.apache.samoa.evaluation.ClusteringEvaluatorProcessor;
 import org.apache.samoa.learners.Learner;
 import org.apache.samoa.learners.clusterers.simple.ClusteringDistributorProcessor;
 import org.apache.samoa.learners.clusterers.simple.DistributedClusterer;
-import org.apache.samoa.moa.streams.InstanceStream;
-import org.apache.samoa.moa.streams.clustering.ClusteringStream;
-import org.apache.samoa.moa.streams.clustering.RandomRBFGeneratorEvents;
 import org.apache.samoa.streams.ClusteringEntranceProcessor;
+import org.apache.samoa.streams.InstanceStream;
+import org.apache.samoa.streams.clustering.ClusteringStream;
+import org.apache.samoa.streams.clustering.RandomRBFGeneratorEvents;
 import org.apache.samoa.topology.ComponentFactory;
 import org.apache.samoa.topology.Stream;
 import org.apache.samoa.topology.Topology;

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-api/src/main/java/org/apache/samoa/tasks/PrequentialEvaluation.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/tasks/PrequentialEvaluation.java b/samoa-api/src/main/java/org/apache/samoa/tasks/PrequentialEvaluation.java
index 575adad..da0057d 100644
--- a/samoa-api/src/main/java/org/apache/samoa/tasks/PrequentialEvaluation.java
+++ b/samoa-api/src/main/java/org/apache/samoa/tasks/PrequentialEvaluation.java
@@ -32,9 +32,9 @@ import org.apache.samoa.learners.ClassificationLearner;
 import org.apache.samoa.learners.Learner;
 import org.apache.samoa.learners.RegressionLearner;
 import org.apache.samoa.learners.classifiers.trees.VerticalHoeffdingTree;
-import org.apache.samoa.moa.streams.InstanceStream;
-import org.apache.samoa.moa.streams.generators.RandomTreeGenerator;
+import org.apache.samoa.streams.InstanceStream;
 import org.apache.samoa.streams.PrequentialSourceProcessor;
+import org.apache.samoa.streams.generators.RandomTreeGenerator;
 import org.apache.samoa.topology.ComponentFactory;
 import org.apache.samoa.topology.Stream;
 import org.apache.samoa.topology.Topology;

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bd1012a/samoa-test/src/test/java/org/apache/samoa/TestParams.java
----------------------------------------------------------------------
diff --git a/samoa-test/src/test/java/org/apache/samoa/TestParams.java b/samoa-test/src/test/java/org/apache/samoa/TestParams.java
index 0d9d639..615d4da 100644
--- a/samoa-test/src/test/java/org/apache/samoa/TestParams.java
+++ b/samoa-test/src/test/java/org/apache/samoa/TestParams.java
@@ -36,18 +36,18 @@ public class TestParams {
 
     public final static String PREQEVAL_VHT_RANDOMTREE = "PrequentialEvaluation -d %s -i %d -f %d -w %d "
         + "-l (org.apache.samoa.learners.classifiers.trees.VerticalHoeffdingTree -p 4) " +
-        "-s (org.apache.samoa.moa.streams.generators.RandomTreeGenerator -c 2 -o 10 -u 10)";
+        "-s (org.apache.samoa.streams.generators.RandomTreeGenerator -c 2 -o 10 -u 10)";
 
     public final static String PREQEVAL_NAIVEBAYES_HYPERPLANE = "PrequentialEvaluation -d %s -i %d -f %d -w %d "
         + "-l (classifiers.SingleClassifier -l org.apache.samoa.learners.classifiers.NaiveBayes) " +
-        "-s (org.apache.samoa.moa.streams.generators.HyperplaneGenerator -c 2)";
+        "-s (org.apache.samoa.streams.generators.HyperplaneGenerator -c 2)";
 
     // setting the number of nominal attributes to zero significantly reduces
     // the processing time,
     // so that it's acceptable in a test case
     public final static String PREQEVAL_BAGGING_RANDOMTREE = "PrequentialEvaluation -d %s -i %d -f %d -w %d "
         + "-l (org.apache.samoa.learners.classifiers.ensemble.Bagging) " +
-        "-s (org.apache.samoa.moa.streams.generators.RandomTreeGenerator -c 2 -o 0 -u 10)";
+        "-s (org.apache.samoa.streams.generators.RandomTreeGenerator -c 2 -o 0 -u 10)";
 
   }
 


[13/17] incubator-samoa git commit: SAMOA-31: Remove unused Utils class Close #49

Posted by gd...@apache.org.
SAMOA-31: Remove unused Utils class
Close #49


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

Branch: refs/heads/master
Commit: c1d42d2f3c066d1e00845146e8d58b376501570e
Parents: 8f888b7
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Sun Mar 6 11:52:05 2016 +0300
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Sun Mar 13 10:59:04 2016 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/samoa/utils/Utils.java | 183 -------------------
 1 file changed, 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/c1d42d2f/samoa-api/src/main/java/org/apache/samoa/utils/Utils.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/utils/Utils.java b/samoa-api/src/main/java/org/apache/samoa/utils/Utils.java
deleted file mode 100644
index f8ae5ef..0000000
--- a/samoa-api/src/main/java/org/apache/samoa/utils/Utils.java
+++ /dev/null
@@ -1,183 +0,0 @@
-package org.apache.samoa.utils;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.jar.Attributes;
-import java.util.jar.JarOutputStream;
-import java.util.jar.Manifest;
-import java.util.zip.ZipEntry;
-
-/**
- * Utils class for building and deploying applications programmatically.
- * 
- * @author severien
- * 
- */
-public class Utils {
-
-  public static void buildSamoaPackage() {
-    try {
-      String output = "/tmp/samoa/samoa.jar";// System.getProperty("user.home") + "/samoa.jar";
-      Manifest manifest = createManifest();
-
-      BufferedOutputStream bo;
-
-      bo = new BufferedOutputStream(new FileOutputStream(output));
-      JarOutputStream jo = new JarOutputStream(bo, manifest);
-
-      String baseDir = System.getProperty("user.dir");
-      System.out.println(baseDir);
-
-      File samoaJar = new File(baseDir + "/target/samoa-0.0.1-SNAPSHOT.jar");
-      addEntry(jo, samoaJar, baseDir + "/target/", "/app/");
-      addLibraries(jo);
-
-      jo.close();
-      bo.close();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-
-  }
-
-  // TODO should get the modules file from the parameters
-  public static void buildModulesPackage(List<String> modulesNames) {
-    System.out.println(System.getProperty("user.dir"));
-    try {
-      String baseDir = System.getProperty("user.dir");
-      List<File> filesArray = new ArrayList<>();
-      for (String module : modulesNames) {
-        module = "/" + module.replace(".", "/") + ".class";
-        filesArray.add(new File(baseDir + module));
-      }
-      String output = System.getProperty("user.home") + "/modules.jar";
-
-      Manifest manifest = new Manifest();
-      manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION,
-          "1.0");
-      manifest.getMainAttributes().put(Attributes.Name.IMPLEMENTATION_URL,
-          "http://samoa.yahoo.com");
-      manifest.getMainAttributes().put(
-          Attributes.Name.IMPLEMENTATION_VERSION, "0.1");
-      manifest.getMainAttributes().put(Attributes.Name.IMPLEMENTATION_VENDOR,
-          "Yahoo");
-      manifest.getMainAttributes().put(
-          Attributes.Name.IMPLEMENTATION_VENDOR_ID, "SAMOA");
-
-      BufferedOutputStream bo;
-
-      bo = new BufferedOutputStream(new FileOutputStream(output));
-      JarOutputStream jo = new JarOutputStream(bo, manifest);
-
-      File[] files = filesArray.toArray(new File[filesArray.size()]);
-      addEntries(jo, files, baseDir, "");
-
-      jo.close();
-      bo.close();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-
-  }
-
-  private static void addLibraries(JarOutputStream jo) {
-    try {
-      String baseDir = System.getProperty("user.dir");
-      String libDir = baseDir + "/target/lib";
-      File inputFile = new File(libDir);
-
-      File[] files = inputFile.listFiles();
-      for (File file : files) {
-        addEntry(jo, file, baseDir, "lib");
-      }
-      jo.close();
-
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-  }
-
-  private static void addEntries(JarOutputStream jo, File[] files, String baseDir, String rootDir) {
-    for (File file : files) {
-
-      if (!file.isDirectory()) {
-        addEntry(jo, file, baseDir, rootDir);
-      } else {
-        File dir = new File(file.getAbsolutePath());
-        addEntries(jo, dir.listFiles(), baseDir, rootDir);
-      }
-    }
-  }
-
-  private static void addEntry(JarOutputStream jo, File file, String baseDir, String rootDir) {
-    try {
-      BufferedInputStream bi = new BufferedInputStream(new FileInputStream(file));
-
-      String path = file.getAbsolutePath().replaceFirst(baseDir, rootDir);
-      jo.putNextEntry(new ZipEntry(path));
-
-      byte[] buf = new byte[1024];
-      int anz;
-      while ((anz = bi.read(buf)) != -1) {
-        jo.write(buf, 0, anz);
-      }
-      bi.close();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-  }
-
-  public static Manifest createManifest() {
-    Manifest manifest = new Manifest();
-    manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION, "1.0");
-    manifest.getMainAttributes().put(Attributes.Name.IMPLEMENTATION_URL, "http://samoa.yahoo.com");
-    manifest.getMainAttributes().put(Attributes.Name.IMPLEMENTATION_VERSION, "0.1");
-    manifest.getMainAttributes().put(Attributes.Name.IMPLEMENTATION_VENDOR, "Yahoo");
-    manifest.getMainAttributes().put(Attributes.Name.IMPLEMENTATION_VENDOR_ID, "SAMOA");
-    Attributes s4Attributes = new Attributes();
-    s4Attributes.putValue("S4-App-Class", "path.to.Class");
-    Attributes.Name name = new Attributes.Name("S4-App-Class");
-    Attributes.Name S4Version = new Attributes.Name("S4-Version");
-    manifest.getMainAttributes().put(name, "samoa.topology.impl.DoTaskApp");
-    manifest.getMainAttributes().put(S4Version, "0.6.0-incubating");
-    return manifest;
-  }
-
-  public static Object getInstance(String className) {
-    Class<?> cls;
-    Object obj = null;
-    try {
-      cls = Class.forName(className);
-      obj = cls.newInstance();
-    } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) {
-      e.printStackTrace();
-    }
-    return obj;
-  }
-}


[14/17] incubator-samoa git commit: SAMOA-28: Delete samoa-api/build directory after tests

Posted by gd...@apache.org.
SAMOA-28: Delete samoa-api/build directory after tests


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

Branch: refs/heads/master
Commit: 1bf3c0270d759dec3bd6649e0418dc4e4cd509e7
Parents: c1d42d2
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Sun Mar 6 16:15:58 2016 +0300
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Sun Mar 13 11:02:20 2016 +0300

----------------------------------------------------------------------
 .../streams/fs/HDFSFileStreamSourceTest.java    | 24 ++++++++------------
 1 file changed, 10 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/1bf3c027/samoa-api/src/test/java/org/apache/samoa/streams/fs/HDFSFileStreamSourceTest.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/test/java/org/apache/samoa/streams/fs/HDFSFileStreamSourceTest.java b/samoa-api/src/test/java/org/apache/samoa/streams/fs/HDFSFileStreamSourceTest.java
index eaba37d..f6846cd 100644
--- a/samoa-api/src/test/java/org/apache/samoa/streams/fs/HDFSFileStreamSourceTest.java
+++ b/samoa-api/src/test/java/org/apache/samoa/streams/fs/HDFSFileStreamSourceTest.java
@@ -1,5 +1,9 @@
 package org.apache.samoa.streams.fs;
 
+import static org.junit.Assert.*;
+
+import java.io.BufferedReader;
+
 /*
  * #%L
  * SAMOA
@@ -21,29 +25,20 @@ package org.apache.samoa.streams.fs;
  */
 
 import java.io.BufferedWriter;
-import java.io.BufferedReader;
-import java.io.File;
+import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
-import java.util.HashSet;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSCluster.Builder;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.samoa.streams.fs.HDFSFileStreamSource;
-
-import static org.junit.Assert.*;
-
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -63,7 +58,9 @@ public class HDFSFileStreamSourceTest {
   @Before
   public void setUp() throws Exception {
     // Start MiniDFSCluster
-    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(new Configuration()).hosts(HOSTS).numDataNodes(1)
+    config = new Configuration();
+    config.set("hdfs.minidfs.basedir", "target/build/test/data/dfs");
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(config).hosts(HOSTS).numDataNodes(1)
         .format(true);
     hdfsCluster = builder.build();
     hdfsCluster.waitActive();
@@ -73,7 +70,6 @@ public class HDFSFileStreamSourceTest {
     streamSource = new HDFSFileStreamSource();
 
     // General config
-    config = new Configuration();
     config.set("fs.defaultFS", hdfsURI);
   }
 


[12/17] incubator-samoa git commit: SAMOA-57: Fix failing test (wrong number of instances)

Posted by gd...@apache.org.
SAMOA-57: Fix failing test (wrong number of instances)


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

Branch: refs/heads/master
Commit: 8f888b7150b3a71230a455b1e218fd9d98f596f9
Parents: de050f9
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Sun Mar 6 13:54:28 2016 +0300
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Sun Mar 6 13:54:28 2016 +0300

----------------------------------------------------------------------
 samoa-threads/src/test/java/org/apache/samoa/AlgosTest.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/8f888b71/samoa-threads/src/test/java/org/apache/samoa/AlgosTest.java
----------------------------------------------------------------------
diff --git a/samoa-threads/src/test/java/org/apache/samoa/AlgosTest.java b/samoa-threads/src/test/java/org/apache/samoa/AlgosTest.java
index 33130c1..031d98d 100644
--- a/samoa-threads/src/test/java/org/apache/samoa/AlgosTest.java
+++ b/samoa-threads/src/test/java/org/apache/samoa/AlgosTest.java
@@ -54,7 +54,7 @@ public class AlgosTest {
         .samplingSize(10_000)
         .inputDelayMicroSec(100) // prevents saturating the system due to unbounded queues
         .evaluationInstances(90_000)
-        .classifiedInstances(105_000)
+        .classifiedInstances(100_000)
         .classificationsCorrect(55f)
         .kappaStat(0f)
         .kappaTempStat(0f)


[06/17] incubator-samoa git commit: SAMOA-47: Integrate Avro Streams with SAMOA

Posted by gd...@apache.org.
SAMOA-47: Integrate Avro Streams with SAMOA


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

Branch: refs/heads/master
Commit: 533f12a12822fcde5dbaf48f4ef2c105f97e7087
Parents: 7497978
Author: jayadeepj <ja...@gmail.com>
Authored: Fri Oct 30 16:54:05 2015 +0530
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Mon Nov 30 13:20:35 2015 +0200

----------------------------------------------------------------------
 .../main/java/org/apache/samoa/moa/streams/AvroFileStream.java    | 1 -
 .../main/java/org/apache/samoa/instances/AvroBinaryLoader.java    | 3 +--
 .../src/main/java/org/apache/samoa/instances/AvroJsonLoader.java  | 1 -
 .../src/main/java/org/apache/samoa/instances/AvroLoader.java      | 1 -
 4 files changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/533f12a1/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
index e684687..0a98acf 100644
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
+++ b/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
@@ -40,7 +40,6 @@ import com.github.javacliparser.StringOption;
  *  InstanceStream implementation to handle Apache Avro Files.
  *  Handles both JSON & Binary encoded streams
  *  
- *  @author jayadeepj
  *
  */
 public class AvroFileStream extends FileStream {

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/533f12a1/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
index c3e32dc..ad07f62 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
@@ -33,7 +33,6 @@ import org.slf4j.LoggerFactory;
 /**
  * Load Data from Binary Avro Stream and parse to corresponding Dense & Parse Instances
  * 
- * @author jayadeepj
  *
  */
 public class AvroBinaryLoader extends AvroLoader {
@@ -94,7 +93,7 @@ public class AvroBinaryLoader extends AvroLoader {
 
 		if(record==null)
 		{
-			//closeReader();
+			closeReader();
 			return null;
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/533f12a1/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
index 827b507..8a345da 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
@@ -38,7 +38,6 @@ import org.slf4j.LoggerFactory;
 /**
  * Load Data from JSON Avro Stream and parse to corresponding Dense & Parse Instances
  * 
- * @author jayadeepj
  *
  */
 public class AvroJsonLoader extends AvroLoader {

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/533f12a1/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
index 09f410f..33db437 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
@@ -35,7 +35,6 @@ import org.apache.avro.io.DatumReader;
  * Load Data from Avro Stream and parse to corresponding Dense & Parse Instances
  * Abstract Class: Subclass this class for different types of Avro Encodings
  * 
- * @author jayadeepj
  *
  */
 public abstract class AvroLoader implements Loader {


[08/17] incubator-samoa git commit: SAMOA-47: Adding Code Review comments

Posted by gd...@apache.org.
SAMOA-47: Adding Code Review comments


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

Branch: refs/heads/master
Commit: 76a373639a18eab07cd2a8a52959ec5bb38e383f
Parents: 533f12a
Author: jayadeepj <ja...@gmail.com>
Authored: Sat Nov 14 12:52:18 2015 +0530
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Mon Nov 30 13:20:36 2015 +0200

----------------------------------------------------------------------
 .../samoa/moa/streams/AvroFileStream.java       | 251 +++++----
 .../org/apache/samoa/instances/ArffLoader.java  |  32 +-
 .../samoa/instances/AvroBinaryLoader.java       | 161 +++---
 .../apache/samoa/instances/AvroJsonLoader.java  | 177 ++++---
 .../org/apache/samoa/instances/AvroLoader.java  | 504 ++++++++++---------
 .../org/apache/samoa/instances/Instances.java   |  55 +-
 .../java/org/apache/samoa/instances/Loader.java |  30 +-
 7 files changed, 608 insertions(+), 602 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/76a37363/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
index 0a98acf..74b31dd 100644
--- a/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
+++ b/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
@@ -35,136 +35,133 @@ import com.github.javacliparser.FileOption;
 import com.github.javacliparser.IntOption;
 import com.github.javacliparser.StringOption;
 
-
 /**
- *  InstanceStream implementation to handle Apache Avro Files.
- *  Handles both JSON & Binary encoded streams
- *  
+ * InstanceStream implementation to handle Apache Avro Files. Handles both JSON & Binary encoded streams
+ * 
  *
  */
 public class AvroFileStream extends FileStream {
 
-	private static final long serialVersionUID = 1L;
-	private static final Logger logger = LoggerFactory.getLogger(AvroFileStream.class);
-
-	public FileOption avroFileOption = new FileOption("avroFile", 'f',"Avro File(s) to load.", null, null, false);
-	public IntOption classIndexOption = new IntOption("classIndex", 'c',"Class index of data. 0 for none or -1 for last attribute in file.",-1, -1, Integer.MAX_VALUE);
-	public StringOption encodingFormatOption = new StringOption("encodingFormatOption", 'e', "Encoding format for Avro Files. Can be JSON/AVRO", "BINARY");
-
-	/** Represents the last read Instance **/
-	protected InstanceExample lastInstanceRead;
-
-	/** Represents the binary input stream of avro data **/
-	protected transient InputStream inputStream = null;
-
-	/** The extension to be considered for the files **/
-	private static final String AVRO_FILE_EXTENSION = "avro";
-
-	/* (non-Javadoc)
-	 * @see org.apache.samoa.streams.FileStream#reset()
-	 * Reset the BINARY encoded Avro Stream & Close the file source
-	 */
-	@Override
-	protected void reset() {
-
-		try {
-			if (this.inputStream != null)
-				this.inputStream.close();
-
-			fileSource.reset();
-		} catch (IOException ioException) {
-			logger.error(AVRO_STREAM_FAILED_RESTART_ERROR+" : {}",ioException);
-			throw new RuntimeException(AVRO_STREAM_FAILED_RESTART_ERROR, ioException);
-		}
-
-		if (!getNextFileStream()) {
-			hitEndOfStream = true;
-			throw new RuntimeException(AVRO_STREAM_EMPTY_STREAM_ERROR);
-		}
-	}
-
-
-	/**
-	 * Get next File Stream & set the class index read from the command line option
-	 * @return
-	 */
-	protected boolean getNextFileStream() {
-		if (this.inputStream != null)
-			try {
-				this.inputStream.close();
-			} catch (IOException ioException) {
-				logger.error(AVRO_STREAM_FAILED_READ_ERROR+" : {}",ioException);
-				throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
-			}
-
-		this.inputStream = this.fileSource.getNextInputStream();
-
-		if (this.inputStream == null)
-			return false;
-
-		this.instances = new Instances(this.inputStream, classIndexOption.getValue(),encodingFormatOption.getValue());
-
-		if (this.classIndexOption.getValue() < 0) {
-			this.instances.setClassIndex(this.instances.numAttributes() - 1);
-		} else if (this.classIndexOption.getValue() > 0) {
-			this.instances.setClassIndex(this.classIndexOption.getValue() - 1);
-		}
-		return true;
-	}
-
-
-	/* (non-Javadoc)
-	 * @see org.apache.samoa.streams.FileStream#readNextInstanceFromFile()
-	 * Read next Instance from File. Return false if unable to read next Instance
-	 */
-	@Override
-	protected boolean readNextInstanceFromFile() {
-		try {
-			if (this.instances.readInstance()) {
-				this.lastInstanceRead = new InstanceExample(this.instances.instance(0));
-				this.instances.delete(); 
-				return true;
-			}
-			if (this.inputStream != null) {
-				this.inputStream.close();
-				this.inputStream = null;
-			}
-			return false;
-		} catch (IOException ioException) {
-			logger.error(AVRO_STREAM_FAILED_READ_ERROR+" : {}",ioException);
-			throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
-		}
-
-	}
-
-	@Override
-	public void prepareForUseImpl(TaskMonitor monitor, ObjectRepository repository) {
-		super.prepareForUseImpl(monitor, repository);
-		String filePath = this.avroFileOption.getFile().getAbsolutePath();
-		this.fileSource.init(filePath, AvroFileStream.AVRO_FILE_EXTENSION);
-		this.lastInstanceRead = null;
-	}
-
-
-	/* (non-Javadoc)
-	 * @see org.apache.samoa.streams.FileStream#getLastInstanceRead()
-	 * Return the last read Instance
-	 */
-	@Override
-	protected InstanceExample getLastInstanceRead() {
-		return this.lastInstanceRead;
-	}
-
-
-	@Override
-	public void getDescription(StringBuilder sb, int indent) {
-		throw new UnsupportedOperationException(AVRO_STREAM_UNSUPPORTED_METHOD);
-	}
-
-	/** Error Messages to for all types of Avro File Streams */
-	protected static final String AVRO_STREAM_FAILED_RESTART_ERROR = "Avro FileStream restart failed."; 
-	protected static final String AVRO_STREAM_EMPTY_STREAM_ERROR = "Avro FileStream is empty."; 
-	protected static final String AVRO_STREAM_FAILED_READ_ERROR = "Failed to read next instance from Avro File Stream.";
-	protected static final String AVRO_STREAM_UNSUPPORTED_METHOD = "This method is not supported for AvroFileStream yet.";	
-	
+  private static final long serialVersionUID = 1L;
+  private static final Logger logger = LoggerFactory.getLogger(AvroFileStream.class);
+
+  public FileOption avroFileOption = new FileOption("avroFile", 'f', "Avro File(s) to load.", null, null, false);
+  public IntOption classIndexOption = new IntOption("classIndex", 'c',
+      "Class index of data. 0 for none or -1 for last attribute in file.", -1, -1, Integer.MAX_VALUE);
+  public StringOption encodingFormatOption = new StringOption("encodingFormatOption", 'e',
+      "Encoding format for Avro Files. Can be JSON/AVRO", "BINARY");
+
+  /** Represents the last read Instance **/
+  protected InstanceExample lastInstanceRead;
+
+  /** Represents the binary input stream of avro data **/
+  protected transient InputStream inputStream = null;
+
+  /** The extension to be considered for the files **/
+  private static final String AVRO_FILE_EXTENSION = "avro";
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.streams.FileStream#reset()
+   * Reset the BINARY encoded Avro Stream & Close the file source
+   */
+  @Override
+  protected void reset() {
+
+    try {
+      if (this.inputStream != null)
+        this.inputStream.close();
+
+      fileSource.reset();
+    } catch (IOException ioException) {
+      logger.error(AVRO_STREAM_FAILED_RESTART_ERROR + " : {}", ioException);
+      throw new RuntimeException(AVRO_STREAM_FAILED_RESTART_ERROR, ioException);
+    }
+
+    if (!getNextFileStream()) {
+      hitEndOfStream = true;
+      throw new RuntimeException(AVRO_STREAM_EMPTY_STREAM_ERROR);
+    }
+  }
+
+  /**
+   * Get next File Stream & set the class index read from the command line option
+   * 
+   * @return
+   */
+  protected boolean getNextFileStream() {
+    if (this.inputStream != null)
+      try {
+        this.inputStream.close();
+      } catch (IOException ioException) {
+        logger.error(AVRO_STREAM_FAILED_READ_ERROR + " : {}", ioException);
+        throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
+      }
+
+    this.inputStream = this.fileSource.getNextInputStream();
+
+    if (this.inputStream == null)
+      return false;
+
+    this.instances = new Instances(this.inputStream, classIndexOption.getValue(), encodingFormatOption.getValue());
+
+    if (this.classIndexOption.getValue() < 0) {
+      this.instances.setClassIndex(this.instances.numAttributes() - 1);
+    } else if (this.classIndexOption.getValue() > 0) {
+      this.instances.setClassIndex(this.classIndexOption.getValue() - 1);
+    }
+    return true;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.streams.FileStream#readNextInstanceFromFile()
+   * Read next Instance from File. Return false if unable to read next Instance
+   */
+  @Override
+  protected boolean readNextInstanceFromFile() {
+    try {
+      if (this.instances.readInstance()) {
+        this.lastInstanceRead = new InstanceExample(this.instances.instance(0));
+        this.instances.delete();
+        return true;
+      }
+      if (this.inputStream != null) {
+        this.inputStream.close();
+        this.inputStream = null;
+      }
+      return false;
+    } catch (IOException ioException) {
+      logger.error(AVRO_STREAM_FAILED_READ_ERROR + " : {}", ioException);
+      throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
+    }
+
+  }
+
+  @Override
+  public void prepareForUseImpl(TaskMonitor monitor, ObjectRepository repository) {
+    super.prepareForUseImpl(monitor, repository);
+    String filePath = this.avroFileOption.getFile().getAbsolutePath();
+    this.fileSource.init(filePath, AvroFileStream.AVRO_FILE_EXTENSION);
+    this.lastInstanceRead = null;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.streams.FileStream#getLastInstanceRead()
+   * Return the last read Instance
+   */
+  @Override
+  protected InstanceExample getLastInstanceRead() {
+    return this.lastInstanceRead;
+  }
+
+  @Override
+  public void getDescription(StringBuilder sb, int indent) {
+    throw new UnsupportedOperationException(AVRO_STREAM_UNSUPPORTED_METHOD);
+  }
+
+  /** Error Messages to for all types of Avro File Streams */
+  protected static final String AVRO_STREAM_FAILED_RESTART_ERROR = "Avro FileStream restart failed.";
+  protected static final String AVRO_STREAM_EMPTY_STREAM_ERROR = "Avro FileStream is empty.";
+  protected static final String AVRO_STREAM_FAILED_READ_ERROR = "Failed to read next instance from Avro File Stream.";
+  protected static final String AVRO_STREAM_UNSUPPORTED_METHOD = "This method is not supported for AvroFileStream yet.";
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/76a37363/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java
index 325d1b8..a25dc62 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java
@@ -86,7 +86,7 @@ public class ArffLoader implements Loader {
       while (numAttribute == 0 && streamTokenizer.ttype != StreamTokenizer.TT_EOF) {
         // For each line
         while (streamTokenizer.ttype != StreamTokenizer.TT_EOL
-               && streamTokenizer.ttype != StreamTokenizer.TT_EOF) {
+            && streamTokenizer.ttype != StreamTokenizer.TT_EOF) {
           // For each item
           if (streamTokenizer.ttype == StreamTokenizer.TT_NUMBER) {
             // System.out.println(streamTokenizer.nval + "Num ");
@@ -95,7 +95,7 @@ public class ArffLoader implements Loader {
 
           } else if (streamTokenizer.sval != null && (
               streamTokenizer.ttype == StreamTokenizer.TT_WORD
-              || streamTokenizer.ttype == 34 || streamTokenizer.ttype == 39)) {
+                  || streamTokenizer.ttype == 34 || streamTokenizer.ttype == 39)) {
             // System.out.println(streamTokenizer.sval + "Str");
             boolean isNumeric = attributes.get(numAttribute).isNumeric();
             double value;
@@ -158,7 +158,7 @@ public class ArffLoader implements Loader {
       streamTokenizer.nextToken(); // Remove the '{' char
       // For each line
       while (streamTokenizer.ttype != StreamTokenizer.TT_EOL
-             && streamTokenizer.ttype != StreamTokenizer.TT_EOF) {
+          && streamTokenizer.ttype != StreamTokenizer.TT_EOF) {
         while (streamTokenizer.ttype != '}') {
           // For each item
           // streamTokenizer.nextToken();
@@ -176,7 +176,7 @@ public class ArffLoader implements Loader {
           if (streamTokenizer.ttype == StreamTokenizer.TT_NUMBER) {
             // System.out.print(streamTokenizer.nval + " ");
             this.setSparseValue(instance, indexValues, attributeValues, numAttribute,
-                                streamTokenizer.nval, true);
+                streamTokenizer.nval, true);
             // numAttribute++;
 
           } else if (streamTokenizer.sval != null && (
@@ -185,12 +185,12 @@ public class ArffLoader implements Loader {
             // System.out.print(streamTokenizer.sval + "-");
             if (attributes.get(numAttribute).isNumeric()) {
               this.setSparseValue(instance, indexValues, attributeValues, numAttribute,
-                                  Double.valueOf(streamTokenizer.sval).doubleValue(), true);
+                  Double.valueOf(streamTokenizer.sval).doubleValue(), true);
             } else {
               this.setSparseValue(instance, indexValues, attributeValues, numAttribute,
-                                  this.instanceInformation
-                                      .attribute(numAttribute).indexOfValue(streamTokenizer.sval),
-                                  false);
+                  this.instanceInformation
+                      .attribute(numAttribute).indexOfValue(streamTokenizer.sval),
+                  false);
             }
           }
           streamTokenizer.nextToken();
@@ -211,14 +211,14 @@ public class ArffLoader implements Loader {
       arrayAttributeValues[i] = attributeValues.get(i).doubleValue();
     }
     instance.addSparseValues(arrayIndexValues, arrayAttributeValues,
-                             this.instanceInformation.numAttributes());
+        this.instanceInformation.numAttributes());
     return instance;
 
   }
 
   private void setSparseValue(Instance instance, List<Integer> indexValues,
-                              List<Double> attributeValues,
-                              int numAttribute, double value, boolean isNumber) {
+      List<Double> attributeValues,
+      int numAttribute, double value, boolean isNumber) {
     double valueAttribute;
     if (isNumber && this.instanceInformation.attribute(numAttribute).isNominal) {
       valueAttribute =
@@ -246,7 +246,7 @@ public class ArffLoader implements Loader {
       streamTokenizer.nextToken(); // Remove the '{' char
       // For each line
       while (streamTokenizer.ttype != StreamTokenizer.TT_EOL
-             && streamTokenizer.ttype != StreamTokenizer.TT_EOF) {
+          && streamTokenizer.ttype != StreamTokenizer.TT_EOF) {
         while (streamTokenizer.ttype != '}') {
           // For each item
           // streamTokenizer.nextToken();
@@ -267,11 +267,11 @@ public class ArffLoader implements Loader {
             // "/"+this.instanceInformation.attribute(numAttribute).indexOfValue(streamTokenizer.sval)+" ");
             if (attributes.get(numAttribute).isNumeric()) {
               this.setValue(instance, numAttribute,
-                            Double.valueOf(streamTokenizer.sval).doubleValue(), true);
+                  Double.valueOf(streamTokenizer.sval).doubleValue(), true);
             } else {
               this.setValue(instance, numAttribute,
-                            this.instanceInformation.attribute(numAttribute)
-                                .indexOfValue(streamTokenizer.sval), false);
+                  this.instanceInformation.attribute(numAttribute)
+                      .indexOfValue(streamTokenizer.sval), false);
               // numAttribute++;
             }
           }
@@ -395,6 +395,6 @@ public class ArffLoader implements Loader {
 
   @Override
   public Instance readInstance() {
-	  return readInstance(this.reader);
+    return readInstance(this.reader);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/76a37363/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
index ad07f62..5c57aa1 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
@@ -20,7 +20,6 @@ package org.apache.samoa.instances;
  * #L%
  */
 
-
 import java.io.IOException;
 import java.io.InputStream;
 
@@ -32,88 +31,86 @@ import org.slf4j.LoggerFactory;
 
 /**
  * Load Data from Binary Avro Stream and parse to corresponding Dense & Parse Instances
- * 
- *
  */
 public class AvroBinaryLoader extends AvroLoader {
 
-	private static final long serialVersionUID = 1L;
-	private static final Logger logger = LoggerFactory.getLogger(AvroBinaryLoader.class);
-
-	/** Avro Binary reader for an input stream **/
-	protected DataFileStream<GenericRecord> dataFileStream   = null;
-
-	public AvroBinaryLoader(InputStream inputStream,int classAttribute) {
-		super(classAttribute);
-		initializeSchema(inputStream);
-	}
-
-	/* (non-Javadoc)
-	 * @see org.apache.samoa.instances.AvroLoader#initializeSchema(java.io.InputStream)
-	 */
-	@Override
-	public void initializeSchema(InputStream inputStream)
-	{
-		try {
-			this.datumReader = new GenericDatumReader<GenericRecord>();
-			this.dataFileStream =  new DataFileStream<GenericRecord>(inputStream, datumReader);
-			this.schema = dataFileStream.getSchema();
-
-			this.instanceInformation = getHeader();
-			this.isSparseData = isSparseData();
-
-			if (classAttribute < 0) {
-				this.instanceInformation.setClassIndex(this.instanceInformation.numAttributes() - 1);
-			} else if (classAttribute > 0) {
-				this.instanceInformation.setClassIndex(classAttribute - 1);
-			}
-
-		} catch (IOException ioException) {
-			logger.error(AVRO_LOADER_SCHEMA_READ_ERROR+" : {}",ioException);
-			throw new RuntimeException(AVRO_LOADER_SCHEMA_READ_ERROR+" : " + ioException);
-		}
-	}
-
-	/* (non-Javadoc)
-	 * @see org.apache.samoa.instances.AvroLoader#readInstance()
-	 */
-	@Override
-	public Instance readInstance() {
-
-		GenericRecord record = null;
-
-		try{
-			if (dataFileStream.hasNext()) {
-				record =(GenericRecord) dataFileStream.next();
-			}
-		} catch (Exception ioException) {
-			logger.error(AVRO_LOADER_INSTANCE_READ_ERROR+" : {}",ioException);
-			throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR+" : " + ioException);
-		}
-
-		if(record==null)
-		{
-			closeReader();
-			return null;
-		}
-
-		if(isSparseData)
-			return readInstanceSparse(record);
-
-		return readInstanceDense(record);
-	}
-
-	/**
-	 * Close the Avro Data Stream
-	 */
-	private void closeReader()
-	{
-		if(dataFileStream !=null)
-			try {
-				dataFileStream.close();
-			} catch (IOException ioException) {
-				logger.error(AVRO_LOADER_INSTANCE_READ_ERROR+" : {}",ioException);
-				throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR+" : " + ioException);
-			}
-	}
+  private static final long serialVersionUID = 1L;
+  private static final Logger logger = LoggerFactory.getLogger(AvroBinaryLoader.class);
+
+  /** Avro Binary reader for an input stream **/
+  protected DataFileStream<GenericRecord> dataFileStream = null;
+
+  public AvroBinaryLoader(InputStream inputStream, int classAttribute) {
+    super(classAttribute);
+    initializeSchema(inputStream);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.instances.AvroLoader#initializeSchema(java.io.InputStream)
+   */
+  @Override
+  public void initializeSchema(InputStream inputStream)
+  {
+    try {
+      this.datumReader = new GenericDatumReader<GenericRecord>();
+      this.dataFileStream = new DataFileStream<GenericRecord>(inputStream, datumReader);
+      this.schema = dataFileStream.getSchema();
+
+      this.instanceInformation = getHeader();
+      this.isSparseData = isSparseData();
+
+      if (classAttribute < 0) {
+        this.instanceInformation.setClassIndex(this.instanceInformation.numAttributes() - 1);
+      } else if (classAttribute > 0) {
+        this.instanceInformation.setClassIndex(classAttribute - 1);
+      }
+
+    } catch (IOException ioException) {
+      logger.error(AVRO_LOADER_SCHEMA_READ_ERROR + " : {}", ioException);
+      throw new RuntimeException(AVRO_LOADER_SCHEMA_READ_ERROR + " : " + ioException);
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.instances.AvroLoader#readInstance()
+   */
+  @Override
+  public Instance readInstance() {
+
+    GenericRecord record = null;
+
+    try {
+      if (dataFileStream.hasNext()) {
+        record = (GenericRecord) dataFileStream.next();
+      }
+    } catch (Exception ioException) {
+      logger.error(AVRO_LOADER_INSTANCE_READ_ERROR + " : {}", ioException);
+      throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR + " : " + ioException);
+    }
+
+    if (record == null)
+    {
+      closeReader();
+      return null;
+    }
+
+    if (isSparseData)
+      return readInstanceSparse(record);
+
+    return readInstanceDense(record);
+  }
+
+  /**
+   * Close the Avro Data Stream
+   */
+  private void closeReader()
+  {
+    if (dataFileStream != null)
+      try {
+        dataFileStream.close();
+      } catch (IOException ioException) {
+        logger.error(AVRO_LOADER_INSTANCE_READ_ERROR + " : {}", ioException);
+        throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR + " : " + ioException);
+      }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/76a37363/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
index 8a345da..b765405 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
@@ -20,7 +20,6 @@ package org.apache.samoa.instances;
  * #L%
  */
 
-
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
@@ -37,96 +36,94 @@ import org.slf4j.LoggerFactory;
 
 /**
  * Load Data from JSON Avro Stream and parse to corresponding Dense & Parse Instances
- * 
- *
  */
 public class AvroJsonLoader extends AvroLoader {
 
-	private static final long serialVersionUID = 1L;
-	private static final Logger logger = LoggerFactory.getLogger(AvroJsonLoader.class);
-
-	/** The Character reader for JSON read */
-	protected Reader reader  = null;
-
-	public AvroJsonLoader(InputStream inputStream, int classAttribute) {
-		super(classAttribute);
-		initializeSchema(inputStream);
-	}
-
-	/* (non-Javadoc)
-	 * @see org.apache.samoa.instances.AvroLoader#initializeSchema(java.io.InputStream)
-	 */
-	@Override
-	public void initializeSchema(InputStream inputStream)
-	{
-		String schemaString = null;
-		try {
-			this.reader = new BufferedReader(new InputStreamReader(inputStream));
-			schemaString = ((BufferedReader)this.reader).readLine();
-			this.schema = new Schema.Parser().parse(schemaString);
-			this.datumReader = new GenericDatumReader<GenericRecord>(schema);
-			this.instanceInformation = getHeader();
-			this.isSparseData = isSparseData();
-
-			if (classAttribute < 0) {
-				this.instanceInformation.setClassIndex(this.instanceInformation.numAttributes() - 1);
-			} else if (classAttribute > 0) {
-				this.instanceInformation.setClassIndex(classAttribute - 1);
-			}
-
-		} catch (IOException ioException) {
-			logger.error(AVRO_LOADER_SCHEMA_READ_ERROR+" : {}",ioException);
-			throw new RuntimeException(AVRO_LOADER_SCHEMA_READ_ERROR+" : " + ioException);
-		}
-	}
-
-	/* (non-Javadoc)
-	 * @see org.apache.samoa.instances.AvroLoader#readInstance()
-	 */
-	@Override
-	public Instance readInstance() {
-
-		String line = null;
-		Decoder decoder = null;
-		GenericRecord record = null;
-
-		try{
-			while ((line = ((BufferedReader)reader).readLine()) != null) {
-				if(line==null || line.trim().length()<=0)
-					continue;
-
-				decoder = DecoderFactory.get().jsonDecoder(schema, line);
-				record  = datumReader.read(null, decoder);
-				break;
-			}
-		} catch (IOException ioException) {
-			logger.error(AVRO_LOADER_INSTANCE_READ_ERROR+" : {}",ioException);
-			throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR+" : " + ioException);
-		}
-
-		if(record==null)
-		{
-			closeReader();
-			return null;
-		}
-
-		if(isSparseData)
-			return readInstanceSparse(record);
-
-		return readInstanceDense(record);
-	}
-
-	/**
-	 * Close the Avro Data Stream
-	 */
-	private void closeReader()
-	{
-		if(reader !=null)
-			try {
-				reader.close();
-			} catch (IOException ioException) {
-				logger.error(AVRO_LOADER_INSTANCE_READ_ERROR+" : {}",ioException);
-				throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR+" : " + ioException);
-			}
-	}
+  private static final long serialVersionUID = 1L;
+  private static final Logger logger = LoggerFactory.getLogger(AvroJsonLoader.class);
+
+  /** The Character reader for JSON read */
+  protected Reader reader = null;
+
+  public AvroJsonLoader(InputStream inputStream, int classAttribute) {
+    super(classAttribute);
+    initializeSchema(inputStream);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.instances.AvroLoader#initializeSchema(java.io.InputStream)
+   */
+  @Override
+  public void initializeSchema(InputStream inputStream)
+  {
+    String schemaString = null;
+    try {
+      this.reader = new BufferedReader(new InputStreamReader(inputStream));
+      schemaString = ((BufferedReader) this.reader).readLine();
+      this.schema = new Schema.Parser().parse(schemaString);
+      this.datumReader = new GenericDatumReader<GenericRecord>(schema);
+      this.instanceInformation = getHeader();
+      this.isSparseData = isSparseData();
+
+      if (classAttribute < 0) {
+        this.instanceInformation.setClassIndex(this.instanceInformation.numAttributes() - 1);
+      } else if (classAttribute > 0) {
+        this.instanceInformation.setClassIndex(classAttribute - 1);
+      }
+
+    } catch (IOException ioException) {
+      logger.error(AVRO_LOADER_SCHEMA_READ_ERROR + " : {}", ioException);
+      throw new RuntimeException(AVRO_LOADER_SCHEMA_READ_ERROR + " : " + ioException);
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.samoa.instances.AvroLoader#readInstance()
+   */
+  @Override
+  public Instance readInstance() {
+
+    String line = null;
+    Decoder decoder = null;
+    GenericRecord record = null;
+
+    try {
+      while ((line = ((BufferedReader) reader).readLine()) != null) {
+        if (line == null || line.trim().length() <= 0)
+          continue;
+
+        decoder = DecoderFactory.get().jsonDecoder(schema, line);
+        record = datumReader.read(null, decoder);
+        break;
+      }
+    } catch (IOException ioException) {
+      logger.error(AVRO_LOADER_INSTANCE_READ_ERROR + " : {}", ioException);
+      throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR + " : " + ioException);
+    }
+
+    if (record == null)
+    {
+      closeReader();
+      return null;
+    }
+
+    if (isSparseData)
+      return readInstanceSparse(record);
+
+    return readInstanceDense(record);
+  }
+
+  /**
+   * Close the Avro Data Stream
+   */
+  private void closeReader()
+  {
+    if (reader != null)
+      try {
+        reader.close();
+      } catch (IOException ioException) {
+        logger.error(AVRO_LOADER_INSTANCE_READ_ERROR + " : {}", ioException);
+        throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR + " : " + ioException);
+      }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/76a37363/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
index 33db437..0547a5c 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
@@ -20,7 +20,6 @@ package org.apache.samoa.instances;
  * #L%
  */
 
-
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.List;
@@ -32,254 +31,267 @@ import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.io.DatumReader;
 
 /**
- * Load Data from Avro Stream and parse to corresponding Dense & Parse Instances
- * Abstract Class: Subclass this class for different types of Avro Encodings
- * 
+ * Load Data from Avro Stream and parse to corresponding Dense & Parse Instances Abstract Class: Subclass this class for
+ * different types of Avro Encodings
  *
  */
 public abstract class AvroLoader implements Loader {
 
-	private static final long serialVersionUID = 1L;
-
-	/** Representation of the Avro Schema for the Instances being read. Built from the first line in the data  */
-	protected Schema schema = null;
-
-	/**  Meta-data of the Instance */
-	protected InstanceInformation instanceInformation;
-
-	/** List of attributes in the data as read from the schema */
-	protected List<Attribute> attributes;
-
-	/** This variable is to check if the data stored is Sparse or Dense */
-	protected boolean isSparseData;
-
-	protected int classAttribute;
+  private static final long serialVersionUID = 1L;
 
-	/** Datum Reader for Avro Data*/
-	public DatumReader<GenericRecord> datumReader = null;
-
-	public AvroLoader(int classAttribute) {
-		this.classAttribute = classAttribute;
-		this.isSparseData = false;
-	}
-
-	/** Intialize Avro Schema, Meta Data, InstanceInformation from Input Avro Stream */
-	public abstract void initializeSchema(InputStream inputStream);
-
-	/** Read a single SAMOA Instance from Input Avro Stream */
-	public abstract Instance readInstance();
-	
-	/**
-	 * Method to read Dense Instances from Avro File
-	 * @return Instance
-	 */
-	protected Instance readInstanceDense(GenericRecord record)
-	{
-		Instance instance = new DenseInstance(this.instanceInformation.numAttributes() + 1);
-		int numAttribute = 0;
-
-		for (Attribute attribute : attributes) {
-			Object value = record.get(attribute.name);
-
-			boolean isNumeric = attributes.get(numAttribute).isNumeric();
-			boolean isNominal = attributes.get(numAttribute).isNominal();
-
-			if(isNumeric)
-			{
-				if(value instanceof Double)	
-					this.setDenseValue(instance, numAttribute, (double)value);
-				else if (value instanceof Long)	
-					this.setDenseValue(instance, numAttribute, (long)value);
-				else if (value instanceof Integer)	
-					this.setDenseValue(instance, numAttribute, (int)value);
-				else
-					throw new RuntimeException("Invalid data type in the Avro data for Numeric Type : "+attribute.name);
-			}
-			else if(isNominal)
-			{
-				double valueAttribute;
-
-				if (!(value instanceof EnumSymbol))	
-					throw new RuntimeException("Invalid data type in the Avro data for Nominal Type : "+attribute.name);
-
-				EnumSymbol enumSymbolalue = (EnumSymbol)value;
-
-				String stringValue = enumSymbolalue.toString();
-
-				if (("?".equals(stringValue))||(stringValue==null)) {
-					valueAttribute = Double.NaN; 
-				} else {
-					valueAttribute = this.instanceInformation.attribute(numAttribute).indexOfValue(stringValue);
-				}
-
-				this.setDenseValue(instance, numAttribute, valueAttribute);
-			}
-			numAttribute++;
-		}
-
-		return (numAttribute > 0) ? instance : null;
-
-	}
-
-
-	/**
-	 * Sets a Dense Value in the corresponding attribute index
-	 * @param instance is the Instance where values will be set
-	 * @param numAttribute is the index of the attribute
-	 * @param valueAttribute is the value of the attribute for this Instance
-	 */
-
-	private void setDenseValue(Instance instance, int numAttribute, double valueAttribute) {
-
-		if (this.instanceInformation.classIndex() == numAttribute) 
-			instance.setClassValue(valueAttribute);
-		else 
-			instance.setValue(numAttribute, valueAttribute);
-	}
-
-	/**
-	 * Method to read Sparse Instances from Avro File
-	 * @return Instance
-	 */
-	protected Instance readInstanceSparse(GenericRecord record) {
-
-		Instance instance = new SparseInstance(1.0, null); 
-		int numAttribute = -1;
-		ArrayList<Double> attributeValues = new ArrayList<Double>();
-		List<Integer> indexValues = new ArrayList<Integer>();
-
-		for (Attribute attribute : attributes) {
-			numAttribute++;
-			Object value = record.get(attribute.name);
-
-			boolean isNumeric = attributes.get(numAttribute).isNumeric();
-			boolean isNominal = attributes.get(numAttribute).isNominal();
-
-			/** If value is empty/null iterate to the next attribute.**/
-			if(value==null)
-				continue;
-
-			if(isNumeric)
-			{
-				if(value instanceof Double)	
-					this.setSparseValue(instance,  indexValues, attributeValues, numAttribute, (double)value);
-				else if (value instanceof Long)	
-					this.setSparseValue(instance,indexValues, attributeValues, numAttribute, (long)value);
-				else if (value instanceof Integer)	
-					this.setSparseValue(instance,indexValues, attributeValues, numAttribute, (int)value);
-				else
-					throw new RuntimeException(AVRO_LOADER_INVALID_TYPE_ERROR+" : "+attribute.name);
-			}
-			else if(isNominal)
-			{
-				double valueAttribute;
-
-				if (!(value instanceof EnumSymbol))	
-					throw new RuntimeException(AVRO_LOADER_INVALID_TYPE_ERROR+" : "+attribute.name);
-
-				EnumSymbol enumSymbolalue = (EnumSymbol)value;
-
-				String stringValue = enumSymbolalue.toString();
-
-				if (("?".equals(stringValue))||(stringValue==null)) {
-					valueAttribute = Double.NaN; 
-				} else {
-					valueAttribute = this.instanceInformation.attribute(numAttribute).indexOfValue(stringValue);
-				}
-
-				this.setSparseValue(instance, indexValues, attributeValues, numAttribute, valueAttribute);
-			}
-		}
-
-		int[] arrayIndexValues = new int[attributeValues.size()];
-		double[] arrayAttributeValues = new double[attributeValues.size()];
-		
-		for (int i = 0; i < arrayIndexValues.length; i++) {
-			arrayIndexValues[i] = indexValues.get(i).intValue();
-			arrayAttributeValues[i] = attributeValues.get(i).doubleValue();
-		}
-		
-		instance.addSparseValues(arrayIndexValues, arrayAttributeValues,this.instanceInformation.numAttributes());
-		return instance;
-
-	}
-
-	/**
-	 * Sets a Sparse Value in the corresponding attribute index
-	 * @param instance  is the Instance where values will be set
-	 * @param indexValues is the list of Index values
-	 * @param attributeValues is the list of Attribute values
-	 * @param numAttribute is the index of the attribute
-	 * @param valueAttribute is the value of the attribute for this Instance
-	 */
-	private void setSparseValue(Instance instance, List<Integer> indexValues, List<Double> attributeValues, int numAttribute, double valueAttribute) {
-	
-		if (this.instanceInformation.classIndex() == numAttribute) {
-			instance.setClassValue(valueAttribute);
-		} else {
-			indexValues.add(numAttribute);
-			attributeValues.add(valueAttribute);
-		}
-	}
-
-	/**
-	 * Builds the Meta Data of from the Avro Schema
-	 * @return
-	 */
-	protected InstanceInformation getHeader() {
-
-		String relation = schema.getName();
-		attributes = new ArrayList<Attribute>();
-
-		/** By Definition, the returned list is in the order of their positions. **/
-		List<Schema.Field> fields = schema.getFields();
-
-		for (Field field : fields) {
-			Schema attributeSchema = field.schema();
-			
-			/** Currently SAMOA supports only NOMINAL & Numeric Types.**/
-			if(attributeSchema.getType()==Schema.Type.ENUM)
-			{
-				List<String> attributeLabels = attributeSchema.getEnumSymbols();
-				attributes.add(new Attribute(field.name(), attributeLabels));
-			}
-			else
-				attributes.add(new Attribute(field.name()));
-		}
-		return new InstanceInformation(relation, attributes);
-	}
-
-	/**
-	 * Identifies if the dataset is is Sparse or Dense
-	 * @return boolean
-	 */
-	protected boolean isSparseData()
-	{
-		List<Schema.Field> fields = schema.getFields();
-		for (Field field : fields) {
-			Schema attributeSchema = field.schema(); 
-
-			/** If even one attribute has a null union (nullable attribute) consider it as sparse data**/
-			if(attributeSchema.getType()==Schema.Type.UNION)
-			{
-				List<Schema> unionTypes = attributeSchema.getTypes();
-				for (Schema unionSchema : unionTypes) {
-					if(unionSchema.getType()==Schema.Type.NULL)
-						return true;
-				}
-			}
-
-		}
-		return false;
-	}
-	
-	@Override
-	public InstanceInformation getStructure() {
-		return this.instanceInformation;
-	}
-
-	/** Error Messages to for all types of Avro Loaders */
-	protected static final String AVRO_LOADER_INVALID_TYPE_ERROR = "Invalid data type in the Avro data"; 
-	protected static final String AVRO_LOADER_SCHEMA_READ_ERROR = "Exception while reading the schema from Avro File"; 
-	protected static final String AVRO_LOADER_INSTANCE_READ_ERROR = "Exception while reading the Instance from Avro File.";
+  /** Representation of the Avro Schema for the Instances being read. Built from the first line in the data */
+  protected Schema schema = null;
+
+  /** Meta-data of the Instance */
+  protected InstanceInformation instanceInformation;
+
+  /** List of attributes in the data as read from the schema */
+  protected List<Attribute> attributes;
+
+  /** This variable is to check if the data stored is Sparse or Dense */
+  protected boolean isSparseData;
+
+  protected int classAttribute;
+
+  /** Datum Reader for Avro Data */
+  public DatumReader<GenericRecord> datumReader = null;
+
+  public AvroLoader(int classAttribute) {
+    this.classAttribute = classAttribute;
+    this.isSparseData = false;
+  }
+
+  /** Intialize Avro Schema, Meta Data, InstanceInformation from Input Avro Stream */
+  public abstract void initializeSchema(InputStream inputStream);
+
+  /** Read a single SAMOA Instance from Input Avro Stream */
+  public abstract Instance readInstance();
+
+  /**
+   * Method to read Dense Instances from Avro File
+   * 
+   * @return Instance
+   */
+  protected Instance readInstanceDense(GenericRecord record)
+  {
+    Instance instance = new DenseInstance(this.instanceInformation.numAttributes() + 1);
+    int numAttribute = 0;
+
+    for (Attribute attribute : attributes) {
+      Object value = record.get(attribute.name);
+
+      boolean isNumeric = attributes.get(numAttribute).isNumeric();
+      boolean isNominal = attributes.get(numAttribute).isNominal();
+
+      if (isNumeric)
+      {
+        if (value instanceof Double)
+          this.setDenseValue(instance, numAttribute, (double) value);
+        else if (value instanceof Long)
+          this.setDenseValue(instance, numAttribute, (long) value);
+        else if (value instanceof Integer)
+          this.setDenseValue(instance, numAttribute, (int) value);
+        else
+          throw new RuntimeException("Invalid data type in the Avro data for Numeric Type : " + attribute.name);
+      }
+      else if (isNominal)
+      {
+        double valueAttribute;
+
+        if (!(value instanceof EnumSymbol))
+          throw new RuntimeException("Invalid data type in the Avro data for Nominal Type : " + attribute.name);
+
+        EnumSymbol enumSymbolalue = (EnumSymbol) value;
+
+        String stringValue = enumSymbolalue.toString();
+
+        if (("?".equals(stringValue)) || (stringValue == null)) {
+          valueAttribute = Double.NaN;
+        } else {
+          valueAttribute = this.instanceInformation.attribute(numAttribute).indexOfValue(stringValue);
+        }
+
+        this.setDenseValue(instance, numAttribute, valueAttribute);
+      }
+      numAttribute++;
+    }
+
+    return (numAttribute > 0) ? instance : null;
+
+  }
+
+  /**
+   * Sets a Dense Value in the corresponding attribute index
+   * 
+   * @param instance
+   *          is the Instance where values will be set
+   * @param numAttribute
+   *          is the index of the attribute
+   * @param valueAttribute
+   *          is the value of the attribute for this Instance
+   */
+
+  private void setDenseValue(Instance instance, int numAttribute, double valueAttribute) {
+
+    if (this.instanceInformation.classIndex() == numAttribute)
+      instance.setClassValue(valueAttribute);
+    else
+      instance.setValue(numAttribute, valueAttribute);
+  }
+
+  /**
+   * Method to read Sparse Instances from Avro File
+   * 
+   * @return Instance
+   */
+  protected Instance readInstanceSparse(GenericRecord record) {
+
+    Instance instance = new SparseInstance(1.0, null);
+    int numAttribute = -1;
+    ArrayList<Double> attributeValues = new ArrayList<Double>();
+    List<Integer> indexValues = new ArrayList<Integer>();
+
+    for (Attribute attribute : attributes) {
+      numAttribute++;
+      Object value = record.get(attribute.name);
+
+      boolean isNumeric = attributes.get(numAttribute).isNumeric();
+      boolean isNominal = attributes.get(numAttribute).isNominal();
+
+      /** If value is empty/null iterate to the next attribute. **/
+      if (value == null)
+        continue;
+
+      if (isNumeric)
+      {
+        if (value instanceof Double)
+          this.setSparseValue(instance, indexValues, attributeValues, numAttribute, (double) value);
+        else if (value instanceof Long)
+          this.setSparseValue(instance, indexValues, attributeValues, numAttribute, (long) value);
+        else if (value instanceof Integer)
+          this.setSparseValue(instance, indexValues, attributeValues, numAttribute, (int) value);
+        else
+          throw new RuntimeException(AVRO_LOADER_INVALID_TYPE_ERROR + " : " + attribute.name);
+      }
+      else if (isNominal)
+      {
+        double valueAttribute;
+
+        if (!(value instanceof EnumSymbol))
+          throw new RuntimeException(AVRO_LOADER_INVALID_TYPE_ERROR + " : " + attribute.name);
+
+        EnumSymbol enumSymbolalue = (EnumSymbol) value;
+
+        String stringValue = enumSymbolalue.toString();
+
+        if (("?".equals(stringValue)) || (stringValue == null)) {
+          valueAttribute = Double.NaN;
+        } else {
+          valueAttribute = this.instanceInformation.attribute(numAttribute).indexOfValue(stringValue);
+        }
+
+        this.setSparseValue(instance, indexValues, attributeValues, numAttribute, valueAttribute);
+      }
+    }
+
+    int[] arrayIndexValues = new int[attributeValues.size()];
+    double[] arrayAttributeValues = new double[attributeValues.size()];
+
+    for (int i = 0; i < arrayIndexValues.length; i++) {
+      arrayIndexValues[i] = indexValues.get(i).intValue();
+      arrayAttributeValues[i] = attributeValues.get(i).doubleValue();
+    }
+
+    instance.addSparseValues(arrayIndexValues, arrayAttributeValues, this.instanceInformation.numAttributes());
+    return instance;
+
+  }
+
+  /**
+   * Sets a Sparse Value in the corresponding attribute index
+   * 
+   * @param instance
+   *          is the Instance where values will be set
+   * @param indexValues
+   *          is the list of Index values
+   * @param attributeValues
+   *          is the list of Attribute values
+   * @param numAttribute
+   *          is the index of the attribute
+   * @param valueAttribute
+   *          is the value of the attribute for this Instance
+   */
+  private void setSparseValue(Instance instance, List<Integer> indexValues, List<Double> attributeValues,
+      int numAttribute, double valueAttribute) {
+
+    if (this.instanceInformation.classIndex() == numAttribute) {
+      instance.setClassValue(valueAttribute);
+    } else {
+      indexValues.add(numAttribute);
+      attributeValues.add(valueAttribute);
+    }
+  }
+
+  /**
+   * Builds the Meta Data of from the Avro Schema
+   * 
+   * @return
+   */
+  protected InstanceInformation getHeader() {
+
+    String relation = schema.getName();
+    attributes = new ArrayList<Attribute>();
+
+    /** By Definition, the returned list is in the order of their positions. **/
+    List<Schema.Field> fields = schema.getFields();
+
+    for (Field field : fields) {
+      Schema attributeSchema = field.schema();
+
+      /** Currently SAMOA supports only NOMINAL & Numeric Types. **/
+      if (attributeSchema.getType() == Schema.Type.ENUM)
+      {
+        List<String> attributeLabels = attributeSchema.getEnumSymbols();
+        attributes.add(new Attribute(field.name(), attributeLabels));
+      }
+      else
+        attributes.add(new Attribute(field.name()));
+    }
+    return new InstanceInformation(relation, attributes);
+  }
+
+  /**
+   * Identifies if the dataset is is Sparse or Dense
+   * 
+   * @return boolean
+   */
+  protected boolean isSparseData()
+  {
+    List<Schema.Field> fields = schema.getFields();
+    for (Field field : fields) {
+      Schema attributeSchema = field.schema();
+
+      /** If even one attribute has a null union (nullable attribute) consider it as sparse data **/
+      if (attributeSchema.getType() == Schema.Type.UNION)
+      {
+        List<Schema> unionTypes = attributeSchema.getTypes();
+        for (Schema unionSchema : unionTypes) {
+          if (unionSchema.getType() == Schema.Type.NULL)
+            return true;
+        }
+      }
+
+    }
+    return false;
+  }
+
+  @Override
+  public InstanceInformation getStructure() {
+    return this.instanceInformation;
+  }
+
+  /** Error Messages to for all types of Avro Loaders */
+  protected static final String AVRO_LOADER_INVALID_TYPE_ERROR = "Invalid data type in the Avro data";
+  protected static final String AVRO_LOADER_SCHEMA_READ_ERROR = "Exception while reading the schema from Avro File";
+  protected static final String AVRO_LOADER_INSTANCE_READ_ERROR = "Exception while reading the Instance from Avro File.";
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/76a37363/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java b/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java
index 707d7f2..f7fb0d3 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java
@@ -46,11 +46,12 @@ public class Instances implements Serializable {
    * The instances.
    */
   protected List<Instance> instances;
-  
+
   transient protected Loader loader;
-  
 
-  protected static enum AVRO_ENCODING_FORMAT{JSON,BINARY}
+  protected static enum AVRO_ENCODING_FORMAT {
+    JSON, BINARY
+  }
 
   protected int classAttribute;
 
@@ -73,24 +74,24 @@ public class Instances implements Serializable {
   }
 
   public Instances(Reader reader, int size, int classAttribute) {
-	  this.classAttribute = classAttribute;
-	  loader = new ArffLoader(reader, 0, classAttribute);
-	  this.instanceInformation = loader.getStructure();
-	  this.instances = new ArrayList<>();
+    this.classAttribute = classAttribute;
+    loader = new ArffLoader(reader, 0, classAttribute);
+    this.instanceInformation = loader.getStructure();
+    this.instances = new ArrayList<>();
   }
 
   public Instances(InputStream inputStream, int classAttribute, String encodingFormat) {
-	  this.classAttribute = classAttribute;
+    this.classAttribute = classAttribute;
 
-	  if(encodingFormat.equalsIgnoreCase(AVRO_ENCODING_FORMAT.BINARY.toString()))
-		  loader = new AvroBinaryLoader(inputStream, classAttribute);
-	  else
-		  loader = new AvroJsonLoader(inputStream, classAttribute);
+    if (encodingFormat.equalsIgnoreCase(AVRO_ENCODING_FORMAT.BINARY.toString()))
+      loader = new AvroBinaryLoader(inputStream, classAttribute);
+    else
+      loader = new AvroJsonLoader(inputStream, classAttribute);
 
-	  this.instanceInformation = loader.getStructure();
-	  this.instances = new ArrayList<>();
+    this.instanceInformation = loader.getStructure();
+    this.instances = new ArrayList<>();
   }
-	
+
   public Instances(Instances chunk, int capacity) {
     this(chunk);
   }
@@ -191,22 +192,22 @@ public class Instances implements Serializable {
 
   public boolean readInstance(Reader fileReader) {
 
-	  if (loader == null) {
-		  loader = new ArffLoader(fileReader, 0, this.classAttribute);
-	  }
-	  return readInstance() ;
+    if (loader == null) {
+      loader = new ArffLoader(fileReader, 0, this.classAttribute);
+    }
+    return readInstance();
   }
 
   public boolean readInstance() {
 
-	  Instance inst = loader.readInstance();
-	  if (inst != null) {
-		  inst.setDataset(this);
-		  add(inst);
-		  return true;
-	  } else {
-		  return false;
-	  }
+    Instance inst = loader.readInstance();
+    if (inst != null) {
+      inst.setDataset(this);
+      add(inst);
+      return true;
+    } else {
+      return false;
+    }
   }
 
   public void delete() {

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/76a37363/samoa-instances/src/main/java/org/apache/samoa/instances/Loader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/Loader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/Loader.java
index f806bf5..7e04fbb 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/Loader.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/Loader.java
@@ -20,26 +20,28 @@ package org.apache.samoa.instances;
  * #L%
  */
 
-
 import java.io.Serializable;
 
 /**
  * Loads Instances from streams of different types of Input Formats e.g ARFF & AVRO
+ * 
  * @author jayadeepj
  */
 
-public interface Loader extends Serializable{
-
-	/**
-	 * Fetch the Meta-data from the data 
-	 * @return InstanceInformation
-	 */
-	public InstanceInformation getStructure();
-
-	/**
-	 * Read a single instance from the Stream
-	 * @return Instance
-	 */
-	public Instance readInstance(); 
+public interface Loader extends Serializable {
+
+  /**
+   * Fetch the Meta-data from the data
+   * 
+   * @return InstanceInformation
+   */
+  public InstanceInformation getStructure();
+
+  /**
+   * Read a single instance from the Stream
+   * 
+   * @return Instance
+   */
+  public Instance readInstance();
 
 }


[09/17] incubator-samoa git commit: SAMOA-56: Update Developers-Guide link. Fix #45

Posted by gd...@apache.org.
SAMOA-56: Update Developers-Guide link. Fix #45


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

Branch: refs/heads/master
Commit: 4375bce7b3598a1b8c3ec4a8183b12835b5a4105
Parents: 76a3736
Author: darionyaphet <da...@gmail.com>
Authored: Mon Dec 21 11:12:04 2015 +0800
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Sun Jan 31 15:53:48 2016 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/4375bce7/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index a2f3fd1..4229864 100644
--- a/README.md
+++ b/README.md
@@ -86,7 +86,7 @@ Journal of Machine Learning Research, 16(Jan):149−153, 2015.
 
 ## Apache SAMOA Developer's Guide
 
-<p><a href="http://samoa.incubator.apache.org/SAMOA-Developers-Guide-0-0-1.pdf"><img style="max-width:95%;border:3px solid black;" src="http://samoa.incubator.apache.org/Manual.png" alt="SAMOA Developer's Guide" height="250"> </a></p>
+<p><a href="https://samoa.incubator.apache.org/documentation/SAMOA-Developers-Guide-0.0.1.pdf"><img style="max-width:95%;border:3px solid black;" src="http://samoa.incubator.apache.org/Manual.png" alt="SAMOA Developer's Guide" height="250"> </a></p>
 
 ## Contributors
 [List of contributors to the Apache SAMOA project](http://samoa.incubator.apache.org/documentation/Team.html)


[11/17] incubator-samoa git commit: SAMOA-57: Remove support for S4

Posted by gd...@apache.org.
SAMOA-57: Remove support for S4


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

Branch: refs/heads/master
Commit: de050f994d895dfb599d209ba267fcf5de132f2e
Parents: b86ab83
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Sun Mar 6 13:49:06 2016 +0300
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Sun Mar 6 13:49:06 2016 +0300

----------------------------------------------------------------------
 .travis.yml                                     |  10 -
 pom.xml                                         |  12 -
 samoa-s4/pom.xml                                | 136 ----------
 samoa-s4/samoa-s4-adapter/pom.xml               |  54 ----
 .../samoa/topology/adapter/S4AdapterApp.java    |  45 ----
 .../adapter/S4EntranceProcessingItem.java       |  74 -----
 .../samoa/topology/adapter/package-info.java    |  28 --
 samoa-s4/src/main/assembly/samoa-s4.xml         |  78 ------
 .../samoa/topology/impl/S4ComponentFactory.java |  97 -------
 .../apache/samoa/topology/impl/S4DoTask.java    | 268 -------------------
 .../topology/impl/S4EntranceProcessingItem.java | 119 --------
 .../org/apache/samoa/topology/impl/S4Event.java |  91 -------
 .../samoa/topology/impl/S4ProcessingItem.java   | 186 -------------
 .../apache/samoa/topology/impl/S4Stream.java    | 184 -------------
 .../apache/samoa/topology/impl/S4Submitter.java | 144 ----------
 .../apache/samoa/topology/impl/S4Topology.java  |  63 -----
 .../samoa/topology/impl/SamoaSerializer.java    |  99 -------
 .../topology/impl/SamoaSerializerModule.java    |  35 ---
 18 files changed, 1723 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index a122ebf..6cf1ad3 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -8,16 +8,6 @@ jdk:
 - oraclejdk7
 
 install:
-- git clone https://github.com/apache/incubator-s4.git
-- cd incubator-s4
-- git checkout tags/0.6.0-Final
-- mv ../bin/s4-build/gradlew .
-- wget http://people.apache.org/~gdfm/gradle-wrapper-1.4.jar
-- mv gradle-wrapper-1.4.jar ./lib/
-- mv ../bin/s4-build/gradle-wrapper-1.4.properties ./lib/
-- ./gradlew install
-- ./gradlew s4-tools::installApp
-- cd ..
 - echo "<settings><servers><server><id>apache.snapshots.https</id><username>${SOSS_USERNAME}</username><password>${SOSS_PASSWORD}</password></server></servers></settings>" > ${HOME}/.m2/settings.xml
 #- cat ${HOME}/.m2/settings.xml
 

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index af8fe98..3304937 100644
--- a/pom.xml
+++ b/pom.xml
@@ -76,15 +76,6 @@
             </modules>
         </profile>
         <profile>
-            <id>s4</id>
-            <modules>
-                <module>samoa-instances</module>
-                <module>samoa-api</module>
-                <module>samoa-s4</module>
-                <module>samoa-test</module>
-            </modules>
-        </profile>
-        <profile>
             <id>flink</id>
             <modules>
                 <module>samoa-instances</module>
@@ -111,7 +102,6 @@
                 <module>samoa-threads</module>
                 <module>samoa-storm</module>
                 <module>samoa-flink</module>
-                <module>samoa-s4</module>
                 <module>samoa-samza</module>
                 <module>samoa-test</module>
             </modules>
@@ -135,7 +125,6 @@
         <kryo.version>2.21</kryo.version>
         <metrics-core.version>2.2.0</metrics-core.version>
         <miniball.version>1.0.3</miniball.version>
-        <s4.version>0.6.0-incubating</s4.version>
         <samza.version>0.7.0</samza.version>
         <flink.version>0.10.1</flink.version>
         <slf4j-log4j12.version>1.7.2</slf4j-log4j12.version>
@@ -219,7 +208,6 @@
                         <root>samoa-instances</root>
                         <root>samoa-local</root>
                         <root>samoa-storm</root>
-                        <root>samoa-s4</root>
                         <root>samoa-flink</root>
                         <root>samoa-samza</root>
                         <root>samoa-test</root>

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/pom.xml
----------------------------------------------------------------------
diff --git a/samoa-s4/pom.xml b/samoa-s4/pom.xml
deleted file mode 100644
index f3e24a3..0000000
--- a/samoa-s4/pom.xml
+++ /dev/null
@@ -1,136 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  #%L
-  SAMOA
-  %%
-  Copyright (C) 2014 - 2015 Apache Software Foundation
-  %%
-  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.
-  #L%
-  -->
-
-<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>
-  <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-  </properties>
-
-  <name>samoa-s4</name>
-  <description>S4 bindings for SAMOA</description>
-
-  <artifactId>samoa-s4</artifactId>
-  <parent>
-    <groupId>org.apache.samoa</groupId>
-    <artifactId>samoa</artifactId>
-    <version>0.4.0-incubating-SNAPSHOT</version>
-  </parent>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.samoa</groupId>
-      <artifactId>samoa-api</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.github.javacliparser</groupId>
-      <artifactId>javacliparser</artifactId>
-      <version>${javacliparser.version}</version>
-    </dependency>
-
-    <!-- S4 dependencies need to be installed separately as they are not available via Maven yet -->
-    <dependency>
-      <groupId>org.apache.s4</groupId>
-      <artifactId>s4-base</artifactId>
-      <version>${s4.version}</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.s4</groupId>
-      <artifactId>s4-comm</artifactId>
-      <version>${s4.version}</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.s4</groupId>
-      <artifactId>s4-core</artifactId>
-      <version>${s4.version}</version>
-      <scope>provided</scope>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <version>${maven-dependency-plugin.version}</version>
-        <configuration>
-          <outputDirectory>${project.build.directory}/lib</outputDirectory>
-          <overWriteReleases>false</overWriteReleases>
-          <overWriteSnapshots>false</overWriteSnapshots>
-          <overWriteIfNewer>true</overWriteIfNewer>
-
-          <excludeGroupIds>org.apache.s4</excludeGroupIds>
-          <excludeTransitive>true</excludeTransitive>
-        </configuration>
-        <executions>
-          <execution>
-            <id>copy-dependencies</id>
-            <phase>package</phase>
-            <goals>
-              <goal>copy-dependencies</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-
-      <!-- SAMOA assembly -->
-      <plugin>
-        <artifactId>maven-assembly-plugin</artifactId>
-        <version>${maven-assembly-plugin.version}</version>
-        <configuration>
-          <descriptors>
-            <descriptor>src/main/assembly/samoa-s4.xml</descriptor>
-          </descriptors>
-          <finalName>SAMOA-S4-${project.version}</finalName>
-          <attach>false</attach>
-          <outputDirectory>../target</outputDirectory>
-          <appendAssemblyId>false</appendAssemblyId>
-          <archive>
-            <manifestEntries>
-              <Bundle-Version>${parsedVersion.osgiVersion}</Bundle-Version>
-              <Bundle-Description>${project.description}</Bundle-Description>
-              <Implementation-Version>${project.version}</Implementation-Version>
-              <Implementation-Vendor>Yahoo Labs</Implementation-Vendor>
-              <Implementation-Vendor-Id>SAMOA</Implementation-Vendor-Id>
-              <S4-App-Class>org.apache.samoa.topology.impl.S4DoTask</S4-App-Class>
-              <S4-Version>${s4.version}</S4-Version>
-            </manifestEntries>
-          </archive>
-        </configuration>
-        <executions>
-          <execution>
-            <id>make-assembly</id> <!-- this is used for inheritance merges -->
-            <phase>package</phase> <!-- bind to the packaging phase -->
-            <goals>
-              <goal>single</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/samoa-s4-adapter/pom.xml
----------------------------------------------------------------------
diff --git a/samoa-s4/samoa-s4-adapter/pom.xml b/samoa-s4/samoa-s4-adapter/pom.xml
deleted file mode 100644
index 5a66a1e..0000000
--- a/samoa-s4/samoa-s4-adapter/pom.xml
+++ /dev/null
@@ -1,54 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  #%L
-  SAMOA
-  %%
-  Copyright (C) 2014 - 2015 Apache Software Foundation
-  %%
-  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.
-  #L%
-  -->
-
-<!--
- 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.
--->
-<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>
-
-  <artifactId>samoa-s4-adapter</artifactId>
-  <groupId>org.apache.samoa</groupId>
-  <version>0.1</version>
-  <name>samoa-s4-adapter</name>
-  <description>Adapter module to connect to external stream and also to provide entrance processing items for SAMOA</description>
-
-  <dependencies>
-    <dependency>
-      <artifactId>samoa-s4</artifactId>
-      <groupId>org.apache.samoa</groupId>
-      <version>0.1</version>
-    </dependency>
-  </dependencies>
-
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/S4AdapterApp.java
----------------------------------------------------------------------
diff --git a/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/S4AdapterApp.java b/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/S4AdapterApp.java
deleted file mode 100644
index 1634502..0000000
--- a/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/S4AdapterApp.java
+++ /dev/null
@@ -1,45 +0,0 @@
-package samoa.topology.adapter;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import org.apache.s4.core.adapter.AdapterApp;
-
-import samoa.sandbox.SourceProcessor;
-import samoa.streams.StreamSourceProcessor;
-
-public class S4AdapterApp extends AdapterApp {
-
-	S4EntranceProcessingItem entrancePI;
-	StreamSourceProcessor sourceProcessor;
-	
-	@Override
-	protected void onInit() {
-		entrancePI = new S4EntranceProcessingItem(this);
-		sourceProcessor = new StreamSourceProcessor();
-		entrancePI.setProcessor(sourceProcessor);
-	}
-	
-	@Override
-	protected void onStart() {
-		
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/S4EntranceProcessingItem.java
----------------------------------------------------------------------
diff --git a/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/S4EntranceProcessingItem.java b/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/S4EntranceProcessingItem.java
deleted file mode 100644
index 4c22a0b..0000000
--- a/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/S4EntranceProcessingItem.java
+++ /dev/null
@@ -1,74 +0,0 @@
-package samoa.topology.adapter;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import org.apache.s4.core.App;
-import org.apache.s4.core.ProcessingElement;
-
-import samoa.core.Processor;
-import samoa.topology.EntranceProcessingItem;
-import samoa.topology.impl.DoTaskApp;
-import weka.core.Instance;
-
-public class S4EntranceProcessingItem extends ProcessingElement implements EntranceProcessingItem {
-
-	private Processor processor;
-	//DoTaskApp app;
-	
-	
-	public S4EntranceProcessingItem(App app){
-		super(app);
-		//this.app = (DoTaskApp) app;
-		this.setSingleton(true);
-		
-	}
-
-	@Override
-	public Processor getProcessor() {
-		return this.processor;
-	}
-
-	@Override
-	public void put(Instance inst) {
-		// do nothing
-		//may not needed
-
-	}
-
-	@Override
-	protected void onCreate() {
-		
-		//		if (this.processor != null){
-//			this.processor = this.processor.newProcessor(this.processor);
-//			this.processor.onCreate(Integer.parseInt(getId()));
-//		}
-	}
-
-	@Override
-	protected void onRemove() {
-		//do nothing
-		
-	}
-	
-	public void setProcessor(Processor processor){
-		this.processor = processor;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/package-info.java
----------------------------------------------------------------------
diff --git a/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/package-info.java b/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/package-info.java
deleted file mode 100644
index 2203a32..0000000
--- a/samoa-s4/samoa-s4-adapter/src/main/java/samoa/topology/adapter/package-info.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * 
- */
-/**
- * @author severien
- *
- */
-package samoa.topology.adapter;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/assembly/samoa-s4.xml
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/assembly/samoa-s4.xml b/samoa-s4/src/main/assembly/samoa-s4.xml
deleted file mode 100644
index 41a93f5..0000000
--- a/samoa-s4/src/main/assembly/samoa-s4.xml
+++ /dev/null
@@ -1,78 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  #%L
-  SAMOA
-  %%
-  Copyright (C) 2014 - 2015 Apache Software Foundation
-  %%
-  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.
-  #L%
-  -->
-
-<!--
- 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.
--->
-<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
-  <id>dist</id>
-  <formats>
-    <format>jar</format>
-  </formats>
-  <includeBaseDirectory>false</includeBaseDirectory>
-
-  <fileSets>
-    <!-- SAMOA API artifacts -->
-    <fileSet>
-      <outputDirectory>lib/</outputDirectory>
-      <directory>../samoa-api/target/lib/</directory>
-      <includes>
-        <include>*</include>
-      </includes>
-    </fileSet>
-    <fileSet>
-      <outputDirectory>app/</outputDirectory>
-      <directory>../samoa-api/target/</directory>
-      <includes>
-        <include>samoa-api-*.jar</include>
-      </includes>
-    </fileSet>
-
-    <!-- SAMOA S4 artifacts -->
-    <fileSet>
-      <outputDirectory>app/</outputDirectory>
-      <directory>target/</directory>
-      <includes>
-        <include>samoa-s4-*.jar</include>
-      </includes>
-    </fileSet>
-    <fileSet>
-      <outputDirectory>/</outputDirectory>
-      <directory>target/</directory>
-      <includes>
-        <include>lib/*</include>
-      </includes>
-    </fileSet>
-  </fileSets>
-
-</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4ComponentFactory.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4ComponentFactory.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4ComponentFactory.java
deleted file mode 100644
index ebd18f9..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4ComponentFactory.java
+++ /dev/null
@@ -1,97 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import org.apache.samoa.core.EntranceProcessor;
-import org.apache.samoa.core.Processor;
-import org.apache.samoa.topology.ComponentFactory;
-import org.apache.samoa.topology.EntranceProcessingItem;
-import org.apache.samoa.topology.IProcessingItem;
-import org.apache.samoa.topology.ProcessingItem;
-import org.apache.samoa.topology.Stream;
-import org.apache.samoa.topology.Topology;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * S4 Platform Component Factory
- * 
- * @author severien
- * 
- */
-public class S4ComponentFactory implements ComponentFactory {
-
-  public static final Logger logger = LoggerFactory.getLogger(S4ComponentFactory.class);
-  protected S4DoTask app;
-
-  @Override
-  public ProcessingItem createPi(Processor processor, int paralellism) {
-    S4ProcessingItem processingItem = new S4ProcessingItem(app);
-    // TODO refactor how to set the paralellism level
-    processingItem.setParalellismLevel(paralellism);
-    processingItem.setProcessor(processor);
-
-    return processingItem;
-  }
-
-  @Override
-  public ProcessingItem createPi(Processor processor) {
-    return this.createPi(processor, 1);
-  }
-
-  @Override
-  public EntranceProcessingItem createEntrancePi(EntranceProcessor entranceProcessor) {
-    // TODO Create source Entry processing item that connects to an external
-    // stream
-    S4EntranceProcessingItem entrancePi = new S4EntranceProcessingItem(entranceProcessor, app);
-    entrancePi.setParallelism(1); // FIXME should not be set to 1 statically
-    return entrancePi;
-  }
-
-  @Override
-  public Stream createStream(IProcessingItem sourcePi) {
-    S4Stream aStream = new S4Stream(app);
-    return aStream;
-  }
-
-  @Override
-  public Topology createTopology(String topoName) {
-    return new S4Topology(topoName);
-  }
-
-  /**
-   * Initialization method.
-   * 
-   * @param evalTask
-   */
-  public void init(String evalTask) {
-    // Task is initiated in the DoTaskApp
-  }
-
-  /**
-   * Sets S4 application.
-   * 
-   * @param app
-   */
-  public void setApp(S4DoTask app) {
-    this.app = app;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4DoTask.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4DoTask.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4DoTask.java
deleted file mode 100644
index d52e981..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4DoTask.java
+++ /dev/null
@@ -1,268 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-/**
- * License
- */
-
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.s4.base.Event;
-import org.apache.s4.base.KeyFinder;
-import org.apache.s4.core.App;
-import org.apache.s4.core.ProcessingElement;
-import org.apache.s4.core.Stream;
-import org.apache.samoa.core.Globals;
-import org.apache.samoa.tasks.Task;
-import org.apache.samoa.topology.ComponentFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.github.javacliparser.Option;
-import com.github.javacliparser.ClassOption;
-import com.google.inject.Inject;
-import com.google.inject.name.Named;
-
-/*
- * S4 App that runs samoa Tasks
- *
- * */
-
-/**
- * The Class DoTaskApp.
- */
-final public class S4DoTask extends App {
-
-  private final Logger logger = LoggerFactory.getLogger(S4DoTask.class);
-  Task task;
-
-  @Inject
-  @Named("evalTask")
-  public String evalTask;
-
-  public S4DoTask() {
-    super();
-  }
-
-  /** The engine. */
-  protected ComponentFactory componentFactory;
-
-  /**
-   * Gets the factory.
-   * 
-   * @return the factory
-   */
-  public ComponentFactory getFactory() {
-    return componentFactory;
-  }
-
-  /**
-   * Sets the factory.
-   * 
-   * @param factory
-   *          the new factory
-   */
-  public void setFactory(ComponentFactory factory) {
-    this.componentFactory = factory;
-  }
-
-  /*
-   * Build the application
-   * 
-   * @see org.apache.s4.core.App#onInit()
-   */
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.s4.core.App#onInit()
-   */
-  @Override
-  protected void onInit() {
-    logger.info("DoTaskApp onInit");
-    // ConsoleReporters prints S4 metrics
-    // MetricsRegistry mr = new MetricsRegistry();
-    //
-    // CsvReporter.enable(new File(System.getProperty("user.home")
-    // + "/monitor/"), 10, TimeUnit.SECONDS);
-    // ConsoleReporter.enable(10, TimeUnit.SECONDS);
-    try {
-      System.err.println();
-      System.err.println(Globals.getWorkbenchInfoString());
-      System.err.println();
-
-    } catch (Exception ex) {
-      ex.printStackTrace();
-    }
-    S4ComponentFactory factory = new S4ComponentFactory();
-    factory.setApp(this);
-
-    // logger.debug("LC {}", lc);
-
-    // task = TaskProvider.getTask(evalTask);
-
-    // EXAMPLE OPTIONS
-    // -l Clustream -g Clustream -i 100000 -s (RandomRBFGeneratorEvents -K
-    // 5 -N 0.0)
-    // String[] args = new String[] {evalTask,"-l", "Clustream","-g",
-    // "Clustream", "-i", "100000", "-s", "(RamdomRBFGeneratorsEvents",
-    // "-K", "5", "-N", "0.0)"};
-    // String[] args = new String[] { evalTask, "-l", "clustream.Clustream",
-    // "-g", "clustream.Clustream", "-i", "100000", "-s",
-    // "(RandomRBFGeneratorEvents", "-K", "5", "-N", "0.0)" };
-    logger.debug("PARAMETERS {}", evalTask);
-    // params = params.replace(":", " ");
-    List<String> parameters = new ArrayList<String>();
-    // parameters.add(evalTask);
-    try {
-      parameters.addAll(Arrays.asList(URLDecoder.decode(evalTask, "UTF-8").split(" ")));
-    } catch (UnsupportedEncodingException ex) {
-      ex.printStackTrace();
-    }
-    String[] args = parameters.toArray(new String[0]);
-    Option[] extraOptions = new Option[] {};
-    // build a single string by concatenating cli options
-    StringBuilder cliString = new StringBuilder();
-    for (int i = 0; i < args.length; i++) {
-      cliString.append(" ").append(args[i]);
-    }
-
-    // parse options
-    try {
-      task = (Task) ClassOption.cliStringToObject(cliString.toString(), Task.class, extraOptions);
-      task.setFactory(factory);
-      task.init();
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.s4.core.App#onStart()
-   */
-  @Override
-  protected void onStart() {
-    logger.info("Starting DoTaskApp... App Partition [{}]", this.getPartitionId());
-    // <<<<<<< HEAD Task doesn't have start in latest storm-impl
-    // TODO change the way the app starts
-    // if (this.getPartitionId() == 0)
-    S4Topology s4topology = (S4Topology) getTask().getTopology();
-    S4EntranceProcessingItem epi = (S4EntranceProcessingItem) s4topology.getEntranceProcessingItem();
-    while (epi.injectNextEvent())
-      // inject events from the EntrancePI
-      ;
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.s4.core.App#onClose()
-   */
-  @Override
-  protected void onClose() {
-    System.out.println("Closing DoTaskApp...");
-
-  }
-
-  /**
-   * Gets the task.
-   * 
-   * @return the task
-   */
-  public Task getTask() {
-    return task;
-  }
-
-  // These methods are protected in App and can not be accessed from outside.
-  // They are
-  // called from parallel classifiers and evaluations. Is there a better way
-  // to do that?
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.s4.core.App#createPE(java.lang.Class)
-   */
-  @Override
-  public <T extends ProcessingElement> T createPE(Class<T> type) {
-    return super.createPE(type);
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.s4.core.App#createStream(java.lang.String,
-   * org.apache.s4.base.KeyFinder, org.apache.s4.core.ProcessingElement[])
-   */
-  @Override
-  public <T extends Event> Stream<T> createStream(String name, KeyFinder<T> finder,
-      ProcessingElement... processingElements) {
-    return super.createStream(name, finder, processingElements);
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.s4.core.App#createStream(java.lang.String,
-   * org.apache.s4.core.ProcessingElement[])
-   */
-  @Override
-  public <T extends Event> Stream<T> createStream(String name, ProcessingElement... processingElements) {
-    return super.createStream(name, processingElements);
-  }
-
-  // @com.beust.jcommander.Parameters(separators = "=")
-  // class Parameters {
-  //
-  // @Parameter(names={"-lc","-local"}, description="Local clustering method")
-  // private String localClustering;
-  //
-  // @Parameter(names={"-gc","-global"},
-  // description="Global clustering method")
-  // private String globalClustering;
-  //
-  // }
-  //
-  // class ParametersConverter {// implements IStringConverter<String[]> {
-  //
-  //
-  // public String[] convertToArgs(String value) {
-  //
-  // String[] params = value.split(",");
-  // String[] args = new String[params.length*2];
-  // for(int i=0; i<params.length ; i++) {
-  // args[i] = params[i].split("=")[0];
-  // args[i+1] = params[i].split("=")[1];
-  // i++;
-  // }
-  // return args;
-  // }
-  //
-  // }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4EntranceProcessingItem.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4EntranceProcessingItem.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4EntranceProcessingItem.java
deleted file mode 100644
index 771cbc8..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4EntranceProcessingItem.java
+++ /dev/null
@@ -1,119 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import org.apache.s4.core.App;
-import org.apache.s4.core.ProcessingElement;
-import org.apache.samoa.core.ContentEvent;
-import org.apache.samoa.core.EntranceProcessor;
-import org.apache.samoa.topology.EntranceProcessingItem;
-import org.apache.samoa.topology.Stream;
-
-// TODO adapt this entrance processing item to connect to external streams so the application doesnt need to use an AdapterApp
-
-public class S4EntranceProcessingItem extends ProcessingElement implements EntranceProcessingItem {
-
-  private EntranceProcessor entranceProcessor;
-  // private S4DoTask app;
-  private int parallelism;
-  protected Stream outputStream;
-
-  /**
-   * Constructor of an S4 entrance processing item.
-   * 
-   * @param app
-   *          : S4 application
-   */
-  public S4EntranceProcessingItem(EntranceProcessor entranceProcessor, App app) {
-    super(app);
-    this.entranceProcessor = entranceProcessor;
-    // this.app = (S4DoTask) app;
-    // this.setSingleton(true);
-  }
-
-  public void setParallelism(int parallelism) {
-    this.parallelism = parallelism;
-  }
-
-  public int getParallelism() {
-    return this.parallelism;
-  }
-
-  @Override
-  public EntranceProcessor getProcessor() {
-    return this.entranceProcessor;
-  }
-
-  //
-  // @Override
-  // public void put(Instance inst) {
-  // // do nothing
-  // // may not needed
-  // }
-
-  @Override
-  protected void onCreate() {
-    // was commented
-    if (this.entranceProcessor != null) {
-      // TODO revisit if we need to change it to a clone() call
-      this.entranceProcessor = (EntranceProcessor) this.entranceProcessor.newProcessor(this.entranceProcessor);
-      this.entranceProcessor.onCreate(Integer.parseInt(getId()));
-    }
-  }
-
-  @Override
-  protected void onRemove() {
-    // do nothing
-  }
-
-  //
-  // /**
-  // * Sets the entrance processing item processor.
-  // *
-  // * @param processor
-  // */
-  // public void setProcessor(Processor processor) {
-  // this.entranceProcessor = processor;
-  // }
-
-  @Override
-  public void setName(String name) {
-    super.setName(name);
-  }
-
-  @Override
-  public EntranceProcessingItem setOutputStream(Stream stream) {
-    if (this.outputStream != null)
-      throw new IllegalStateException("Output stream for an EntrancePI sohuld be initialized only once");
-    this.outputStream = stream;
-    return this;
-  }
-
-  public boolean injectNextEvent() {
-    if (entranceProcessor.hasNext()) {
-      ContentEvent nextEvent = this.entranceProcessor.nextEvent();
-      outputStream.put(nextEvent);
-      return entranceProcessor.hasNext();
-    } else
-      return false;
-    // return !nextEvent.isLastEvent();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Event.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Event.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Event.java
deleted file mode 100644
index 154715b..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Event.java
+++ /dev/null
@@ -1,91 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-/**
- * License
- */
-
-import net.jcip.annotations.Immutable;
-
-import org.apache.s4.base.Event;
-import org.apache.samoa.core.ContentEvent;
-
-/**
- * The Class InstanceEvent.
- */
-@Immutable
-final public class S4Event extends Event {
-
-  private String key;
-
-  public String getKey() {
-    return key;
-  }
-
-  public void setKey(String key) {
-    this.key = key;
-  }
-
-  /** The content event. */
-  private ContentEvent contentEvent;
-
-  /**
-   * Instantiates a new instance event.
-   */
-  public S4Event() {
-    // Needed for serialization of kryo
-  }
-
-  /**
-   * Instantiates a new instance event.
-   * 
-   * @param contentEvent
-   *          the content event
-   */
-  public S4Event(ContentEvent contentEvent) {
-    if (contentEvent != null) {
-      this.contentEvent = contentEvent;
-      this.key = contentEvent.getKey();
-
-    }
-  }
-
-  /**
-   * Gets the content event.
-   * 
-   * @return the content event
-   */
-  public ContentEvent getContentEvent() {
-    return contentEvent;
-  }
-
-  /**
-   * Sets the content event.
-   * 
-   * @param contentEvent
-   *          the new content event
-   */
-  public void setContentEvent(ContentEvent contentEvent) {
-    this.contentEvent = contentEvent;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4ProcessingItem.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4ProcessingItem.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4ProcessingItem.java
deleted file mode 100644
index b9c7467..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4ProcessingItem.java
+++ /dev/null
@@ -1,186 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.s4.base.KeyFinder;
-import org.apache.s4.core.App;
-import org.apache.s4.core.ProcessingElement;
-import org.apache.samoa.core.Processor;
-import org.apache.samoa.topology.ProcessingItem;
-import org.apache.samoa.topology.Stream;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * S4 Platform platform specific processing item, inherits from S4 ProcessinElemnt.
- * 
- * @author severien
- * 
- */
-public class S4ProcessingItem extends ProcessingElement implements
-    ProcessingItem {
-
-  public static final Logger logger = LoggerFactory
-      .getLogger(S4ProcessingItem.class);
-
-  private Processor processor;
-  private int paralellismLevel;
-  private S4DoTask app;
-
-  private static final String NAME = "PROCESSING-ITEM-";
-  private static int OBJ_COUNTER = 0;
-
-  /**
-   * Constructor of S4 ProcessingItem.
-   * 
-   * @param app
-   *          : S4 application
-   */
-  public S4ProcessingItem(App app) {
-    super(app);
-    super.setName(NAME + OBJ_COUNTER);
-    OBJ_COUNTER++;
-    this.app = (S4DoTask) app;
-    this.paralellismLevel = 1;
-  }
-
-  @Override
-  public String getName() {
-    return super.getName();
-  }
-
-  /**
-   * Gets processing item paralellism level.
-   * 
-   * @return int
-   */
-  public int getParalellismLevel() {
-    return paralellismLevel;
-  }
-
-  /**
-   * Sets processing item paralellism level.
-   * 
-   * @param paralellismLevel
-   */
-  public void setParalellismLevel(int paralellismLevel) {
-    this.paralellismLevel = paralellismLevel;
-  }
-
-  /**
-   * onEvent method.
-   * 
-   * @param event
-   */
-  public void onEvent(S4Event event) {
-    if (processor.process(event.getContentEvent()) == true) {
-      close();
-    }
-  }
-
-  /**
-   * Sets S4 processing item processor.
-   * 
-   * @param processor
-   */
-  public void setProcessor(Processor processor) {
-    this.processor = processor;
-  }
-
-  // Methods from ProcessingItem
-  @Override
-  public Processor getProcessor() {
-    return processor;
-  }
-
-  /**
-   * KeyFinder sets the keys for a specific event.
-   * 
-   * @return KeyFinder
-   */
-  private KeyFinder<S4Event> getKeyFinder() {
-    KeyFinder<S4Event> keyFinder = new KeyFinder<S4Event>() {
-      @Override
-      public List<String> get(S4Event s4event) {
-        List<String> results = new ArrayList<String>();
-        results.add(s4event.getKey());
-        return results;
-      }
-    };
-
-    return keyFinder;
-  }
-
-  @Override
-  public ProcessingItem connectInputAllStream(Stream inputStream) {
-
-    S4Stream stream = (S4Stream) inputStream;
-    stream.setParallelism(this.paralellismLevel);
-    stream.addStream(inputStream.getStreamId(),
-        getKeyFinder(), this, S4Stream.BROADCAST);
-    return this;
-  }
-
-  @Override
-  public ProcessingItem connectInputKeyStream(Stream inputStream) {
-
-    S4Stream stream = (S4Stream) inputStream;
-    stream.setParallelism(this.paralellismLevel);
-    stream.addStream(inputStream.getStreamId(),
-        getKeyFinder(), this, S4Stream.GROUP_BY_KEY);
-
-    return this;
-  }
-
-  @Override
-  public ProcessingItem connectInputShuffleStream(Stream inputStream) {
-    S4Stream stream = (S4Stream) inputStream;
-    stream.setParallelism(this.paralellismLevel);
-    stream.addStream(inputStream.getStreamId(),
-        getKeyFinder(), this, S4Stream.SHUFFLE);
-
-    return this;
-  }
-
-  // Methods from ProcessingElement
-  @Override
-  protected void onCreate() {
-    logger.debug("PE ID {}", getId());
-    if (this.processor != null) {
-      this.processor = this.processor.newProcessor(this.processor);
-      this.processor.onCreate(Integer.parseInt(getId()));
-    }
-  }
-
-  @Override
-  protected void onRemove() {
-    // do nothing
-  }
-
-  @Override
-  public int getParallelism() {
-    return this.paralellismLevel;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Stream.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Stream.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Stream.java
deleted file mode 100644
index 734462e..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Stream.java
+++ /dev/null
@@ -1,184 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.s4.base.KeyFinder;
-import org.apache.samoa.core.ContentEvent;
-import org.apache.samoa.topology.AbstractStream;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * S4 Platform specific stream.
- * 
- * @author severien
- * 
- */
-public class S4Stream extends AbstractStream {
-
-  public static final int SHUFFLE = 0;
-  public static final int GROUP_BY_KEY = 1;
-  public static final int BROADCAST = 2;
-
-  private static final Logger logger = LoggerFactory.getLogger(S4Stream.class);
-
-  private S4DoTask app;
-  private int processingItemParalellism;
-  private int shuffleCounter;
-
-  private static final String NAME = "STREAM-";
-  private static int OBJ_COUNTER = 0;
-
-  /* The stream list */
-  public List<StreamType> streams;
-
-  public S4Stream(S4DoTask app) {
-    super();
-    this.app = app;
-    this.processingItemParalellism = 1;
-    this.shuffleCounter = 0;
-    this.streams = new ArrayList<StreamType>();
-    this.setStreamId(NAME + OBJ_COUNTER);
-    OBJ_COUNTER++;
-  }
-
-  public S4Stream(S4DoTask app, S4ProcessingItem pi) {
-    super();
-    this.app = app;
-    this.processingItemParalellism = 1;
-    this.shuffleCounter = 0;
-    this.streams = new ArrayList<StreamType>();
-    this.setStreamId(NAME + OBJ_COUNTER);
-    OBJ_COUNTER++;
-
-  }
-
-  /**
-   * 
-   * @return
-   */
-  public int getParallelism() {
-    return processingItemParalellism;
-  }
-
-  public void setParallelism(int parallelism) {
-    this.processingItemParalellism = parallelism;
-  }
-
-  public void addStream(String streamID, KeyFinder<S4Event> finder,
-      S4ProcessingItem pi, int type) {
-    String streamName = streamID + "_" + pi.getName();
-    org.apache.s4.core.Stream<S4Event> stream = this.app.createStream(
-        streamName, pi);
-    stream.setName(streamName);
-    logger.debug("Stream name S4Stream {}", streamName);
-    if (finder != null)
-      stream.setKey(finder);
-    this.streams.add(new StreamType(stream, type));
-
-  }
-
-  @Override
-  public void put(ContentEvent event) {
-
-    for (int i = 0; i < streams.size(); i++) {
-
-      switch (streams.get(i).getType()) {
-      case SHUFFLE:
-        S4Event s4event = new S4Event(event);
-        s4event.setStreamId(streams.get(i).getStream().getName());
-        if (getParallelism() == 1) {
-          s4event.setKey("0");
-        } else {
-          s4event.setKey(Integer.toString(shuffleCounter));
-        }
-        streams.get(i).getStream().put(s4event);
-        shuffleCounter++;
-        if (shuffleCounter >= (getParallelism())) {
-          shuffleCounter = 0;
-        }
-
-        break;
-
-      case GROUP_BY_KEY:
-        S4Event s4event1 = new S4Event(event);
-        s4event1.setStreamId(streams.get(i).getStream().getName());
-        HashCodeBuilder hb = new HashCodeBuilder();
-        hb.append(event.getKey());
-        String key = Integer.toString(hb.build() % getParallelism());
-        s4event1.setKey(key);
-        streams.get(i).getStream().put(s4event1);
-        break;
-
-      case BROADCAST:
-        for (int p = 0; p < this.getParallelism(); p++) {
-          S4Event s4event2 = new S4Event(event);
-          s4event2.setStreamId(streams.get(i).getStream().getName());
-          s4event2.setKey(Integer.toString(p));
-          streams.get(i).getStream().put(s4event2);
-        }
-        break;
-
-      default:
-        break;
-      }
-
-    }
-
-  }
-
-  /**
-   * Subclass for definig stream connection type
-   * 
-   * @author severien
-   * 
-   */
-  class StreamType {
-    org.apache.s4.core.Stream<S4Event> stream;
-    int type;
-
-    public StreamType(org.apache.s4.core.Stream<S4Event> s, int t) {
-      this.stream = s;
-      this.type = t;
-    }
-
-    public org.apache.s4.core.Stream<S4Event> getStream() {
-      return stream;
-    }
-
-    public void setStream(org.apache.s4.core.Stream<S4Event> stream) {
-      this.stream = stream;
-    }
-
-    public int getType() {
-      return type;
-    }
-
-    public void setType(int type) {
-      this.type = type;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Submitter.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Submitter.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Submitter.java
deleted file mode 100644
index 22807a6..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Submitter.java
+++ /dev/null
@@ -1,144 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.io.File;
-import java.net.HttpURLConnection;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.s4.core.util.AppConfig;
-import org.apache.s4.core.util.ParsingUtils;
-import org.apache.s4.deploy.DeploymentUtils;
-import org.apache.samoa.tasks.Task;
-import org.apache.samoa.topology.ISubmitter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.beust.jcommander.JCommander;
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.Parameters;
-
-public class S4Submitter implements ISubmitter {
-
-  private static Logger logger = LoggerFactory.getLogger(S4Submitter.class);
-
-  @Override
-  public void deployTask(Task task) {
-    // TODO: Get application FROM HTTP server
-    // TODO: Initializa a http server to serve the app package
-
-    String appURIString = null;
-    // File app = new File(System.getProperty("user.dir")
-    // + "/src/site/dist/SAMOA-S4-0.1-dist.jar");
-
-    // TODO: String app url http://localhost:8000/SAMOA-S4-0.1-dist.jar
-    try {
-      URL appURL = new URL("http://localhost:8000/SAMOA-S4-0.1.jar");
-      appURIString = appURL.toString();
-    } catch (MalformedURLException e1) {
-      e1.printStackTrace();
-    }
-
-    // try {
-    // appURIString = app.toURI().toURL().toString();
-    // } catch (MalformedURLException e) {
-    // e.printStackTrace();
-    // }
-    if (task == null) {
-      logger.error("Can't execute since evaluation task is not set!");
-      return;
-    } else {
-      logger.info("Deploying SAMOA S4 task [{}] from location [{}]. ",
-          task.getClass().getSimpleName(), appURIString);
-    }
-
-    String[] args = { "-c=testCluster2",
-        "-appClass=" + S4DoTask.class.getName(),
-        "-appName=" + "samoaApp",
-        "-p=evalTask=" + task.getClass().getSimpleName(),
-        "-zk=localhost:2181", "-s4r=" + appURIString, "-emc=" + SamoaSerializerModule.class.getName() };
-    // "-emc=" + S4MOAModule.class.getName(),
-    // "@" +
-    // Resources.getResource("s4moa.properties").getFile(),
-
-    S4Config s4config = new S4Config();
-    JCommander jc = new JCommander(s4config);
-    jc.parse(args);
-
-    Map<String, String> namedParameters = new HashMap<String, String>();
-    for (String parameter : s4config.namedParameters) {
-      String[] param = parameter.split("=");
-      namedParameters.put(param[0], param[1]);
-    }
-
-    AppConfig config = new AppConfig.Builder()
-        .appClassName(s4config.appClass).appName(s4config.appName)
-        .appURI(s4config.appURI).namedParameters(namedParameters)
-        .build();
-
-    DeploymentUtils.initAppConfig(config, s4config.clusterName, true,
-        s4config.zkString);
-
-    System.out.println("Suposedly deployed on S4");
-  }
-
-  public void initHTTPServer() {
-
-  }
-
-  @Parameters(separators = "=")
-  public static class S4Config {
-
-    @Parameter(names = { "-c", "-cluster" }, description = "Cluster name", required = true)
-    String clusterName = null;
-
-    @Parameter(names = "-appClass", description = "Main App class", required = false)
-    String appClass = null;
-
-    @Parameter(names = "-appName", description = "Application name", required = false)
-    String appName = null;
-
-    @Parameter(names = "-s4r", description = "Application URI", required = false)
-    String appURI = null;
-
-    @Parameter(names = "-zk", description = "ZooKeeper connection string", required = false)
-    String zkString = null;
-
-    @Parameter(names = { "-extraModulesClasses", "-emc" }, description = "Comma-separated list of additional configuration modules (they will be instantiated through their constructor without arguments).", required = false)
-    List<String> extraModules = new ArrayList<String>();
-
-    @Parameter(names = { "-p", "-namedStringParameters" }, description = "Comma-separated list of inline configuration "
-        + "parameters, taking precedence over homonymous configuration parameters from configuration files. "
-        + "Syntax: '-p=name1=value1,name2=value2 '", required = false, converter = ParsingUtils.InlineConfigParameterConverter.class)
-    List<String> namedParameters = new ArrayList<String>();
-
-  }
-
-  @Override
-  public void setLocal(boolean bool) {
-    // TODO S4 works the same for local and distributed environments
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Topology.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Topology.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Topology.java
deleted file mode 100644
index 413cfda..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/S4Topology.java
+++ /dev/null
@@ -1,63 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import org.apache.samoa.topology.AbstractTopology;
-import org.apache.samoa.topology.EntranceProcessingItem;
-
-public class S4Topology extends AbstractTopology {
-
-  // CASEY: it seems evaluationTask is not used.
-  // Remove it for now
-
-  // private String _evaluationTask;
-
-  // S4Topology(String topoName, String evalTask) {
-  // super(topoName);
-  // }
-  //
-  // S4Topology(String topoName) {
-  // this(topoName, null);
-  // }
-
-  // @Override
-  // public void setEvaluationTask(String evalTask) {
-  // _evaluationTask = evalTask;
-  // }
-  //
-  // @Override
-  // public String getEvaluationTask() {
-  // return _evaluationTask;
-  // }
-
-  S4Topology(String topoName) {
-    super(topoName);
-  }
-
-  protected EntranceProcessingItem getEntranceProcessingItem() {
-    if (this.getEntranceProcessingItems() == null)
-      return null;
-    if (this.getEntranceProcessingItems().size() < 1)
-      return null;
-    // TODO: support multiple entrance PIs
-    return (EntranceProcessingItem) this.getEntranceProcessingItems().toArray()[0];
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/SamoaSerializer.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/SamoaSerializer.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/SamoaSerializer.java
deleted file mode 100644
index 9f9f144..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/SamoaSerializer.java
+++ /dev/null
@@ -1,99 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import java.nio.ByteBuffer;
-
-import org.apache.s4.base.SerializerDeserializer;
-import org.apache.samoa.learners.classifiers.trees.AttributeContentEvent;
-import org.apache.samoa.learners.classifiers.trees.ComputeContentEvent;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
-import com.google.inject.Inject;
-import com.google.inject.assistedinject.Assisted;
-
-public class SamoaSerializer implements SerializerDeserializer {
-
-  private ThreadLocal<Kryo> kryoThreadLocal;
-  private ThreadLocal<Output> outputThreadLocal;
-
-  private int initialBufferSize = 2048;
-  private int maxBufferSize = 256 * 1024;
-
-  public void setMaxBufferSize(int maxBufferSize) {
-    this.maxBufferSize = maxBufferSize;
-  }
-
-  /**
-   * 
-   * @param classLoader
-   *          classloader able to handle classes to serialize/deserialize. For instance, application-level events can
-   *          only be handled by the application classloader.
-   */
-  @Inject
-  public SamoaSerializer(@Assisted final ClassLoader classLoader) {
-    kryoThreadLocal = new ThreadLocal<Kryo>() {
-
-      @Override
-      protected Kryo initialValue() {
-        Kryo kryo = new Kryo();
-        kryo.setClassLoader(classLoader);
-        kryo.register(AttributeContentEvent.class, new AttributeContentEvent.AttributeCEFullPrecSerializer());
-        kryo.register(ComputeContentEvent.class, new ComputeContentEvent.ComputeCEFullPrecSerializer());
-        kryo.setRegistrationRequired(false);
-        return kryo;
-      }
-    };
-
-    outputThreadLocal = new ThreadLocal<Output>() {
-      @Override
-      protected Output initialValue() {
-        Output output = new Output(initialBufferSize, maxBufferSize);
-        return output;
-      }
-    };
-
-  }
-
-  @Override
-  public Object deserialize(ByteBuffer rawMessage) {
-    Input input = new Input(rawMessage.array());
-    try {
-      return kryoThreadLocal.get().readClassAndObject(input);
-    } finally {
-      input.close();
-    }
-  }
-
-  @SuppressWarnings("resource")
-  @Override
-  public ByteBuffer serialize(Object message) {
-    Output output = outputThreadLocal.get();
-    try {
-      kryoThreadLocal.get().writeClassAndObject(output, message);
-      return ByteBuffer.wrap(output.toBytes());
-    } finally {
-      output.clear();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/de050f99/samoa-s4/src/main/java/org/apache/samoa/topology/impl/SamoaSerializerModule.java
----------------------------------------------------------------------
diff --git a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/SamoaSerializerModule.java b/samoa-s4/src/main/java/org/apache/samoa/topology/impl/SamoaSerializerModule.java
deleted file mode 100644
index e530a09..0000000
--- a/samoa-s4/src/main/java/org/apache/samoa/topology/impl/SamoaSerializerModule.java
+++ /dev/null
@@ -1,35 +0,0 @@
-package org.apache.samoa.topology.impl;
-
-/*
- * #%L
- * SAMOA
- * %%
- * Copyright (C) 2014 - 2015 Apache Software Foundation
- * %%
- * 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.
- * #L%
- */
-
-import org.apache.s4.base.SerializerDeserializer;
-
-import com.google.inject.AbstractModule;
-
-public class SamoaSerializerModule extends AbstractModule {
-
-  @Override
-  protected void configure() {
-    bind(SerializerDeserializer.class).to(SamoaSerializer.class);
-
-  }
-
-}


[04/17] incubator-samoa git commit: SAMOA-48: Fix flawed bagging test

Posted by gd...@apache.org.
SAMOA-48: Fix flawed bagging test


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

Branch: refs/heads/master
Commit: dc99c7fd99c076407b9922b9d81741c6146e1e0b
Parents: b02882e
Author: Gianmarco De Francisci Morales <gd...@apache.org>
Authored: Mon Nov 9 11:51:45 2015 +0200
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Mon Nov 9 14:37:44 2015 +0200

----------------------------------------------------------------------
 samoa-local/src/test/java/org/apache/samoa/AlgosTest.java | 10 ++--------
 1 file changed, 2 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/dc99c7fd/samoa-local/src/test/java/org/apache/samoa/AlgosTest.java
----------------------------------------------------------------------
diff --git a/samoa-local/src/test/java/org/apache/samoa/AlgosTest.java b/samoa-local/src/test/java/org/apache/samoa/AlgosTest.java
index f35b92a..1f7be7e 100644
--- a/samoa-local/src/test/java/org/apache/samoa/AlgosTest.java
+++ b/samoa-local/src/test/java/org/apache/samoa/AlgosTest.java
@@ -27,7 +27,6 @@ public class AlgosTest {
 
   @Test
   public void testVHTLocal() throws Exception {
-
     TestParams vhtConfig = new TestParams.Builder()
         .inputInstances(200_000)
         .samplingSize(20_000)
@@ -42,7 +41,6 @@ public class AlgosTest {
         .taskClassName(LocalDoTask.class.getName())
         .build();
     TestUtils.test(vhtConfig);
-
   }
 
   @Test
@@ -50,8 +48,8 @@ public class AlgosTest {
     TestParams baggingConfig = new TestParams.Builder()
         .inputInstances(200_000)
         .samplingSize(20_000)
-        .evaluationInstances(180_000)
-        .classifiedInstances(210_000)
+        .evaluationInstances(200_000)
+        .classifiedInstances(200_000)
         .classificationsCorrect(60f)
         .kappaStat(0f)
         .kappaTempStat(0f)
@@ -61,12 +59,10 @@ public class AlgosTest {
         .taskClassName(LocalDoTask.class.getName())
         .build();
     TestUtils.test(baggingConfig);
-
   }
 
   @Test
   public void testNaiveBayesLocal() throws Exception {
-
     TestParams vhtConfig = new TestParams.Builder()
         .inputInstances(200_000)
         .samplingSize(20_000)
@@ -81,7 +77,5 @@ public class AlgosTest {
         .taskClassName(LocalDoTask.class.getName())
         .build();
     TestUtils.test(vhtConfig);
-
   }
-
 }


[07/17] incubator-samoa git commit: SAMOA-47: Integrate Avro Streams with SAMOA

Posted by gd...@apache.org.
SAMOA-47: Integrate Avro Streams with SAMOA


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

Branch: refs/heads/master
Commit: 74979782fabfa04b7701ae4c83b91bc38402c351
Parents: b84e8ac
Author: jayadeepj <ja...@gmail.com>
Authored: Fri Oct 30 14:57:06 2015 +0530
Committer: Gianmarco De Francisci Morales <gd...@apache.org>
Committed: Mon Nov 30 13:20:35 2015 +0200

----------------------------------------------------------------------
 pom.xml                                         |   1 +
 .../samoa/moa/streams/AvroFileStream.java       | 171 +++++++++++
 samoa-instances/pom.xml                         |   7 +
 .../org/apache/samoa/instances/ArffLoader.java  |   8 +-
 .../samoa/instances/AvroBinaryLoader.java       | 120 ++++++++
 .../apache/samoa/instances/AvroJsonLoader.java  | 133 +++++++++
 .../org/apache/samoa/instances/AvroLoader.java  | 286 +++++++++++++++++++
 .../org/apache/samoa/instances/Instances.java   |  54 ++--
 .../java/org/apache/samoa/instances/Loader.java |  45 +++
 9 files changed, 806 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/74979782/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 41fc5bd..71b131f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -144,6 +144,7 @@
         <storm.version>0.9.4</storm.version>
         <!-- storm 0.8.2 loads zookeeper classes with hardcoded names from 3.3 version-->
         <zookeeper.storm.version>3.4.6</zookeeper.storm.version>
+        <avro.version>1.7.7</avro.version>
     </properties>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/74979782/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
----------------------------------------------------------------------
diff --git a/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java b/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
new file mode 100644
index 0000000..e684687
--- /dev/null
+++ b/samoa-api/src/main/java/org/apache/samoa/moa/streams/AvroFileStream.java
@@ -0,0 +1,171 @@
+package org.apache.samoa.moa.streams;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.samoa.instances.Instances;
+import org.apache.samoa.moa.core.InstanceExample;
+import org.apache.samoa.moa.core.ObjectRepository;
+import org.apache.samoa.moa.tasks.TaskMonitor;
+import org.apache.samoa.streams.FileStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.javacliparser.FileOption;
+import com.github.javacliparser.IntOption;
+import com.github.javacliparser.StringOption;
+
+
+/**
+ *  InstanceStream implementation to handle Apache Avro Files.
+ *  Handles both JSON & Binary encoded streams
+ *  
+ *  @author jayadeepj
+ *
+ */
+public class AvroFileStream extends FileStream {
+
+	private static final long serialVersionUID = 1L;
+	private static final Logger logger = LoggerFactory.getLogger(AvroFileStream.class);
+
+	public FileOption avroFileOption = new FileOption("avroFile", 'f',"Avro File(s) to load.", null, null, false);
+	public IntOption classIndexOption = new IntOption("classIndex", 'c',"Class index of data. 0 for none or -1 for last attribute in file.",-1, -1, Integer.MAX_VALUE);
+	public StringOption encodingFormatOption = new StringOption("encodingFormatOption", 'e', "Encoding format for Avro Files. Can be JSON/AVRO", "BINARY");
+
+	/** Represents the last read Instance **/
+	protected InstanceExample lastInstanceRead;
+
+	/** Represents the binary input stream of avro data **/
+	protected transient InputStream inputStream = null;
+
+	/** The extension to be considered for the files **/
+	private static final String AVRO_FILE_EXTENSION = "avro";
+
+	/* (non-Javadoc)
+	 * @see org.apache.samoa.streams.FileStream#reset()
+	 * Reset the BINARY encoded Avro Stream & Close the file source
+	 */
+	@Override
+	protected void reset() {
+
+		try {
+			if (this.inputStream != null)
+				this.inputStream.close();
+
+			fileSource.reset();
+		} catch (IOException ioException) {
+			logger.error(AVRO_STREAM_FAILED_RESTART_ERROR+" : {}",ioException);
+			throw new RuntimeException(AVRO_STREAM_FAILED_RESTART_ERROR, ioException);
+		}
+
+		if (!getNextFileStream()) {
+			hitEndOfStream = true;
+			throw new RuntimeException(AVRO_STREAM_EMPTY_STREAM_ERROR);
+		}
+	}
+
+
+	/**
+	 * Get next File Stream & set the class index read from the command line option
+	 * @return
+	 */
+	protected boolean getNextFileStream() {
+		if (this.inputStream != null)
+			try {
+				this.inputStream.close();
+			} catch (IOException ioException) {
+				logger.error(AVRO_STREAM_FAILED_READ_ERROR+" : {}",ioException);
+				throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
+			}
+
+		this.inputStream = this.fileSource.getNextInputStream();
+
+		if (this.inputStream == null)
+			return false;
+
+		this.instances = new Instances(this.inputStream, classIndexOption.getValue(),encodingFormatOption.getValue());
+
+		if (this.classIndexOption.getValue() < 0) {
+			this.instances.setClassIndex(this.instances.numAttributes() - 1);
+		} else if (this.classIndexOption.getValue() > 0) {
+			this.instances.setClassIndex(this.classIndexOption.getValue() - 1);
+		}
+		return true;
+	}
+
+
+	/* (non-Javadoc)
+	 * @see org.apache.samoa.streams.FileStream#readNextInstanceFromFile()
+	 * Read next Instance from File. Return false if unable to read next Instance
+	 */
+	@Override
+	protected boolean readNextInstanceFromFile() {
+		try {
+			if (this.instances.readInstance()) {
+				this.lastInstanceRead = new InstanceExample(this.instances.instance(0));
+				this.instances.delete(); 
+				return true;
+			}
+			if (this.inputStream != null) {
+				this.inputStream.close();
+				this.inputStream = null;
+			}
+			return false;
+		} catch (IOException ioException) {
+			logger.error(AVRO_STREAM_FAILED_READ_ERROR+" : {}",ioException);
+			throw new RuntimeException(AVRO_STREAM_FAILED_READ_ERROR, ioException);
+		}
+
+	}
+
+	@Override
+	public void prepareForUseImpl(TaskMonitor monitor, ObjectRepository repository) {
+		super.prepareForUseImpl(monitor, repository);
+		String filePath = this.avroFileOption.getFile().getAbsolutePath();
+		this.fileSource.init(filePath, AvroFileStream.AVRO_FILE_EXTENSION);
+		this.lastInstanceRead = null;
+	}
+
+
+	/* (non-Javadoc)
+	 * @see org.apache.samoa.streams.FileStream#getLastInstanceRead()
+	 * Return the last read Instance
+	 */
+	@Override
+	protected InstanceExample getLastInstanceRead() {
+		return this.lastInstanceRead;
+	}
+
+
+	@Override
+	public void getDescription(StringBuilder sb, int indent) {
+		throw new UnsupportedOperationException(AVRO_STREAM_UNSUPPORTED_METHOD);
+	}
+
+	/** Error Messages to for all types of Avro File Streams */
+	protected static final String AVRO_STREAM_FAILED_RESTART_ERROR = "Avro FileStream restart failed."; 
+	protected static final String AVRO_STREAM_EMPTY_STREAM_ERROR = "Avro FileStream is empty."; 
+	protected static final String AVRO_STREAM_FAILED_READ_ERROR = "Failed to read next instance from Avro File Stream.";
+	protected static final String AVRO_STREAM_UNSUPPORTED_METHOD = "This method is not supported for AvroFileStream yet.";	
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/74979782/samoa-instances/pom.xml
----------------------------------------------------------------------
diff --git a/samoa-instances/pom.xml b/samoa-instances/pom.xml
index ed24597..64ffcd8 100644
--- a/samoa-instances/pom.xml
+++ b/samoa-instances/pom.xml
@@ -34,4 +34,11 @@
     <artifactId>samoa</artifactId>
     <version>0.4.0-incubating-SNAPSHOT</version>
   </parent>
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.avro</groupId>
+			<artifactId>avro</artifactId>
+			<version>${avro.version}</version>
+		</dependency>
+	</dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/74979782/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java
index 3d314f0..325d1b8 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/ArffLoader.java
@@ -23,7 +23,6 @@ package org.apache.samoa.instances;
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.Reader;
-import java.io.Serializable;
 import java.io.StreamTokenizer;
 import java.util.ArrayList;
 import java.util.List;
@@ -33,7 +32,7 @@ import java.util.logging.Logger;
 /**
  * @author abifet
  */
-public class ArffLoader implements Serializable {
+public class ArffLoader implements Loader {
 
   protected InstanceInformation instanceInformation;
 
@@ -393,4 +392,9 @@ public class ArffLoader implements Serializable {
       this.instanceInformation.setClassIndex(classAttribute - 1);
     }
   }
+
+  @Override
+  public Instance readInstance() {
+	  return readInstance(this.reader);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/74979782/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
new file mode 100644
index 0000000..c3e32dc
--- /dev/null
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroBinaryLoader.java
@@ -0,0 +1,120 @@
+package org.apache.samoa.instances;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Load Data from Binary Avro Stream and parse to corresponding Dense & Parse Instances
+ * 
+ * @author jayadeepj
+ *
+ */
+public class AvroBinaryLoader extends AvroLoader {
+
+	private static final long serialVersionUID = 1L;
+	private static final Logger logger = LoggerFactory.getLogger(AvroBinaryLoader.class);
+
+	/** Avro Binary reader for an input stream **/
+	protected DataFileStream<GenericRecord> dataFileStream   = null;
+
+	public AvroBinaryLoader(InputStream inputStream,int classAttribute) {
+		super(classAttribute);
+		initializeSchema(inputStream);
+	}
+
+	/* (non-Javadoc)
+	 * @see org.apache.samoa.instances.AvroLoader#initializeSchema(java.io.InputStream)
+	 */
+	@Override
+	public void initializeSchema(InputStream inputStream)
+	{
+		try {
+			this.datumReader = new GenericDatumReader<GenericRecord>();
+			this.dataFileStream =  new DataFileStream<GenericRecord>(inputStream, datumReader);
+			this.schema = dataFileStream.getSchema();
+
+			this.instanceInformation = getHeader();
+			this.isSparseData = isSparseData();
+
+			if (classAttribute < 0) {
+				this.instanceInformation.setClassIndex(this.instanceInformation.numAttributes() - 1);
+			} else if (classAttribute > 0) {
+				this.instanceInformation.setClassIndex(classAttribute - 1);
+			}
+
+		} catch (IOException ioException) {
+			logger.error(AVRO_LOADER_SCHEMA_READ_ERROR+" : {}",ioException);
+			throw new RuntimeException(AVRO_LOADER_SCHEMA_READ_ERROR+" : " + ioException);
+		}
+	}
+
+	/* (non-Javadoc)
+	 * @see org.apache.samoa.instances.AvroLoader#readInstance()
+	 */
+	@Override
+	public Instance readInstance() {
+
+		GenericRecord record = null;
+
+		try{
+			if (dataFileStream.hasNext()) {
+				record =(GenericRecord) dataFileStream.next();
+			}
+		} catch (Exception ioException) {
+			logger.error(AVRO_LOADER_INSTANCE_READ_ERROR+" : {}",ioException);
+			throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR+" : " + ioException);
+		}
+
+		if(record==null)
+		{
+			//closeReader();
+			return null;
+		}
+
+		if(isSparseData)
+			return readInstanceSparse(record);
+
+		return readInstanceDense(record);
+	}
+
+	/**
+	 * Close the Avro Data Stream
+	 */
+	private void closeReader()
+	{
+		if(dataFileStream !=null)
+			try {
+				dataFileStream.close();
+			} catch (IOException ioException) {
+				logger.error(AVRO_LOADER_INSTANCE_READ_ERROR+" : {}",ioException);
+				throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR+" : " + ioException);
+			}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/74979782/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
new file mode 100644
index 0000000..827b507
--- /dev/null
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroJsonLoader.java
@@ -0,0 +1,133 @@
+package org.apache.samoa.instances;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Load Data from JSON Avro Stream and parse to corresponding Dense & Parse Instances
+ * 
+ * @author jayadeepj
+ *
+ */
+public class AvroJsonLoader extends AvroLoader {
+
+	private static final long serialVersionUID = 1L;
+	private static final Logger logger = LoggerFactory.getLogger(AvroJsonLoader.class);
+
+	/** The Character reader for JSON read */
+	protected Reader reader  = null;
+
+	public AvroJsonLoader(InputStream inputStream, int classAttribute) {
+		super(classAttribute);
+		initializeSchema(inputStream);
+	}
+
+	/* (non-Javadoc)
+	 * @see org.apache.samoa.instances.AvroLoader#initializeSchema(java.io.InputStream)
+	 */
+	@Override
+	public void initializeSchema(InputStream inputStream)
+	{
+		String schemaString = null;
+		try {
+			this.reader = new BufferedReader(new InputStreamReader(inputStream));
+			schemaString = ((BufferedReader)this.reader).readLine();
+			this.schema = new Schema.Parser().parse(schemaString);
+			this.datumReader = new GenericDatumReader<GenericRecord>(schema);
+			this.instanceInformation = getHeader();
+			this.isSparseData = isSparseData();
+
+			if (classAttribute < 0) {
+				this.instanceInformation.setClassIndex(this.instanceInformation.numAttributes() - 1);
+			} else if (classAttribute > 0) {
+				this.instanceInformation.setClassIndex(classAttribute - 1);
+			}
+
+		} catch (IOException ioException) {
+			logger.error(AVRO_LOADER_SCHEMA_READ_ERROR+" : {}",ioException);
+			throw new RuntimeException(AVRO_LOADER_SCHEMA_READ_ERROR+" : " + ioException);
+		}
+	}
+
+	/* (non-Javadoc)
+	 * @see org.apache.samoa.instances.AvroLoader#readInstance()
+	 */
+	@Override
+	public Instance readInstance() {
+
+		String line = null;
+		Decoder decoder = null;
+		GenericRecord record = null;
+
+		try{
+			while ((line = ((BufferedReader)reader).readLine()) != null) {
+				if(line==null || line.trim().length()<=0)
+					continue;
+
+				decoder = DecoderFactory.get().jsonDecoder(schema, line);
+				record  = datumReader.read(null, decoder);
+				break;
+			}
+		} catch (IOException ioException) {
+			logger.error(AVRO_LOADER_INSTANCE_READ_ERROR+" : {}",ioException);
+			throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR+" : " + ioException);
+		}
+
+		if(record==null)
+		{
+			closeReader();
+			return null;
+		}
+
+		if(isSparseData)
+			return readInstanceSparse(record);
+
+		return readInstanceDense(record);
+	}
+
+	/**
+	 * Close the Avro Data Stream
+	 */
+	private void closeReader()
+	{
+		if(reader !=null)
+			try {
+				reader.close();
+			} catch (IOException ioException) {
+				logger.error(AVRO_LOADER_INSTANCE_READ_ERROR+" : {}",ioException);
+				throw new RuntimeException(AVRO_LOADER_INSTANCE_READ_ERROR+" : " + ioException);
+			}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/74979782/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
new file mode 100644
index 0000000..09f410f
--- /dev/null
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/AvroLoader.java
@@ -0,0 +1,286 @@
+package org.apache.samoa.instances;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.generic.GenericData.EnumSymbol;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+
+/**
+ * Load Data from Avro Stream and parse to corresponding Dense & Parse Instances
+ * Abstract Class: Subclass this class for different types of Avro Encodings
+ * 
+ * @author jayadeepj
+ *
+ */
+public abstract class AvroLoader implements Loader {
+
+	private static final long serialVersionUID = 1L;
+
+	/** Representation of the Avro Schema for the Instances being read. Built from the first line in the data  */
+	protected Schema schema = null;
+
+	/**  Meta-data of the Instance */
+	protected InstanceInformation instanceInformation;
+
+	/** List of attributes in the data as read from the schema */
+	protected List<Attribute> attributes;
+
+	/** This variable is to check if the data stored is Sparse or Dense */
+	protected boolean isSparseData;
+
+	protected int classAttribute;
+
+	/** Datum Reader for Avro Data*/
+	public DatumReader<GenericRecord> datumReader = null;
+
+	public AvroLoader(int classAttribute) {
+		this.classAttribute = classAttribute;
+		this.isSparseData = false;
+	}
+
+	/** Intialize Avro Schema, Meta Data, InstanceInformation from Input Avro Stream */
+	public abstract void initializeSchema(InputStream inputStream);
+
+	/** Read a single SAMOA Instance from Input Avro Stream */
+	public abstract Instance readInstance();
+	
+	/**
+	 * Method to read Dense Instances from Avro File
+	 * @return Instance
+	 */
+	protected Instance readInstanceDense(GenericRecord record)
+	{
+		Instance instance = new DenseInstance(this.instanceInformation.numAttributes() + 1);
+		int numAttribute = 0;
+
+		for (Attribute attribute : attributes) {
+			Object value = record.get(attribute.name);
+
+			boolean isNumeric = attributes.get(numAttribute).isNumeric();
+			boolean isNominal = attributes.get(numAttribute).isNominal();
+
+			if(isNumeric)
+			{
+				if(value instanceof Double)	
+					this.setDenseValue(instance, numAttribute, (double)value);
+				else if (value instanceof Long)	
+					this.setDenseValue(instance, numAttribute, (long)value);
+				else if (value instanceof Integer)	
+					this.setDenseValue(instance, numAttribute, (int)value);
+				else
+					throw new RuntimeException("Invalid data type in the Avro data for Numeric Type : "+attribute.name);
+			}
+			else if(isNominal)
+			{
+				double valueAttribute;
+
+				if (!(value instanceof EnumSymbol))	
+					throw new RuntimeException("Invalid data type in the Avro data for Nominal Type : "+attribute.name);
+
+				EnumSymbol enumSymbolalue = (EnumSymbol)value;
+
+				String stringValue = enumSymbolalue.toString();
+
+				if (("?".equals(stringValue))||(stringValue==null)) {
+					valueAttribute = Double.NaN; 
+				} else {
+					valueAttribute = this.instanceInformation.attribute(numAttribute).indexOfValue(stringValue);
+				}
+
+				this.setDenseValue(instance, numAttribute, valueAttribute);
+			}
+			numAttribute++;
+		}
+
+		return (numAttribute > 0) ? instance : null;
+
+	}
+
+
+	/**
+	 * Sets a Dense Value in the corresponding attribute index
+	 * @param instance is the Instance where values will be set
+	 * @param numAttribute is the index of the attribute
+	 * @param valueAttribute is the value of the attribute for this Instance
+	 */
+
+	private void setDenseValue(Instance instance, int numAttribute, double valueAttribute) {
+
+		if (this.instanceInformation.classIndex() == numAttribute) 
+			instance.setClassValue(valueAttribute);
+		else 
+			instance.setValue(numAttribute, valueAttribute);
+	}
+
+	/**
+	 * Method to read Sparse Instances from Avro File
+	 * @return Instance
+	 */
+	protected Instance readInstanceSparse(GenericRecord record) {
+
+		Instance instance = new SparseInstance(1.0, null); 
+		int numAttribute = -1;
+		ArrayList<Double> attributeValues = new ArrayList<Double>();
+		List<Integer> indexValues = new ArrayList<Integer>();
+
+		for (Attribute attribute : attributes) {
+			numAttribute++;
+			Object value = record.get(attribute.name);
+
+			boolean isNumeric = attributes.get(numAttribute).isNumeric();
+			boolean isNominal = attributes.get(numAttribute).isNominal();
+
+			/** If value is empty/null iterate to the next attribute.**/
+			if(value==null)
+				continue;
+
+			if(isNumeric)
+			{
+				if(value instanceof Double)	
+					this.setSparseValue(instance,  indexValues, attributeValues, numAttribute, (double)value);
+				else if (value instanceof Long)	
+					this.setSparseValue(instance,indexValues, attributeValues, numAttribute, (long)value);
+				else if (value instanceof Integer)	
+					this.setSparseValue(instance,indexValues, attributeValues, numAttribute, (int)value);
+				else
+					throw new RuntimeException(AVRO_LOADER_INVALID_TYPE_ERROR+" : "+attribute.name);
+			}
+			else if(isNominal)
+			{
+				double valueAttribute;
+
+				if (!(value instanceof EnumSymbol))	
+					throw new RuntimeException(AVRO_LOADER_INVALID_TYPE_ERROR+" : "+attribute.name);
+
+				EnumSymbol enumSymbolalue = (EnumSymbol)value;
+
+				String stringValue = enumSymbolalue.toString();
+
+				if (("?".equals(stringValue))||(stringValue==null)) {
+					valueAttribute = Double.NaN; 
+				} else {
+					valueAttribute = this.instanceInformation.attribute(numAttribute).indexOfValue(stringValue);
+				}
+
+				this.setSparseValue(instance, indexValues, attributeValues, numAttribute, valueAttribute);
+			}
+		}
+
+		int[] arrayIndexValues = new int[attributeValues.size()];
+		double[] arrayAttributeValues = new double[attributeValues.size()];
+		
+		for (int i = 0; i < arrayIndexValues.length; i++) {
+			arrayIndexValues[i] = indexValues.get(i).intValue();
+			arrayAttributeValues[i] = attributeValues.get(i).doubleValue();
+		}
+		
+		instance.addSparseValues(arrayIndexValues, arrayAttributeValues,this.instanceInformation.numAttributes());
+		return instance;
+
+	}
+
+	/**
+	 * Sets a Sparse Value in the corresponding attribute index
+	 * @param instance  is the Instance where values will be set
+	 * @param indexValues is the list of Index values
+	 * @param attributeValues is the list of Attribute values
+	 * @param numAttribute is the index of the attribute
+	 * @param valueAttribute is the value of the attribute for this Instance
+	 */
+	private void setSparseValue(Instance instance, List<Integer> indexValues, List<Double> attributeValues, int numAttribute, double valueAttribute) {
+	
+		if (this.instanceInformation.classIndex() == numAttribute) {
+			instance.setClassValue(valueAttribute);
+		} else {
+			indexValues.add(numAttribute);
+			attributeValues.add(valueAttribute);
+		}
+	}
+
+	/**
+	 * Builds the Meta Data of from the Avro Schema
+	 * @return
+	 */
+	protected InstanceInformation getHeader() {
+
+		String relation = schema.getName();
+		attributes = new ArrayList<Attribute>();
+
+		/** By Definition, the returned list is in the order of their positions. **/
+		List<Schema.Field> fields = schema.getFields();
+
+		for (Field field : fields) {
+			Schema attributeSchema = field.schema();
+			
+			/** Currently SAMOA supports only NOMINAL & Numeric Types.**/
+			if(attributeSchema.getType()==Schema.Type.ENUM)
+			{
+				List<String> attributeLabels = attributeSchema.getEnumSymbols();
+				attributes.add(new Attribute(field.name(), attributeLabels));
+			}
+			else
+				attributes.add(new Attribute(field.name()));
+		}
+		return new InstanceInformation(relation, attributes);
+	}
+
+	/**
+	 * Identifies if the dataset is is Sparse or Dense
+	 * @return boolean
+	 */
+	protected boolean isSparseData()
+	{
+		List<Schema.Field> fields = schema.getFields();
+		for (Field field : fields) {
+			Schema attributeSchema = field.schema(); 
+
+			/** If even one attribute has a null union (nullable attribute) consider it as sparse data**/
+			if(attributeSchema.getType()==Schema.Type.UNION)
+			{
+				List<Schema> unionTypes = attributeSchema.getTypes();
+				for (Schema unionSchema : unionTypes) {
+					if(unionSchema.getType()==Schema.Type.NULL)
+						return true;
+				}
+			}
+
+		}
+		return false;
+	}
+	
+	@Override
+	public InstanceInformation getStructure() {
+		return this.instanceInformation;
+	}
+
+	/** Error Messages to for all types of Avro Loaders */
+	protected static final String AVRO_LOADER_INVALID_TYPE_ERROR = "Invalid data type in the Avro data"; 
+	protected static final String AVRO_LOADER_SCHEMA_READ_ERROR = "Exception while reading the schema from Avro File"; 
+	protected static final String AVRO_LOADER_INSTANCE_READ_ERROR = "Exception while reading the Instance from Avro File.";
+}

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/74979782/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java b/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java
index 556caaa..707d7f2 100644
--- a/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/Instances.java
@@ -30,6 +30,7 @@ import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
+import java.io.InputStream;
 
 /**
  * 
@@ -45,8 +46,11 @@ public class Instances implements Serializable {
    * The instances.
    */
   protected List<Instance> instances;
+  
+  transient protected Loader loader;
+  
 
-  transient protected ArffLoader arff;
+  protected static enum AVRO_ENCODING_FORMAT{JSON,BINARY}
 
   protected int classAttribute;
 
@@ -69,12 +73,24 @@ public class Instances implements Serializable {
   }
 
   public Instances(Reader reader, int size, int classAttribute) {
-    this.classAttribute = classAttribute;
-    arff = new ArffLoader(reader, 0, classAttribute);
-    this.instanceInformation = arff.getStructure();
-    this.instances = new ArrayList<>();
+	  this.classAttribute = classAttribute;
+	  loader = new ArffLoader(reader, 0, classAttribute);
+	  this.instanceInformation = loader.getStructure();
+	  this.instances = new ArrayList<>();
   }
 
+  public Instances(InputStream inputStream, int classAttribute, String encodingFormat) {
+	  this.classAttribute = classAttribute;
+
+	  if(encodingFormat.equalsIgnoreCase(AVRO_ENCODING_FORMAT.BINARY.toString()))
+		  loader = new AvroBinaryLoader(inputStream, classAttribute);
+	  else
+		  loader = new AvroJsonLoader(inputStream, classAttribute);
+
+	  this.instanceInformation = loader.getStructure();
+	  this.instances = new ArrayList<>();
+  }
+	
   public Instances(Instances chunk, int capacity) {
     this(chunk);
   }
@@ -175,18 +191,22 @@ public class Instances implements Serializable {
 
   public boolean readInstance(Reader fileReader) {
 
-    // ArffReader arff = new ArffReader(reader, this, m_Lines, 1);
-    if (arff == null) {
-      arff = new ArffLoader(fileReader, 0, this.classAttribute);
-    }
-    Instance inst = arff.readInstance(fileReader);
-    if (inst != null) {
-      inst.setDataset(this);
-      add(inst);
-      return true;
-    } else {
-      return false;
-    }
+	  if (loader == null) {
+		  loader = new ArffLoader(fileReader, 0, this.classAttribute);
+	  }
+	  return readInstance() ;
+  }
+
+  public boolean readInstance() {
+
+	  Instance inst = loader.readInstance();
+	  if (inst != null) {
+		  inst.setDataset(this);
+		  add(inst);
+		  return true;
+	  } else {
+		  return false;
+	  }
   }
 
   public void delete() {

http://git-wip-us.apache.org/repos/asf/incubator-samoa/blob/74979782/samoa-instances/src/main/java/org/apache/samoa/instances/Loader.java
----------------------------------------------------------------------
diff --git a/samoa-instances/src/main/java/org/apache/samoa/instances/Loader.java b/samoa-instances/src/main/java/org/apache/samoa/instances/Loader.java
new file mode 100644
index 0000000..f806bf5
--- /dev/null
+++ b/samoa-instances/src/main/java/org/apache/samoa/instances/Loader.java
@@ -0,0 +1,45 @@
+package org.apache.samoa.instances;
+
+/*
+ * #%L
+ * SAMOA
+ * %%
+ * Copyright (C) 2014 - 2015 Apache Software Foundation
+ * %%
+ * 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.
+ * #L%
+ */
+
+
+import java.io.Serializable;
+
+/**
+ * Loads Instances from streams of different types of Input Formats e.g ARFF & AVRO
+ * @author jayadeepj
+ */
+
+public interface Loader extends Serializable{
+
+	/**
+	 * Fetch the Meta-data from the data 
+	 * @return InstanceInformation
+	 */
+	public InstanceInformation getStructure();
+
+	/**
+	 * Read a single instance from the Stream
+	 * @return Instance
+	 */
+	public Instance readInstance(); 
+
+}