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();
+
+}