You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by ca...@apache.org on 2018/01/09 21:48:18 UTC

[01/50] [abbrv] incubator-rya git commit: RYA-377 Kafka implementation of the QueryChangeLog

Repository: incubator-rya
Updated Branches:
  refs/heads/master 31e06cb1b -> 9b8162ab7


RYA-377 Kafka implementation of the QueryChangeLog

Refactored serialization to be more abstract.
Changed QueryChange to be serializable.


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

Branch: refs/heads/master
Commit: 555a5957e85a627fd6ade3832862930fa50887d8
Parents: 3ccfbad
Author: Andrew Smith <sm...@gmail.com>
Authored: Wed Oct 25 19:01:53 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:12:59 2018 -0500

----------------------------------------------------------------------
 extras/rya.streams/api/pom.xml                  |   5 +
 .../rya/streams/api/queries/ChangeLogEntry.java |   6 +-
 .../rya/streams/api/queries/QueryChange.java    |  10 +-
 .../rya/streams/api/queries/QueryChangeLog.java |   2 +-
 .../kafka/queries/KafkaQueryChangeLog.java      | 158 ++++++++++++++
 .../kafka/serialization/ObjectDeserializer.java |  72 +++++++
 .../kafka/serialization/ObjectSerializer.java   |  73 +++++++
 .../VisibilityBindingSetDeserializer.java       |  36 +---
 .../VisibilityBindingSetSerializer.java         |  35 +---
 .../VisibilityStatementDeserializer.java        |  36 +---
 .../VisibilityStatementSerializer.java          |  35 +---
 .../queries/QueryChangeDeserializer.java        |  38 ++++
 .../serialization/queries/QueryChangeSerde.java |  57 ++++++
 .../queries/QueryChangeSerializer.java          |  39 ++++
 .../kafka/queries/KafkaQueryChangeLogIT.java    | 205 +++++++++++++++++++
 15 files changed, 669 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/api/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/pom.xml b/extras/rya.streams/api/pom.xml
index 13716de..2a1f51c 100644
--- a/extras/rya.streams/api/pom.xml
+++ b/extras/rya.streams/api/pom.xml
@@ -49,6 +49,11 @@ under the License.
             <artifactId>guava</artifactId>
         </dependency>
         
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+        
         <!-- Test dependences -->
         <dependency>
             <groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/ChangeLogEntry.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/ChangeLogEntry.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/ChangeLogEntry.java
index 2a5e8a1..d57cee9 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/ChangeLogEntry.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/ChangeLogEntry.java
@@ -35,7 +35,7 @@ import edu.umd.cs.findbugs.annotations.NonNull;
 @DefaultAnnotation(NonNull.class)
 public class ChangeLogEntry<T> {
 
-    private final int position;
+    private final long position;
     private final T entry;
 
     /**
@@ -44,7 +44,7 @@ public class ChangeLogEntry<T> {
      * @param position - The position of this entry within the change log.
      * @param entry - The value that is stored at this position within the change log. (not null)
      */
-    public ChangeLogEntry(final int position, final T entry) {
+    public ChangeLogEntry(final long position, final T entry) {
         this.position = position;
         this.entry = requireNonNull(entry);
     }
@@ -52,7 +52,7 @@ public class ChangeLogEntry<T> {
     /**
      * @return The position of this entry within the change log.
      */
-    public int getPosition() {
+    public long getPosition() {
         return position;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChange.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChange.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChange.java
index 55f87f7..90af79c 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChange.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChange.java
@@ -20,10 +20,12 @@ package org.apache.rya.streams.api.queries;
 
 import static java.util.Objects.requireNonNull;
 
+import java.io.Serializable;
 import java.util.Objects;
-import java.util.Optional;
 import java.util.UUID;
 
+import com.google.common.base.Optional;
+
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
 
@@ -33,8 +35,8 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * Immutable.
  */
 @DefaultAnnotation(NonNull.class)
-public final class QueryChange {
-
+public final class QueryChange implements Serializable {
+    private static final long serialVersionUID = 1L;
     private final UUID queryId;
     private final ChangeType changeType;
     private final Optional<String> sparql;
@@ -111,7 +113,7 @@ public final class QueryChange {
      * @return A {@link QueryChange} built using the provided values.
      */
     public static QueryChange delete(final UUID queryId) {
-        return new QueryChange(queryId, ChangeType.DELETE, Optional.empty());
+        return new QueryChange(queryId, ChangeType.DELETE, Optional.absent());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java
index ba0e878..824eebc 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java
@@ -51,7 +51,7 @@ public interface QueryChangeLog {
      * @return The entries that are at and after the specified position.
      * @throws QueryChangeLogException The entries could not be fetched.
      */
-    public CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> readFromPosition(int position) throws QueryChangeLogException;
+    public CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> readFromPosition(long position) throws QueryChangeLogException;
 
     /**
      * One of the {@link QueryChangeLog} functions failed.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLog.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLog.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLog.java
new file mode 100644
index 0000000..19622ae
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLog.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.queries;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.rya.streams.api.queries.ChangeLogEntry;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.api.queries.QueryChangeLog;
+
+import com.google.common.collect.Lists;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * A Kafka implementation of a {@link QueryChangeLog}.
+ */
+@DefaultAnnotation(NonNull.class)
+public class KafkaQueryChangeLog implements QueryChangeLog {
+    /*
+     * Key is '?' since you cannot have parallel processing over a sequential
+     * change log, so there is only one partition.
+     */
+    private final Producer<?, QueryChange> producer;
+
+    /*
+     * Key is '?' since you cannot have parallel processing over a sequential
+     * change log, so there is only one partition.
+     */
+    private final Consumer<?, QueryChange> consumer;
+
+    private final String topic;
+
+    /**
+     * Creates a new {@link KafkaQueryChangeLog}.
+     *
+     * @param producer - The producer to use to add {@link QueryChange}s to a kafka topic. (not null)
+     * @param consumer - The consumer to use to read {@link QueryChange}s from a kafka topic. (not null)
+     * @param topic - The topic on kafka to read/write from. (not null)
+     */
+    public KafkaQueryChangeLog(final Producer<?, QueryChange> producer,
+            final Consumer<?, QueryChange> consumer,
+            final String topic) {
+        this.producer = requireNonNull(producer);
+        this.consumer = requireNonNull(consumer);
+        this.topic = requireNonNull(topic);
+    }
+
+    @Override
+    public void write(final QueryChange newChange) throws QueryChangeLogException {
+        requireNonNull(newChange);
+        producer.send(new ProducerRecord<>(topic, newChange));
+    }
+
+    @Override
+    public CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> readFromStart() throws QueryChangeLogException {
+        final TopicPartition part = new TopicPartition(topic, 0);
+        consumer.assign(Lists.newArrayList(part));
+        consumer.seekToBeginning(Lists.newArrayList(part));
+        return new QueryChangeLogEntryIter(consumer);
+    }
+
+    @Override
+    public CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> readFromPosition(final long position) throws QueryChangeLogException {
+        final TopicPartition part = new TopicPartition(topic, 0);
+        consumer.assign(Lists.newArrayList(part));
+        consumer.seek(part, position);
+        return new QueryChangeLogEntryIter(consumer);
+    }
+
+    /**
+     * A {@link CloseableIteration} to iterate over a consumer's results. Since
+     * the consumer returns in bulk when poll(), a cache of recent polling is
+     * maintained.
+     *
+     * If there are no new results after 3 seconds,
+     * {@link QueryChangeLogEntryIter#hasNext()} will return false.
+     */
+    private class QueryChangeLogEntryIter implements CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> {
+        private final Consumer<?, QueryChange> consumer;
+        private Iterator<ChangeLogEntry<QueryChange>> iterCache;
+
+        /**
+         * Creates a new {@link QueryChangeLogEntryIter}.
+         *
+         * @param consumer - The consumer to iterate over. (not null)
+         */
+        public QueryChangeLogEntryIter(final Consumer<?, QueryChange> consumer) {
+            this.consumer = requireNonNull(consumer);
+        }
+
+        @Override
+        public boolean hasNext() throws QueryChangeLogException {
+            if (iterCache == null || !iterCache.hasNext()) {
+                populateCache();
+            }
+            return iterCache.hasNext();
+        }
+
+        @Override
+        public ChangeLogEntry<QueryChange> next() throws QueryChangeLogException {
+            if (iterCache == null && iterCache.hasNext()) {
+                populateCache();
+            }
+
+            if (iterCache.hasNext()) {
+                return iterCache.next();
+            }
+            throw new QueryChangeLogException("There are no changes in the change log.");
+        }
+
+        @Override
+        public void remove() throws QueryChangeLogException {
+        }
+
+        @Override
+        public void close() throws QueryChangeLogException {
+            consumer.unsubscribe();
+        }
+
+        private void populateCache() {
+            final ConsumerRecords<?, QueryChange> records = consumer.poll(3000L);
+            final List<ChangeLogEntry<QueryChange>> changes = new ArrayList<>();
+            records.forEach(
+                    record -> 
+                        changes.add(new ChangeLogEntry<QueryChange>(record.offset(), record.value()))
+                    );
+            iterCache = changes.iterator();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/ObjectDeserializer.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/ObjectDeserializer.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/ObjectDeserializer.java
new file mode 100644
index 0000000..8a1e50b
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/ObjectDeserializer.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.serialization;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A Kafka {@link Deserializer} that is able to deserialize entities.
+ *
+ * @param T - The type of entity to deserialize.
+ */
+@DefaultAnnotation(NonNull.class)
+public abstract class ObjectDeserializer<T> implements Deserializer<T> {
+
+    private static final Logger log = LoggerFactory.getLogger(ObjectDeserializer.class);
+
+    @Override
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
+        // Nothing to do.
+    }
+
+    @Override
+    public T deserialize(final String topic, final byte[] data) {
+        if(data == null || data.length == 0) {
+            // Returning null because that is the contract of this method.
+            return null;
+        }
+
+        try {
+            return ObjectSerialization.deserialize(data, getDeserializedClass());
+        } catch (final ClassNotFoundException | ClassCastException | IOException e) {
+            log.error("Could not deserialize some data into a " + getDeserializedClass().getName() + ". This data will be skipped.", e);
+
+            // Returning null because that is the contract of this method.
+            return null;
+        }
+    }
+
+    @Override
+    public void close() {
+        // Nothing to do.
+    }
+
+    /**
+     * @return - Used by the {@link ObjectSerialization#deserialize()} and the logger.
+     */
+    protected abstract Class<T> getDeserializedClass();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/ObjectSerializer.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/ObjectSerializer.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/ObjectSerializer.java
new file mode 100644
index 0000000..24f7652
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/ObjectSerializer.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.serialization;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.kafka.common.serialization.Serializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A Kafka {@link Serializer} that is able to serialize entities using Java
+ * object serialization.
+ *
+ * @param T - The type of entity to serialize.
+ */
+@DefaultAnnotation(NonNull.class)
+public abstract class ObjectSerializer<T> implements Serializer<T> {
+
+    private static final Logger log = LoggerFactory.getLogger(ObjectSerializer.class);
+
+    @Override
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
+        // Nothing to do.
+    }
+
+    @Override
+    public byte[] serialize(final String topic, final T data) {
+        if(data == null) {
+            return null;
+        }
+
+        try {
+            return ObjectSerialization.serialize(data);
+        } catch (final IOException e) {
+            log.error("Unable to serialize a " + getSerializedClass().getName() + ".", e);
+
+            // Return null when there is an error since that is the contract of this method.
+            return null;
+        }
+    }
+
+
+    @Override
+    public void close() {
+        // Nothing to do.
+    }
+
+    /**
+     * @return - The class name of T. This is used for logging purposes.
+     */
+    protected abstract Class<T> getSerializedClass();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetDeserializer.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetDeserializer.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetDeserializer.java
index 1232ad9..011a311 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetDeserializer.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetDeserializer.java
@@ -18,13 +18,8 @@
  */
 package org.apache.rya.streams.kafka.serialization;
 
-import java.io.IOException;
-import java.util.Map;
-
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.rya.api.model.VisibilityBindingSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -33,34 +28,9 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * A Kafka {@link Deserializer} that is able to deserialize Java object serialized {@link VisibilityBindingSet}s.
  */
 @DefaultAnnotation(NonNull.class)
-public class VisibilityBindingSetDeserializer implements Deserializer<VisibilityBindingSet> {
-
-    private static final Logger log = LoggerFactory.getLogger(VisibilityBindingSetDeserializer.class);
-
-    @Override
-    public void configure(final Map<String, ?> configs, final boolean isKey) {
-        // Nothing to do.
-    }
-
-    @Override
-    public VisibilityBindingSet deserialize(final String topic, final byte[] data) {
-        if(data == null || data.length == 0) {
-            // Returning null because that is the contract of this method.
-            return null;
-        }
-
-        try {
-            return ObjectSerialization.deserialize(data, VisibilityBindingSet.class);
-        } catch (final ClassNotFoundException | ClassCastException | IOException e) {
-            log.error("Could not deserialize some data into a " + VisibilityBindingSet.class.getName() + ". This data will be skipped.", e);
-
-            // Returning null because that is the contract of this method.
-            return null;
-        }
-    }
-
+public class VisibilityBindingSetDeserializer extends ObjectDeserializer<VisibilityBindingSet> {
     @Override
-    public void close() {
-        // Nothing to do.
+    protected Class<VisibilityBindingSet> getDeserializedClass() {
+        return VisibilityBindingSet.class;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetSerializer.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetSerializer.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetSerializer.java
index b2acdf2..c9cf36f 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetSerializer.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetSerializer.java
@@ -18,13 +18,8 @@
  */
 package org.apache.rya.streams.kafka.serialization;
 
-import java.io.IOException;
-import java.util.Map;
-
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.rya.api.model.VisibilityBindingSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -33,33 +28,9 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * A Kafka {@link Serializer} that is able to serialize {@link VisibilityBinidngSet}s using Java object serialization.
  */
 @DefaultAnnotation(NonNull.class)
-public class VisibilityBindingSetSerializer implements Serializer<VisibilityBindingSet> {
-
-    private static final Logger log = LoggerFactory.getLogger(VisibilityBindingSetDeserializer.class);
-
-    @Override
-    public void configure(final Map<String, ?> configs, final boolean isKey) {
-        // Nothing to do.
-    }
-
-    @Override
-    public byte[] serialize(final String topic, final VisibilityBindingSet data) {
-        if(data == null) {
-            return null;
-        }
-
-        try {
-            return ObjectSerialization.serialize(data);
-        } catch (final IOException e) {
-            log.error("Unable to serialize a " + VisibilityBindingSet.class.getName() + ".", e);
-
-            // Return null when there is an error since that is the contract of this method.
-            return null;
-        }
-    }
-
+public class VisibilityBindingSetSerializer extends ObjectSerializer<VisibilityBindingSet> {
     @Override
-    public void close() {
-        // Nothing to do.
+    protected Class<VisibilityBindingSet> getSerializedClass() {
+        return VisibilityBindingSet.class;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementDeserializer.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementDeserializer.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementDeserializer.java
index c0cd63c..4c03d96 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementDeserializer.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementDeserializer.java
@@ -18,13 +18,8 @@
  */
 package org.apache.rya.streams.kafka.serialization;
 
-import java.io.IOException;
-import java.util.Map;
-
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -33,34 +28,9 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * A Kafka {@link Deserializer} that is able to deserialize Java object serialized {@link VisibilityStatement}s.
  */
 @DefaultAnnotation(NonNull.class)
-public class VisibilityStatementDeserializer implements Deserializer<VisibilityStatement> {
-
-    private static final Logger log = LoggerFactory.getLogger(VisibilityStatement.class);
-
-    @Override
-    public void configure(final Map<String, ?> configs, final boolean isKey) {
-        // Nothing to do.
-    }
-
-    @Override
-    public VisibilityStatement deserialize(final String topic, final byte[] data) {
-        if(data == null || data.length == 0) {
-            // Returning null because that is the contract of this method.
-            return null;
-        }
-
-        try {
-            return ObjectSerialization.deserialize(data, VisibilityStatement.class);
-        } catch (final ClassNotFoundException | ClassCastException | IOException e) {
-            log.error("Could not deserialize some data into a " + VisibilityStatement.class.getName() + ". This data will be skipped.", e);
-
-            // Returning null because that is the contract of this method.
-            return null;
-        }
-    }
-
+public class VisibilityStatementDeserializer extends ObjectDeserializer<VisibilityStatement> {
     @Override
-    public void close() {
-        // Nothing to do.
+    protected Class<VisibilityStatement> getDeserializedClass() {
+        return VisibilityStatement.class;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementSerializer.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementSerializer.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementSerializer.java
index c0b526f..2395bf0 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementSerializer.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementSerializer.java
@@ -18,13 +18,8 @@
  */
 package org.apache.rya.streams.kafka.serialization;
 
-import java.io.IOException;
-import java.util.Map;
-
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -33,33 +28,9 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * A Kafka {@link Serializer} that is able to serialize {@link VisibilityStatement}s using Java object serialization.
  */
 @DefaultAnnotation(NonNull.class)
-public class VisibilityStatementSerializer implements Serializer<VisibilityStatement> {
-
-    private static final Logger log = LoggerFactory.getLogger(VisibilityStatementSerializer.class);
-
-    @Override
-    public void configure(final Map<String, ?> configs, final boolean isKey) {
-        // Nothing to do.
-    }
-
-    @Override
-    public byte[] serialize(final String topic, final VisibilityStatement data) {
-        if(data == null) {
-            return null;
-        }
-
-        try {
-            return ObjectSerialization.serialize(data);
-        } catch (final IOException e) {
-            log.error("Unable to serialize a " + VisibilityStatement.class.getName() + ".", e);
-
-            // Return null when there is an error since that is the contract of this method.
-            return null;
-        }
-    }
-
+public class VisibilityStatementSerializer extends ObjectSerializer<VisibilityStatement> {
     @Override
-    public void close() {
-        // Nothing to do.
+    protected Class<VisibilityStatement> getSerializedClass() {
+        return VisibilityStatement.class;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeDeserializer.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeDeserializer.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeDeserializer.java
new file mode 100644
index 0000000..96538b2
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeDeserializer.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.serialization.queries;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.kafka.serialization.ObjectDeserializer;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A Kafka {@link Deserializer} that is able to deserialize Java object serialized {@link QueryChange}s.
+ */
+@DefaultAnnotation(NonNull.class)
+public class QueryChangeDeserializer extends ObjectDeserializer<QueryChange> {
+
+    @Override
+    protected Class<QueryChange> getDeserializedClass() {
+        return QueryChange.class;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeSerde.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeSerde.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeSerde.java
new file mode 100644
index 0000000..c2e0469
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeSerde.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.serialization.queries;
+
+import java.util.Map;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.rya.streams.api.queries.QueryChange;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Provides a {@link Serializer} and {@link Deserializer} for
+ * {@link QueryChange}s.
+ */
+@DefaultAnnotation(NonNull.class)
+public class QueryChangeSerde implements Serde<QueryChange> {
+
+    @Override
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
+        // Nothing to do.
+    }
+
+    @Override
+    public Serializer<QueryChange> serializer() {
+        return new QueryChangeSerializer();
+    }
+
+    @Override
+    public Deserializer<QueryChange> deserializer() {
+        return new QueryChangeDeserializer();
+    }
+
+    @Override
+    public void close() {
+        // Nothing to do.
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeSerializer.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeSerializer.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeSerializer.java
new file mode 100644
index 0000000..8a36680
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/serialization/queries/QueryChangeSerializer.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.serialization.queries;
+
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.kafka.serialization.ObjectSerializer;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A Kafka {@link Serializer} that is able to serialize {@link QueryChange}s
+ * using Java object serialization.
+ */
+@DefaultAnnotation(NonNull.class)
+public class QueryChangeSerializer extends ObjectSerializer<QueryChange> {
+
+    @Override
+    protected Class<QueryChange> getSerializedClass() {
+        return QueryChange.class;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/555a5957/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
new file mode 100644
index 0000000..9e89ca7
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.queries;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.rya.streams.api.queries.ChangeLogEntry;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.api.queries.QueryChangeLog.QueryChangeLogException;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
+import org.apache.rya.test.kafka.KafkaITBase;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * Integration tests the {@link KafkaQueryChangeLog}.
+ */
+public class KafkaQueryChangeLogIT extends KafkaITBase {
+    KafkaQueryChangeLog changeLog;
+
+    private Producer<?, QueryChange> producer;
+    private Consumer<?, QueryChange> consumer;
+
+    private String topic;
+
+    @Rule
+    public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
+
+    @Before
+    public void setup() {
+        topic = rule.getKafkaTopicName();
+        final Properties producerProperties = rule.createBootstrapServerConfig();
+        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
+
+        final Properties consumerProperties = rule.createBootstrapServerConfig();
+        consumerProperties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
+        consumerProperties.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
+        producer = new KafkaProducer<>(producerProperties);
+        consumer = new KafkaConsumer<>(consumerProperties);
+        changeLog = new KafkaQueryChangeLog(producer, consumer, topic);
+    }
+
+    @After
+    public void cleanup() {
+        producer.flush();
+        producer.close();
+
+        consumer.close();
+    }
+
+    @Test
+    public void testWrite() throws Exception {
+        final String sparql = "SOME QUERY HERE";
+        final UUID uuid = UUID.randomUUID();
+        final QueryChange newChange = QueryChange.create(uuid, sparql);
+        changeLog.write(newChange);
+
+        consumer.subscribe(Lists.newArrayList(topic));
+        final ConsumerRecords<?, QueryChange> records = consumer.poll(2000);
+        assertEquals(1, records.count());
+
+        final QueryChange record = records.iterator().next().value();
+        assertEquals(newChange, record);
+    }
+
+    @Test
+    public void readFromBegining() throws Exception {
+        final List<QueryChange> expected = write10ChangesToChangeLog();
+
+        final CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> iter = changeLog.readFromStart();
+
+        final List<QueryChange> actual = new ArrayList<>();
+        while (iter.hasNext()) {
+            final ChangeLogEntry<QueryChange> entry = iter.next();
+            actual.add(entry.getEntry());
+        }
+        assertEquals(expected, actual);
+    }
+
+    @Test
+    public void readFromBegining_positionStartsNotBegining() throws Exception {
+        final List<QueryChange> expected = write10ChangesToChangeLog();
+
+        // set the position to some non-0 position
+        final TopicPartition partition = new TopicPartition(topic, 0);
+        consumer.assign(Lists.newArrayList(partition));
+        consumer.seek(partition, 5L);
+        final CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> iter = changeLog.readFromStart();
+
+        final List<QueryChange> actual = new ArrayList<>();
+        while (iter.hasNext()) {
+            final ChangeLogEntry<QueryChange> entry = iter.next();
+            actual.add(entry.getEntry());
+        }
+        assertEquals(expected, actual);
+    }
+
+    @Test
+    public void readFromPosition_positionStartsBegining() throws Exception {
+        final List<QueryChange> expected = write10ChangesToChangeLog().subList(5, 10);
+
+        // set the position to some non-0 position
+        final TopicPartition partition = new TopicPartition(topic, 0);
+        consumer.assign(Lists.newArrayList(partition));
+        consumer.seekToBeginning(Lists.newArrayList(partition));
+        final CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> iter = changeLog.readFromPosition(5L);
+
+        final List<QueryChange> actual = new ArrayList<>();
+        while (iter.hasNext()) {
+            final ChangeLogEntry<QueryChange> entry = iter.next();
+            actual.add(entry.getEntry());
+        }
+        assertEquals(expected, actual);
+    }
+
+    @Test
+    public void readFromPosition_positionStartsNotBegining() throws Exception {
+        final List<QueryChange> expected = write10ChangesToChangeLog().subList(5, 10);
+
+        // set the position to some non-0 position
+        final TopicPartition partition = new TopicPartition(topic, 0);
+        consumer.assign(Lists.newArrayList(partition));
+        consumer.seekToEnd(Lists.newArrayList(partition));
+        final CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> iter = changeLog.readFromPosition(5L);
+
+        final List<QueryChange> actual = new ArrayList<>();
+        while (iter.hasNext()) {
+            final ChangeLogEntry<QueryChange> entry = iter.next();
+            actual.add(entry.getEntry());
+        }
+        assertEquals(expected, actual);
+    }
+
+    @Test
+    public void readFromPosition_positionStartsEnd() throws Exception {
+        write10ChangesToChangeLog();
+
+        // set the position to some non-0 position
+        final TopicPartition partition = new TopicPartition(topic, 0);
+        consumer.assign(Lists.newArrayList(partition));
+        consumer.seekToEnd(Lists.newArrayList(partition));
+        final CloseableIteration<ChangeLogEntry<QueryChange>, QueryChangeLogException> iter = changeLog.readFromPosition(10L);
+        int count = 0;
+        while (iter.hasNext()) {
+            // should be empty
+            iter.next();
+            count++;
+        }
+        assertEquals(0, count);
+    }
+
+    private List<QueryChange> write10ChangesToChangeLog() throws Exception {
+        final List<QueryChange> changes = new ArrayList<>();
+        for (int ii = 0; ii < 10; ii++) {
+            final String sparql = "SOME QUERY HERE_" + ii;
+            final UUID uuid = UUID.randomUUID();
+            final QueryChange newChange = QueryChange.create(uuid, sparql);
+            changeLog.write(newChange);
+            changes.add(newChange);
+        }
+        return changes;
+    }
+}



[44/50] [abbrv] incubator-rya git commit: RYA-377 Implement a command for running a Rya Streams query out of the command line client.

Posted by ca...@apache.org.
RYA-377 Implement a command for running a Rya Streams query out of the command line client.


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

Branch: refs/heads/master
Commit: 94423229ebe7b34e0fb6c17fbe022e080cfe79d9
Parents: a5e3618
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Nov 14 18:32:53 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../rya/streams/api/interactor/RunQuery.java    |  41 ++++
 .../api/queries/InMemoryQueryRepository.java    |  21 +-
 .../streams/api/queries/QueryRepository.java    |  14 +-
 .../queries/InMemoryQueryRepositoryTest.java    | 121 ++++++-----
 .../apache/rya/streams/client/CLIDriver.java    |   2 +
 .../client/command/LoadStatementsCommand.java   |   1 +
 .../streams/client/command/RunQueryCommand.java | 155 ++++++++++++++
 .../client/command/StreamResultsCommand.java    |   2 +-
 .../client/command/AddQueryCommandIT.java       |  55 ++---
 .../client/command/DeleteQueryCommandIT.java    | 183 +++++++---------
 .../client/command/ListQueryCommandIT.java      |  56 ++---
 .../client/command/LoadStatementsCommandIT.java |  78 ++-----
 .../client/command/RunQueryCommandIT.java       | 196 +++++++++++++++++
 extras/rya.streams/kafka/pom.xml                |  10 +
 .../apache/rya/streams/kafka/KafkaTopics.java   |  39 ++++
 .../kafka/interactor/KafkaLoadStatements.java   |   5 +-
 .../streams/kafka/interactor/KafkaRunQuery.java | 136 ++++++++++++
 .../processors/join/KeyValueJoinStateStore.java |   5 +-
 .../apache/rya/streams/kafka/KafkaTestUtil.java | 211 -------------------
 .../rya/streams/kafka/RyaStreamsTestUtil.java   | 124 +++++++++++
 .../interactor/KafkaGetQueryResultStreamIT.java |   2 +-
 .../kafka/interactor/KafkaLoadStatementsIT.java |   5 +-
 .../kafka/interactor/KafkaRunQueryIT.java       | 170 +++++++++++++++
 .../processors/StatementPatternProcessorIT.java |  10 +-
 .../processors/filter/FilterProcessorIT.java    |   4 +-
 .../kafka/processors/join/JoinProcessorIT.java  |  12 +-
 .../projection/MultiProjectionProcessorIT.java  |   4 +-
 .../projection/ProjectionProcessorIT.java       |   4 +-
 .../kafka/queries/KafkaQueryChangeLogIT.java    |   2 +-
 .../VisibilityBindingSetKafkaIT.java            |   2 +-
 .../VisibilityStatementKafkaIT.java             |   2 +-
 test/kafka/pom.xml                              |   5 +
 .../rya/test/kafka/KafkaTestInstanceRule.java   |   7 +
 .../apache/rya/test/kafka/KafkaTestUtil.java    | 126 +++++++++++
 34 files changed, 1269 insertions(+), 541 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/RunQuery.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/RunQuery.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/RunQuery.java
new file mode 100644
index 0000000..7f47095
--- /dev/null
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/RunQuery.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.api.interactor;
+
+import java.util.UUID;
+
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Runs a Rya Streams processing topology on the machine this class is invoked on.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface RunQuery {
+
+    /**
+     * Runs the specified query on the machine this method was invoked on.
+     *
+     * @param queryId - The id of the query that will be processed. (not null)
+     * @throws RyaStreamsException The query could not processed.
+     */
+    public void run(UUID queryId) throws RyaStreamsException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java
index c1048fc..80678de 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java
@@ -22,6 +22,7 @@ import static java.util.Objects.requireNonNull;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.locks.ReentrantLock;
@@ -63,13 +64,7 @@ public class InMemoryQueryRepository implements QueryRepository {
         this.changeLog = requireNonNull(changeLog);
 
         // Lazily initialize the queries cache the first time you try to use it.
-        queriesCache = Suppliers.memoize(new Supplier<Map<UUID, StreamsQuery>>() {
-            @Override
-            public Map<UUID, StreamsQuery> get() {
-                // Initialize the queries cache using the current state of the change log.
-                return initializeCache(changeLog);
-            }
-        });
+        queriesCache = Suppliers.memoize(() -> initializeCache(changeLog));
     }
 
     @Override
@@ -98,6 +93,18 @@ public class InMemoryQueryRepository implements QueryRepository {
     }
 
     @Override
+    public Optional<StreamsQuery> get(final UUID queryId) throws QueryRepositoryException {
+        requireNonNull(queryId);
+
+        lock.lock();
+        try {
+            return Optional.ofNullable( queriesCache.get().get(queryId) );
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    @Override
     public void delete(final UUID queryId) throws QueryRepositoryException {
         requireNonNull(queryId);
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java
index 850b2bc..7269588 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java
@@ -18,6 +18,7 @@
  */
 package org.apache.rya.streams.api.queries;
 
+import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 
@@ -32,6 +33,7 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  */
 @DefaultAnnotation(NonNull.class)
 public interface QueryRepository extends AutoCloseable {
+
     /**
      * Adds a new query to Rya Streams.
      *
@@ -42,6 +44,15 @@ public interface QueryRepository extends AutoCloseable {
     public StreamsQuery add(final String query) throws QueryRepositoryException;
 
     /**
+     * Get an existing query from Rya Streams.
+     *
+     * @param queryId - Identifies which query will be fetched.
+     * @return the {@link StreamsQuery} for the id if one exists; otherwise empty.
+     * @throws QueryRepositoryException The query could not be fetched.
+     */
+    public Optional<StreamsQuery> get(UUID queryId) throws QueryRepositoryException;
+
+    /**
      * Removes an existing query from Rya Streams.
      *
      * @param queryID - The {@link UUID} of the query to remove. (not null)
@@ -53,8 +64,7 @@ public interface QueryRepository extends AutoCloseable {
      * Lists all existing queries in Rya Streams.
      *
      * @return - A List of the current {@link StreamsQuery}s
-     * @throws QueryRepositoryException The {@link StreamsQuery}s could not be
-     *         listed.
+     * @throws QueryRepositoryException The {@link StreamsQuery}s could not be listed.
      */
     public Set<StreamsQuery> list() throws QueryRepositoryException;
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/api/src/test/java/org/apache/rya/streams/api/queries/InMemoryQueryRepositoryTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/test/java/org/apache/rya/streams/api/queries/InMemoryQueryRepositoryTest.java b/extras/rya.streams/api/src/test/java/org/apache/rya/streams/api/queries/InMemoryQueryRepositoryTest.java
index 25cbab2..92193ca 100644
--- a/extras/rya.streams/api/src/test/java/org/apache/rya/streams/api/queries/InMemoryQueryRepositoryTest.java
+++ b/extras/rya.streams/api/src/test/java/org/apache/rya/streams/api/queries/InMemoryQueryRepositoryTest.java
@@ -19,16 +19,17 @@
 package org.apache.rya.streams.api.queries;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.util.HashSet;
+import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 
 import org.apache.rya.streams.api.entity.StreamsQuery;
 import org.apache.rya.streams.api.queries.QueryChangeLog.QueryChangeLogException;
-import org.apache.rya.streams.api.queries.QueryRepository.QueryRepositoryException;
 import org.junit.Test;
 
 /**
@@ -37,69 +38,95 @@ import org.junit.Test;
 public class InMemoryQueryRepositoryTest {
 
     @Test
-    public void canReadAddedQueries() throws QueryRepositoryException {
+    public void canReadAddedQueries() throws Exception {
         // Setup a totally in memory QueryRepository.
-        final QueryRepository queries = new InMemoryQueryRepository( new InMemoryQueryChangeLog() );
-
-        // Add some queries to it.
-        final Set<StreamsQuery> expected = new HashSet<>();
-        expected.add( queries.add("query 1") );
-        expected.add( queries.add("query 2") );
-        expected.add( queries.add("query 3") );
-
-        // Show they are in the list of all queries.
-        final Set<StreamsQuery> stored = queries.list();
-        assertEquals(expected, stored);
+        try(final QueryRepository queries = new InMemoryQueryRepository( new InMemoryQueryChangeLog() )) {
+            // Add some queries to it.
+            final Set<StreamsQuery> expected = new HashSet<>();
+            expected.add( queries.add("query 1") );
+            expected.add( queries.add("query 2") );
+            expected.add( queries.add("query 3") );
+
+            // Show they are in the list of all queries.
+            final Set<StreamsQuery> stored = queries.list();
+            assertEquals(expected, stored);
+        }
     }
 
     @Test
-    public void deletedQueriesDisappear() throws QueryRepositoryException {
+    public void deletedQueriesDisappear() throws Exception {
         // Setup a totally in memory QueryRepository.
-        final QueryRepository queries = new InMemoryQueryRepository( new InMemoryQueryChangeLog() );
-
-        // Add some queries to it. The second one we will delete.
-        final Set<StreamsQuery> expected = new HashSet<>();
-        expected.add( queries.add("query 1") );
-        final UUID deletedMeId = queries.add("query 2").getQueryId();
-        expected.add( queries.add("query 3") );
-
-        // Delete the second query.
-        queries.delete( deletedMeId );
-
-        // Show only queries 1 and 3 are in the list.
-        final Set<StreamsQuery> stored = queries.list();
-        assertEquals(expected, stored);
+        try(final QueryRepository queries = new InMemoryQueryRepository( new InMemoryQueryChangeLog() )) {
+            // Add some queries to it. The second one we will delete.
+            final Set<StreamsQuery> expected = new HashSet<>();
+            expected.add( queries.add("query 1") );
+            final UUID deletedMeId = queries.add("query 2").getQueryId();
+            expected.add( queries.add("query 3") );
+
+            // Delete the second query.
+            queries.delete( deletedMeId );
+
+            // Show only queries 1 and 3 are in the list.
+            final Set<StreamsQuery> stored = queries.list();
+            assertEquals(expected, stored);
+        }
     }
 
     @Test
-    public void initializedWithPopulatedChnageLog() throws QueryRepositoryException {
+    public void initializedWithPopulatedChnageLog() throws Exception {
         // Setup a totally in memory QueryRepository. Hold onto the change log so that we can use it again later.
         final QueryChangeLog changeLog = new InMemoryQueryChangeLog();
-        final QueryRepository queries = new InMemoryQueryRepository( changeLog );
-
-        // Add some queries and deletes to it.
-        final Set<StreamsQuery> expected = new HashSet<>();
-        expected.add( queries.add("query 1") );
-        final UUID deletedMeId = queries.add("query 2").getQueryId();
-        expected.add( queries.add("query 3") );
-        queries.delete( deletedMeId );
-
-        // Create a new totally in memory QueryRepository.
-        final QueryRepository initializedQueries = new InMemoryQueryRepository( changeLog );
-
-        // Listing the queries should work using an initialized change log.
-        final Set<StreamsQuery> stored = initializedQueries.list();
-        assertEquals(expected, stored);
+        try(final QueryRepository queries = new InMemoryQueryRepository( changeLog )) {
+            // Add some queries and deletes to it.
+            final Set<StreamsQuery> expected = new HashSet<>();
+            expected.add( queries.add("query 1") );
+            final UUID deletedMeId = queries.add("query 2").getQueryId();
+            expected.add( queries.add("query 3") );
+            queries.delete( deletedMeId );
+
+            // Create a new totally in memory QueryRepository.
+            try(final QueryRepository initializedQueries = new InMemoryQueryRepository( changeLog )) {
+                // Listing the queries should work using an initialized change log.
+                final Set<StreamsQuery> stored = initializedQueries.list();
+                assertEquals(expected, stored);
+            }
+        }
     }
 
     @Test(expected = RuntimeException.class)
-    public void changeLogThrowsExceptions() throws QueryChangeLogException, QueryRepositoryException {
+    public void changeLogThrowsExceptions() throws Exception {
         // Create a mock change log that throws an exception when you try to list what is in it.
         final QueryChangeLog changeLog = mock(QueryChangeLog.class);
         when(changeLog.readFromStart()).thenThrow(new QueryChangeLogException("Mocked exception."));
 
         // Create the QueryRepository and invoke one of the methods.
-        final QueryRepository queries = new InMemoryQueryRepository( changeLog );
-        queries.list();
+        try(final QueryRepository queries = new InMemoryQueryRepository( changeLog )) {
+            queries.list();
+        }
+    }
+
+    @Test
+    public void get_present() throws Exception {
+        // Setup a totally in memory QueryRepository.
+        try(final QueryRepository queries = new InMemoryQueryRepository( new InMemoryQueryChangeLog() )) {
+            // Add a query to it.
+            final StreamsQuery query = queries.add("query 1");
+
+            // Show the fetched query matches the expected ones.
+            final Optional<StreamsQuery> fetched = queries.get(query.getQueryId());
+            assertEquals(query, fetched.get());
+        }
+    }
+
+    @Test
+    public void get_notPresent() throws Exception {
+        // Setup a totally in memory QueryRepository.
+        try(final QueryRepository queries = new InMemoryQueryRepository( new InMemoryQueryChangeLog() )) {
+            // Fetch a query that was never added to the repository.
+            final Optional<StreamsQuery> query = queries.get(UUID.randomUUID());
+
+            // Show it could not be found.
+            assertFalse(query.isPresent());
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java
index 5c0816f..05e75d9 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java
@@ -30,6 +30,7 @@ import org.apache.rya.streams.client.command.AddQueryCommand;
 import org.apache.rya.streams.client.command.DeleteQueryCommand;
 import org.apache.rya.streams.client.command.ListQueriesCommand;
 import org.apache.rya.streams.client.command.LoadStatementsCommand;
+import org.apache.rya.streams.client.command.RunQueryCommand;
 import org.apache.rya.streams.client.command.StreamResultsCommand;
 
 import com.google.common.collect.ImmutableMap;
@@ -63,6 +64,7 @@ public class CLIDriver {
         commandClasses.add(DeleteQueryCommand.class);
         commandClasses.add(ListQueriesCommand.class);
         commandClasses.add(LoadStatementsCommand.class);
+        commandClasses.add(RunQueryCommand.class);
         commandClasses.add(StreamResultsCommand.class);
         final ImmutableMap.Builder<String, RyaStreamsCommand> builder = ImmutableMap.builder();
         for(final Class<? extends RyaStreamsCommand> commandClass : commandClasses) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
index 9414b28..42020b3 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
@@ -127,6 +127,7 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
         final Properties producerProps = buildProperties(params);
         try (final Producer<Object, VisibilityStatement> producer = new KafkaProducer<>(producerProps)) {
             final LoadStatements statements = new KafkaLoadStatements(KafkaTopics.statementsTopic(params.ryaInstance), producer);
+            System.out.printf("Loading statements from file `%s` using visibilities `%s`.\n", statementsPath, params.visibilities);
             statements.fromFile(statementsPath, params.visibilities);
         } catch (final Exception e) {
             System.err.println("Unable to parse statements file: " + statementsPath.toString());

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/RunQueryCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/RunQueryCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/RunQueryCommand.java
new file mode 100644
index 0000000..8f7f162
--- /dev/null
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/RunQueryCommand.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.client.command;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.rya.streams.api.entity.StreamsQuery;
+import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
+import org.apache.rya.streams.api.queries.QueryChangeLog;
+import org.apache.rya.streams.api.queries.QueryRepository;
+import org.apache.rya.streams.client.RyaStreamsCommand;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.interactor.KafkaRunQuery;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLogFactory;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.google.common.base.Strings;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A command that runs a Rya Streams processing topology on the node the client is executed on until it has finished.
+ */
+@DefaultAnnotation(NonNull.class)
+public class RunQueryCommand implements RyaStreamsCommand {
+
+    private class RunParameters extends RyaStreamsCommand.KafkaParameters {
+        @Parameter(names = { "--queryID", "-q" }, required = true, description = "The ID of the query to run.")
+        private String queryId;
+
+        @Parameter(names = {"--zookeepers", "-z"}, required = true, description = "The servers that Zookeeper runs on.")
+        private String zookeeperServers;
+
+        @Override
+        public String toString() {
+            final StringBuilder parameters = new StringBuilder();
+            parameters.append(super.toString());
+
+            if (!Strings.isNullOrEmpty(queryId)) {
+                parameters.append("\tQueryID: " + queryId);
+                parameters.append("\n");
+            }
+            return parameters.toString();
+        }
+    }
+
+    @Override
+    public String getCommand() {
+        return "run-query";
+    }
+
+    @Override
+    public String getDescription() {
+        return "Runs a Rya Streams query until the command is killed. This command also creates the input and output " +
+                "topics required to execute the query.";
+    }
+
+    @Override
+    public String getUsage() {
+        final JCommander parser = new JCommander(new RunParameters());
+
+        final StringBuilder usage = new StringBuilder();
+        parser.usage(usage);
+        return usage.toString();
+    }
+
+    @Override
+    public boolean validArguments(final String[] args) {
+        boolean valid = true;
+        try {
+            new JCommander(new RunParameters(), args);
+        } catch(final ParameterException e) {
+            valid = false;
+        }
+        return valid;
+    }
+
+    @Override
+    public void execute(final String[] args) throws ArgumentsException, ExecutionException {
+        requireNonNull(args);
+
+        // Parse the command line arguments.
+        final RunParameters params = new RunParameters();
+        try {
+            new JCommander(params, args);
+        } catch(final ParameterException e) {
+            throw new ArgumentsException("Could not add a new query because of invalid command line parameters.", e);
+        }
+
+        // Create the Kafka backed QueryChangeLog.
+        final String bootstrapServers = params.kafkaIP + ":" + params.kafkaPort;
+        final String topic = KafkaTopics.queryChangeLogTopic(params.ryaInstance);
+        final QueryChangeLog queryChangeLog = KafkaQueryChangeLogFactory.make(bootstrapServers, topic);
+
+        // Look up the query to be executed from the change log.
+        try(QueryRepository queryRepo = new InMemoryQueryRepository(queryChangeLog)) {
+            try {
+                final UUID queryId = UUID.fromString( params.queryId );
+                final Optional<StreamsQuery> query = queryRepo.get(queryId);
+
+                if(!query.isPresent()) {
+                    throw new ArgumentsException("There is no registered query for queryId " + params.queryId);
+                }
+
+                // Make sure the topics required by the application exists for the specified Rya instances.
+                final Set<String> topics = new HashSet<>();
+                topics.add( KafkaTopics.statementsTopic(params.ryaInstance) );
+                topics.add( KafkaTopics.queryResultsTopic(queryId) );
+                KafkaTopics.createTopic(params.zookeeperServers, topics, 1, 1);
+
+                // Run the query that uses those topics.
+                final KafkaRunQuery runQuery = new KafkaRunQuery(
+                        params.kafkaIP,
+                        params.kafkaPort,
+                        KafkaTopics.statementsTopic(params.ryaInstance),
+                        KafkaTopics.queryResultsTopic(queryId),
+                        queryRepo,
+                        new TopologyFactory());
+                runQuery.run(queryId);
+            } catch(final Exception e) {
+                throw new ExecutionException("Could not execute the Run Query command.", e);
+            }
+        } catch(final ArgumentsException | ExecutionException e) {
+            // Rethrow the exceptions that are advertised by execute.
+            throw e;
+        } catch (final Exception e) {
+            throw new ExecutionException("Problem encountered while closing the QueryRepository.", e);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
index 9de978b..64f78a3 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
@@ -126,7 +126,7 @@ public class StreamResultsCommand implements RyaStreamsCommand {
         // Execute the command.
         final GetQueryResultStream getQueryResultStream = new KafkaGetQueryResultStream(params.kafkaIP, params.kafkaPort);
 
-        try (final QueryResultStream stream = getQueryResultStream.fromNow(queryId)) {
+        try (final QueryResultStream stream = getQueryResultStream.fromStart(queryId)) {
             while(!finished.get()) {
                 for(final VisibilityBindingSet visBs : stream.poll(1000)) {
                     System.out.println(visBs);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
index ee4378e..3a412d2 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
@@ -20,17 +20,11 @@ package org.apache.rya.streams.client.command;
 
 import static org.junit.Assert.assertEquals;
 
-import java.util.Properties;
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.streams.api.entity.StreamsQuery;
@@ -43,6 +37,7 @@ import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -54,49 +49,27 @@ import org.junit.Test;
 public class AddQueryCommandIT {
 
     private final String ryaInstance = UUID.randomUUID().toString();
-
-    private String kafkaIp;
-    private String kafkaPort;
     private QueryRepository queryRepo;
 
-    private Producer<?, QueryChange> queryProducer = null;
-    private Consumer<?, QueryChange> queryConsumer = null;
-
     @Rule
-    public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
 
     @Before
     public void setup() {
-        final Properties props = rule.createBootstrapServerConfig();
-        final String location = props.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
-        final String[] tokens = location.split(":");
-
-        kafkaIp = tokens[0];
-        kafkaPort = tokens[1];
-
-        // Initialize the QueryRepository.
-        final Properties producerProperties = new Properties();
-        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
-        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
-
-        final Properties consumerProperties = new Properties();
-        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
-        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
-
-        queryProducer = new KafkaProducer<>(producerProperties);
-        queryConsumer = new KafkaConsumer<>(consumerProperties);
-
+        // Make sure the topic that the change log uses exists.
         final String changeLogTopic = KafkaTopics.queryChangeLogTopic("" + ryaInstance);
+        kafka.createTopic(changeLogTopic);
+
+        // Setup the QueryRepository used by the test.
+        final Producer<?, QueryChange> queryProducer = KafkaTestUtil.makeProducer(kafka, StringSerializer.class, QueryChangeSerializer.class);
+        final Consumer<?, QueryChange>queryConsumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, QueryChangeDeserializer.class);
         final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, changeLogTopic);
         queryRepo = new InMemoryQueryRepository(changeLog);
     }
 
     @After
-    public void cleanup() {
-        queryProducer.close();
-        queryConsumer.close();
+    public void cleanup() throws Exception {
+        queryRepo.close();
     }
 
     @Test
@@ -105,8 +78,8 @@ public class AddQueryCommandIT {
         final String query = "SELECT * WHERE { ?person <urn:name> ?name }";
         final String[] args = new String[] {
                 "-r", "" + ryaInstance,
-                "-i", kafkaIp,
-                "-p", kafkaPort,
+                "-i", kafka.getKafkaHostname(),
+                "-p", kafka.getKafkaPort(),
                 "-q", query
         };
 
@@ -126,8 +99,8 @@ public class AddQueryCommandIT {
         final String query = "SELECT * WHERE { ?person <urn:name> ?name }";
         final String[] args = new String[] {
                 "--ryaInstance", "" + ryaInstance,
-                "--kafkaHostname", kafkaIp,
-                "--kafkaPort", kafkaPort,
+                "--kafkaHostname", kafka.getKafkaHostname(),
+                "--kafkaPort", kafka.getKafkaPort(),
                 "--query", query
         };
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
index c5dad3d..91647f2 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
@@ -18,20 +18,15 @@
  */
 package org.apache.rya.streams.client.command;
 
+import static java.util.Objects.requireNonNull;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
-import java.util.Properties;
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.streams.api.entity.StreamsQuery;
@@ -44,8 +39,7 @@ import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.After;
-import org.junit.Before;
+import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.Rule;
 import org.junit.Test;
 
@@ -54,124 +48,103 @@ import org.junit.Test;
  */
 public class DeleteQueryCommandIT {
 
-    private final String ryaInstance = UUID.randomUUID().toString();
-
-    private String kafkaIp;
-    private String kafkaPort;
-
-    private Producer<?, QueryChange> queryProducer = null;
-    private Consumer<?, QueryChange> queryConsumer = null;
-
     @Rule
-    public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
-
-    @Before
-    public void setup() {
-        final Properties props = rule.createBootstrapServerConfig();
-        final String location = props.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
-        final String[] tokens = location.split(":");
-
-        kafkaIp = tokens[0];
-        kafkaPort = tokens[1];
-    }
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
 
     /**
      * This test simulates executing many commands and each of them use their own InMemoryQueryRepository. We need
      * to re-create the repo outside of the command to ensure it has the most up to date values inside of it.
+     *
+     * @param ryaInstance - The rya instance the repository is connected to. (not null)
+     * @param createTopic - Set this to true if the topic doesn't exist yet.
      */
-    private QueryRepository makeQueryRepository() {
-        final Properties producerProperties = new Properties();
-        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
-        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
-
-        final Properties consumerProperties = new Properties();
-        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
-        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
-
-        cleanup();
-        queryProducer = new KafkaProducer<>(producerProperties);
-        queryConsumer = new KafkaConsumer<>(consumerProperties);
+    private QueryRepository makeQueryRepository(final String ryaInstance, final boolean createTopic) {
+        requireNonNull(ryaInstance);
 
+        // Make sure the topic that the change log uses exists.
         final String changeLogTopic = KafkaTopics.queryChangeLogTopic("" + ryaInstance);
+        if(createTopic) {
+            kafka.createTopic(changeLogTopic);
+        }
+
+        // Setup the QueryRepository used by the test.
+        final Producer<?, QueryChange> queryProducer = KafkaTestUtil.makeProducer(kafka, StringSerializer.class, QueryChangeSerializer.class);
+        final Consumer<?, QueryChange>queryConsumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, QueryChangeDeserializer.class);
         final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, changeLogTopic);
         return new InMemoryQueryRepository(changeLog);
     }
 
-    @After
-    public void cleanup() {
-        if(queryProducer != null) {
-            queryProducer.close();
-        }
-        if(queryConsumer != null) {
-            queryConsumer.close();
-        }
-    }
-
     @Test
     public void shortParams() throws Exception {
+        final String ryaInstance = UUID.randomUUID().toString();
+
         // Add a few queries to Rya Streams.
-        QueryRepository repo = makeQueryRepository();
-        repo.add("query1");
-        final UUID query2Id = repo.add("query2").getQueryId();
-        repo.add("query3");
-
-        // Show that all three of the queries were added.
-        Set<StreamsQuery> queries = repo.list();
-        assertEquals(3, queries.size());
-
-        // Delete query 2 using the delete query command.
-        final String[] deleteArgs = new String[] {
-                "-r", "" + ryaInstance,
-                "-i", kafkaIp,
-                "-p", kafkaPort,
-                "-q", query2Id.toString()
-        };
-
-        final DeleteQueryCommand deleteCommand = new DeleteQueryCommand();
-        deleteCommand.execute(deleteArgs);
-
-        // Show query2 was deleted.
-        repo = makeQueryRepository();
-        queries = repo.list();
-        assertEquals(2, queries.size());
-
-        for(final StreamsQuery query : queries) {
-            assertNotEquals(query2Id, query.getQueryId());
+        try(QueryRepository repo = makeQueryRepository(ryaInstance, true)) {
+            repo.add("query1");
+            final UUID query2Id = repo.add("query2").getQueryId();
+            repo.add("query3");
+
+            // Show that all three of the queries were added.
+            Set<StreamsQuery> queries = repo.list();
+            assertEquals(3, queries.size());
+
+            // Delete query 2 using the delete query command.
+            final String[] deleteArgs = new String[] {
+                    "-r", "" + ryaInstance,
+                    "-i", kafka.getKafkaHostname(),
+                    "-p", kafka.getKafkaPort(),
+                    "-q", query2Id.toString()
+            };
+
+            final DeleteQueryCommand deleteCommand = new DeleteQueryCommand();
+            deleteCommand.execute(deleteArgs);
+
+            // Show query2 was deleted.
+            try(QueryRepository repo2 = makeQueryRepository(ryaInstance, false)) {
+                queries = repo2.list();
+                assertEquals(2, queries.size());
+
+                for(final StreamsQuery query : queries) {
+                    assertNotEquals(query2Id, query.getQueryId());
+                }
+            }
         }
     }
 
     @Test
     public void longParams() throws Exception {
+        final String ryaInstance = UUID.randomUUID().toString();
+
         // Add a few queries to Rya Streams.
-        QueryRepository repo = makeQueryRepository();
-        repo.add("query1");
-        final UUID query2Id = repo.add("query2").getQueryId();
-        repo.add("query3");
-
-        // Show that all three of the queries were added.
-        Set<StreamsQuery> queries = repo.list();
-        assertEquals(3, queries.size());
-
-        // Delete query 2 using the delete query command.
-        final String[] deleteArgs = new String[] {
-                "--ryaInstance", "" + ryaInstance,
-                "--kafkaHostname", kafkaIp,
-                "--kafkaPort", kafkaPort,
-                "--queryID", query2Id.toString()
-        };
-
-        final DeleteQueryCommand deleteCommand = new DeleteQueryCommand();
-        deleteCommand.execute(deleteArgs);
-
-        // Show query2 was deleted.
-        repo = makeQueryRepository();
-        queries = repo.list();
-        assertEquals(2, queries.size());
-
-        for(final StreamsQuery query : queries) {
-            assertNotEquals(query2Id, query.getQueryId());
+        try(QueryRepository repo = makeQueryRepository(ryaInstance, true)) {
+            repo.add("query1");
+            final UUID query2Id = repo.add("query2").getQueryId();
+            repo.add("query3");
+
+            // Show that all three of the queries were added.
+            Set<StreamsQuery> queries = repo.list();
+            assertEquals(3, queries.size());
+
+            // Delete query 2 using the delete query command.
+            final String[] deleteArgs = new String[] {
+                    "--ryaInstance", "" + ryaInstance,
+                    "--kafkaHostname", kafka.getKafkaHostname(),
+                    "--kafkaPort", kafka.getKafkaPort(),
+                    "--queryID", query2Id.toString()
+            };
+
+            final DeleteQueryCommand deleteCommand = new DeleteQueryCommand();
+            deleteCommand.execute(deleteArgs);
+
+            // Show query2 was deleted.
+            try(QueryRepository repo2 = makeQueryRepository(ryaInstance, false)) {
+                queries = repo2.list();
+                assertEquals(2, queries.size());
+
+                for(final StreamsQuery query : queries) {
+                    assertNotEquals(query2Id, query.getQueryId());
+                }
+            }
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
index b32967e..00b4ce0 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
@@ -18,16 +18,10 @@
  */
 package org.apache.rya.streams.client.command;
 
-import java.util.Properties;
 import java.util.UUID;
 
-import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
@@ -39,6 +33,7 @@ import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -50,52 +45,29 @@ import org.junit.Test;
 public class ListQueryCommandIT {
 
     private final String ryaInstance = UUID.randomUUID().toString();
-
-    private String kafkaIp;
-    private String kafkaPort;
     private QueryRepository queryRepo;
 
-    private Producer<?, QueryChange> queryProducer = null;
-    private Consumer<?, QueryChange> queryConsumer = null;
-
     @Rule
-    public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
 
     @Before
     public void setup() {
-        final Properties props = rule.createBootstrapServerConfig();
-        final String location = props.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
-        final String[] tokens = location.split(":");
-
-        kafkaIp = tokens[0];
-        kafkaPort = tokens[1];
-
-        // Initialize the QueryRepository.
-        final Properties producerProperties = new Properties();
-        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
-        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
-
-        final Properties consumerProperties = new Properties();
-        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
-        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
-
-        queryProducer = new KafkaProducer<>(producerProperties);
-        queryConsumer = new KafkaConsumer<>(consumerProperties);
-
+        // Make sure the topic that the change log uses exists.
         final String changeLogTopic = KafkaTopics.queryChangeLogTopic("" + ryaInstance);
+        kafka.createTopic(changeLogTopic);
+
+        // Setup the QueryRepository used by the test.
+        final Producer<?, QueryChange> queryProducer = KafkaTestUtil.makeProducer(kafka, StringSerializer.class, QueryChangeSerializer.class);
+        final Consumer<?, QueryChange>queryConsumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, QueryChangeDeserializer.class);
         final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, changeLogTopic);
         queryRepo = new InMemoryQueryRepository(changeLog);
     }
 
     @After
-    public void cleanup() {
-        queryProducer.close();
-        queryConsumer.close();
+    public void cleanup() throws Exception {
+        queryRepo.close();
     }
 
-
     @Test
     public void shortParams() throws Exception {
         // Add a few queries to Rya Streams.
@@ -106,8 +78,8 @@ public class ListQueryCommandIT {
         // Execute the List Queries command.
         final String[] args = new String[] {
                 "-r", "" + ryaInstance,
-                "-i", kafkaIp,
-                "-p", kafkaPort
+                "-i", kafka.getKafkaHostname(),
+                "-p", kafka.getKafkaPort()
         };
 
         final ListQueriesCommand command = new ListQueriesCommand();
@@ -124,8 +96,8 @@ public class ListQueryCommandIT {
         // Execute the List Queries command.
         final String[] args = new String[] {
                 "--ryaInstance", "" + ryaInstance,
-                "--kafkaHostname", kafkaIp,
-                "--kafkaPort", kafkaPort
+                "--kafkaHostname", kafka.getKafkaHostname(),
+                "--kafkaPort", kafka.getKafkaPort()
         };
 
         final ListQueriesCommand command = new ListQueriesCommand();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/LoadStatementsCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/LoadStatementsCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/LoadStatementsCommandIT.java
index 95a4876..03c31b4 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/LoadStatementsCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/LoadStatementsCommandIT.java
@@ -26,21 +26,16 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Properties;
 import java.util.UUID;
 
-import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Before;
+import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.Rule;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;
@@ -55,21 +50,8 @@ public class LoadStatementsCommandIT {
 
     private final String ryaInstance = UUID.randomUUID().toString();
 
-    private String kafkaIp;
-    private String kafkaPort;
-
     @Rule
-    public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
-
-    @Before
-    public void setup() {
-        final Properties props = rule.createBootstrapServerConfig();
-        final String location = props.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
-        final String[] tokens = location.split(":");
-
-        kafkaIp = tokens[0];
-        kafkaPort = tokens[1];
-    }
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
 
     @Test
     public void shortParams() throws Exception {
@@ -77,35 +59,27 @@ public class LoadStatementsCommandIT {
         final String visibilities = "a|b|c";
         final String[] args = new String[] {
                 "-r", "" + ryaInstance,
-                "-i", kafkaIp,
-                "-p", kafkaPort,
+                "-i", kafka.getKafkaHostname(),
+                "-p", kafka.getKafkaPort(),
                 "-f", TURTLE_FILE.toString(),
                 "-v", visibilities
         };
 
+        // Load the file of statements into the Statements topic.
         new LoadStatementsCommand().execute(args);
 
         // Show that the statements were loaded into the topic.
-        // Read a VisibilityBindingSet from the test topic.
         final List<VisibilityStatement> read = new ArrayList<>();
 
-        final Properties consumerProps = new Properties();
-        consumerProps.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
-        consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
-        consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
-        consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VisibilityStatementDeserializer.class.getName());
-
-        try(final Consumer<String, VisibilityStatement> consumer = new KafkaConsumer<>(consumerProps)) {
-            final String topic = KafkaTopics.statementsTopic(ryaInstance);
-            consumer.subscribe(Arrays.asList(topic));
-            final ConsumerRecords<String, VisibilityStatement> records = consumer.poll(3000);
+        try(final Consumer<String, VisibilityStatement> consumer =
+                KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, VisibilityStatementDeserializer.class)) {
+            // Subscribe for messages.
+            consumer.subscribe( Arrays.asList(KafkaTopics.statementsTopic(ryaInstance)) );
 
-            assertEquals(3, records.count());
-            final Iterator<ConsumerRecord<String, VisibilityStatement>> iter = records.iterator();
+            // Read the messages and extract their values.
+            final Iterator<ConsumerRecord<String, VisibilityStatement>> iter = consumer.poll(3000).iterator();
             while(iter.hasNext()) {
-                final VisibilityStatement visiSet = iter.next().value();
-                read.add(visiSet);
+                read.add( iter.next().value() );
             }
         }
 
@@ -131,35 +105,27 @@ public class LoadStatementsCommandIT {
         final String visibilities = "a|b|c";
         final String[] args = new String[] {
                 "--ryaInstance", "" + ryaInstance,
-                "--kafkaHostname", kafkaIp,
-                "--kafkaPort", kafkaPort,
+                "--kafkaHostname", kafka.getKafkaHostname(),
+                "--kafkaPort", kafka.getKafkaPort(),
                 "--statementsFile", TURTLE_FILE.toString(),
                 "--visibilities", visibilities
         };
 
+        // Load the file of statements into the Statements topic.
         new LoadStatementsCommand().execute(args);
 
         // Show that the statements were loaded into the topic.
-        // Read a VisibilityBindingSet from the test topic.
         final List<VisibilityStatement> read = new ArrayList<>();
 
-        final Properties consumerProps = new Properties();
-        consumerProps.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
-        consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
-        consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
-        consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VisibilityStatementDeserializer.class.getName());
-
-        try(final Consumer<String, VisibilityStatement> consumer = new KafkaConsumer<>(consumerProps)) {
-            final String topic = KafkaTopics.statementsTopic(ryaInstance);
-            consumer.subscribe(Arrays.asList(topic));
-            final ConsumerRecords<String, VisibilityStatement> records = consumer.poll(3000);
+        try(final Consumer<String, VisibilityStatement> consumer =
+                KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, VisibilityStatementDeserializer.class)) {
+            // Subscribe for messages.
+            consumer.subscribe( Arrays.asList(KafkaTopics.statementsTopic(ryaInstance)) );
 
-            assertEquals(3, records.count());
-            final Iterator<ConsumerRecord<String, VisibilityStatement>> iter = records.iterator();
+            // Read the messages and extract their values.
+            final Iterator<ConsumerRecord<String, VisibilityStatement>> iter = consumer.poll(3000).iterator();
             while(iter.hasNext()) {
-                final VisibilityStatement visiSet = iter.next().value();
-                read.add(visiSet);
+                read.add( iter.next().value() );
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/RunQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/RunQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/RunQueryCommandIT.java
new file mode 100644
index 0000000..788b41f
--- /dev/null
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/RunQueryCommandIT.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.client.command;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.api.entity.StreamsQuery;
+import org.apache.rya.streams.api.interactor.LoadStatements;
+import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.api.queries.QueryChangeLog;
+import org.apache.rya.streams.api.queries.QueryRepository;
+import org.apache.rya.streams.client.RyaStreamsCommand.ArgumentsException;
+import org.apache.rya.streams.client.RyaStreamsCommand.ExecutionException;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Integration tests the methods of {@link RunQueryCommand}.
+ */
+public class RunQueryCommandIT {
+
+    private final String ryaInstance = UUID.randomUUID().toString();
+
+    private QueryRepository queryRepo;
+    private Producer<String, VisibilityStatement> stmtProducer = null;
+    private Consumer<String, VisibilityBindingSet> resultConsumer = null;
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Before
+    public void setup() {
+        // Make sure the topic that the change log uses exists.
+        final String changeLogTopic = KafkaTopics.queryChangeLogTopic("" + ryaInstance);
+        kafka.createTopic(changeLogTopic);
+
+        // Setup the QueryRepository used by the test.
+        final Producer<?, QueryChange> queryProducer = KafkaTestUtil.makeProducer(kafka, StringSerializer.class, QueryChangeSerializer.class);
+        final Consumer<?, QueryChange>queryConsumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, QueryChangeDeserializer.class);
+        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, changeLogTopic);
+        queryRepo = new InMemoryQueryRepository(changeLog);
+
+        // Initialize the Statements Producer and the Results Consumer.
+        stmtProducer = KafkaTestUtil.makeProducer(kafka, StringSerializer.class, VisibilityStatementSerializer.class);
+        resultConsumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, VisibilityBindingSetDeserializer.class);
+    }
+
+    @After
+    public void cleanup() throws Exception{
+        stmtProducer.close();
+        resultConsumer.close();
+        queryRepo.close();
+    }
+
+    @Test(expected = ExecutionException.class)
+    public void runUnregisteredQuery() throws Exception {
+        // Arguments that run a query that is not registered with Rya Streams.
+        final String[] args = new String[] {
+                "--ryaInstance", "" + ryaInstance,
+                "--kafkaHostname", kafka.getKafkaHostname(),
+                "--kafkaPort", kafka.getKafkaPort(),
+                "--queryID", UUID.randomUUID().toString()
+        };
+
+        // Run the test. This will throw an exception.
+        final RunQueryCommand command = new RunQueryCommand();
+        command.execute(args);
+    }
+
+    @Test
+    public void runQuery() throws Exception {
+        // Register a query with the Query Repository.
+        final StreamsQuery sQuery = queryRepo.add("SELECT * WHERE { ?person <urn:worksAt> ?business . }");
+
+        // Arguments that run the query we just registered with Rya Streams.
+        final String[] args = new String[] {
+                "--ryaInstance", "" + ryaInstance,
+                "--kafkaHostname", kafka.getKafkaHostname(),
+                "--kafkaPort", kafka.getKafkaPort(),
+                "--queryID", sQuery.getQueryId().toString(),
+                "--zookeepers", kafka.getZookeeperServers()
+        };
+
+        // Create a new Thread that runs the command.
+        final Thread commandThread = new Thread() {
+            @Override
+            public void run() {
+                final RunQueryCommand command = new RunQueryCommand();
+                try {
+                    command.execute(args);
+                } catch (ArgumentsException | ExecutionException e) {
+                    // Do nothing. Test will still fail because the expected results will be missing.
+                }
+            }
+        };
+
+        // Create the statements that will be loaded.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(vf.createStatement(
+                vf.createURI("urn:Alice"),
+                vf.createURI("urn:worksAt"),
+                vf.createURI("urn:BurgerJoint")), "a"));
+        statements.add(new VisibilityStatement(vf.createStatement(
+                vf.createURI("urn:Bob"),
+                vf.createURI("urn:worksAt"),
+                vf.createURI("urn:TacoShop")), "a"));
+        statements.add(new VisibilityStatement(vf.createStatement(
+                vf.createURI("urn:Charlie"),
+                vf.createURI("urn:worksAt"),
+                vf.createURI("urn:TacoShop")), "a"));
+
+        // Create the expected results.
+        final List<VisibilityBindingSet> expected = new ArrayList<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add(new VisibilityBindingSet(bs, "a"));
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoShop"));
+        expected.add(new VisibilityBindingSet(bs, "a"));
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Charlie"));
+        bs.addBinding("business", vf.createURI("urn:TacoShop"));
+        expected.add(new VisibilityBindingSet(bs, "a"));
+
+        // Execute the test. This will result in a set of results that were read from the results topic.
+        final List<VisibilityBindingSet> results;
+        try {
+            // Wait for the program to start.
+            commandThread.start();
+            Thread.sleep(5000);
+
+            // Write some statements to the program.
+            final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+            final LoadStatements loadStatements = new KafkaLoadStatements(statementsTopic, stmtProducer);
+            loadStatements.fromCollection(statements);
+
+            // Read the output of the streams program.
+            final String resultsTopic = KafkaTopics.queryResultsTopic(sQuery.getQueryId());
+            resultConsumer.subscribe( Lists.newArrayList(resultsTopic) );
+            results = KafkaTestUtil.pollForResults(500, 6, 3, resultConsumer);
+        } finally {
+            // Tear down the test.
+            commandThread.interrupt();
+            commandThread.join(3000);
+        }
+
+        // Show the read results matched the expected ones.
+        assertEquals(expected, results);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/pom.xml b/extras/rya.streams/kafka/pom.xml
index 16a8b8e..0ccbb6e 100644
--- a/extras/rya.streams/kafka/pom.xml
+++ b/extras/rya.streams/kafka/pom.xml
@@ -55,6 +55,16 @@ under the License.
         <!-- Kafka dependencies -->
         <dependency>
             <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka_2.11</artifactId>
+            <exclusions>
+                <exclusion>
+                    <artifactId>slf4j-log4j12</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
             <artifactId>kafka-clients</artifactId>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
index a8fbf23..3e0df50 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
@@ -20,12 +20,19 @@ package org.apache.rya.streams.kafka;
 
 import static java.util.Objects.requireNonNull;
 
+import java.util.Properties;
+import java.util.Set;
 import java.util.UUID;
 
+import org.I0Itec.zkclient.ZkClient;
 import org.apache.rya.streams.api.queries.QueryChangeLog;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
+import kafka.admin.AdminUtils;
+import kafka.admin.RackAwareMode;
+import kafka.utils.ZKStringSerializer$;
+import kafka.utils.ZkUtils;
 
 /**
  * Creates the Kafka topic names that are used for Rya Streams systems.
@@ -66,4 +73,36 @@ public class KafkaTopics {
         requireNonNull(queryId);
         return "QueryResults-" + queryId.toString();
     }
+
+    /**
+     * Creates a set of Kafka topics for each topic that does not already exist.
+     *
+     * @param zookeeperServers - The Zookeeper servers that are used by the Kafka Streams program. (not null)
+     * @param topicNames - The topics that will be created. (not null)
+     * @param partitions - The number of partitions that each of the topics will have.
+     * @param replicationFactor - The replication factor of the topics that are created.
+     */
+    public static void createTopic(
+            final String zookeeperServers,
+            final Set<String> topicNames,
+            final int partitions,
+            final int replicationFactor) {
+        requireNonNull(zookeeperServers);
+        requireNonNull(topicNames);
+
+        ZkUtils zkUtils = null;
+        try {
+            zkUtils = ZkUtils.apply(new ZkClient(zookeeperServers, 30000, 30000, ZKStringSerializer$.MODULE$), false);
+            for(final String topicName : topicNames) {
+                if(!AdminUtils.topicExists(zkUtils, topicName)) {
+                    AdminUtils.createTopic(zkUtils, topicName, partitions, replicationFactor, new Properties(), RackAwareMode.Disabled$.MODULE$);
+                }
+            }
+        }
+        finally {
+            if(zkUtils != null) {
+                zkUtils.close();
+            }
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
index 8ab3ab6..d3ec650 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
@@ -64,7 +64,6 @@ public class KafkaLoadStatements implements LoadStatements {
         this.producer = requireNonNull(producer);
     }
 
-
     @Override
     public void fromFile(final Path statementsPath, final String visibilities) throws RyaStreamsException {
         requireNonNull(statementsPath);
@@ -77,7 +76,7 @@ public class KafkaLoadStatements implements LoadStatements {
         parser.setRDFHandler(new RDFHandlerBase() {
             @Override
             public void startRDF() throws RDFHandlerException {
-                log.trace("starting loading statements.");
+                log.trace("Starting loading statements.");
             }
 
             @Override
@@ -89,7 +88,7 @@ public class KafkaLoadStatements implements LoadStatements {
             @Override
             public void endRDF() throws RDFHandlerException {
                 producer.flush();
-                log.trace("done.");
+                log.trace("Done.");
             }
         });
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaRunQuery.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaRunQuery.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaRunQuery.java
new file mode 100644
index 0000000..e587998
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaRunQuery.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.interactor;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Optional;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.streams.api.entity.StreamsQuery;
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+import org.apache.rya.streams.api.interactor.RunQuery;
+import org.apache.rya.streams.api.queries.QueryRepository;
+import org.apache.rya.streams.kafka.topology.TopologyBuilderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A Kafka Streams implementation of {@link RunQuery}.
+ */
+@DefaultAnnotation(NonNull.class)
+public class KafkaRunQuery implements RunQuery {
+    private static final Logger log = LoggerFactory.getLogger(KafkaRunQuery.class);
+
+    private final String kafkaHostname;
+    private final String kafkaPort;
+    private final String statementsTopic;
+    private final String resultsTopic;
+    private final TopologyBuilderFactory topologyFactory;
+    private final QueryRepository queryRepo;
+
+    /**
+     * Constructs an instance of {@link KafkaRunQuery}.
+     *
+     * @param kafkaHostname - The hostname of the Kafka Broker to connect to. (not null)
+     * @param kafkaPort - The port of the Kafka Broker to connect to. (not null)
+     * @param statementsTopic - The name of the topic that statements will be read from. (not null)
+     * @param resultsTopic - The name of the topic that query results will be writen to. (not null)
+     * @param queryRepo - The query repository that holds queries that are registered. (not null)
+     * @param topologyFactory - Builds Kafka Stream processing topologies from SPARQL. (not null)
+     */
+    public KafkaRunQuery(
+            final String kafkaHostname,
+            final String kafkaPort,
+            final String statementsTopic,
+            final String resultsTopic,
+            final QueryRepository queryRepo,
+            final TopologyBuilderFactory topologyFactory) {
+        this.kafkaHostname = requireNonNull( kafkaHostname );
+        this.kafkaPort = requireNonNull( kafkaPort );
+        this.statementsTopic = requireNonNull(statementsTopic );
+        this.resultsTopic = requireNonNull( resultsTopic );
+        this.topologyFactory = requireNonNull( topologyFactory );
+        this.queryRepo = requireNonNull( queryRepo );
+    }
+
+    @Override
+    public void run(final UUID queryId) throws RyaStreamsException {
+        requireNonNull(queryId);
+
+        // Fetch the query from the repository. Throw an exception if it isn't present.
+        final Optional<StreamsQuery> query = queryRepo.get(queryId);
+        if(!query.isPresent()) {
+            throw new RyaStreamsException("Could not run the Query with ID " + queryId + " because no such query " +
+                    "is currently registered.");
+        }
+
+        // Build a processing topology using the SPARQL, provided statements topic, and provided results topic.
+        final String sparql = query.get().getSparql();
+        final TopologyBuilder topologyBuilder;
+        try {
+            topologyBuilder = topologyFactory.build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+        } catch (final Exception e) {
+            throw new RyaStreamsException("Could not run the Query with ID " + queryId + " because a processing " +
+                    "topolgoy could not be built for the SPARQL " + sparql, e);
+        }
+
+        // Setup the Kafka Stream program.
+        final Properties streamsProps = new Properties();
+        streamsProps.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaHostname + ":" + kafkaPort);
+        streamsProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "KafkaRunQuery-" + UUID.randomUUID());
+
+        final KafkaStreams streams = new KafkaStreams(topologyBuilder, new StreamsConfig(streamsProps));
+
+        // If an unhandled exception is thrown, rethrow it.
+        streams.setUncaughtExceptionHandler((t, e) -> {
+            // Log the problem and kill the program.
+            log.error("Unhandled exception while processing the Rya Streams query. Shutting down.", e);
+            System.exit(1);
+        });
+
+        // Setup a shutdown hook that kills the streams program at shutdown.
+        final CountDownLatch awaitTermination = new CountDownLatch(1);
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                awaitTermination.countDown();
+            }
+        });
+
+        // Run the streams program and wait for termination.
+        streams.start();
+        try {
+            awaitTermination.await();
+        } catch (final InterruptedException e) {
+            log.warn("Interrupted while waiting for termination. Shutting down.");
+        }
+        streams.close();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
index d73b40e..d12957a 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
@@ -34,6 +34,7 @@ import org.openrdf.query.impl.MapBindingSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
@@ -70,12 +71,12 @@ public class KeyValueJoinStateStore implements JoinStateStore {
     /**
      * This is the minimum value in UTF-8 character.
      */
-    private static final String START_RANGE_SUFFIX = new String(new byte[] { 0x00 } );
+    private static final String START_RANGE_SUFFIX = new String(new byte[] { 0x00 }, Charsets.UTF_8);
 
     /**
      * This is the maximum value of a UTF-8 character.
      */
-    private static final String END_RANGE_SUFFIX = new String(new byte[] { (byte) 0XFF } );
+    private static final String END_RANGE_SUFFIX = new String(new byte[] { (byte) 0XFF }, Charsets.UTF_8);
 
     /**
      * A default empty value that is stored for a start of range or end of range marker.


[03/50] [abbrv] incubator-rya git commit: RYA-377 Added missing license headers to test data.

Posted by ca...@apache.org.
RYA-377 Added missing license headers to test data.


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

Branch: refs/heads/master
Commit: a95fe5add788ccce805fb2fd4b7b09a3893779dc
Parents: 8399dfe
Author: kchilton2 <ke...@gmail.com>
Authored: Thu Oct 26 13:57:08 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:12:59 2018 -0500

----------------------------------------------------------------------
 .../kafka/interactor/KafkaLoadStatementsIT.java    |  3 +--
 .../kafka/src/test/resources/invalid.INVALID       | 17 +++++++++++++++++
 .../kafka/src/test/resources/statements.ttl        | 16 ++++++++++++++++
 3 files changed, 34 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a95fe5ad/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
index 8103e57..5a81d23 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
@@ -39,7 +39,6 @@ import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
 import org.apache.rya.test.kafka.KafkaITBase;
@@ -97,7 +96,7 @@ public class KafkaLoadStatementsIT extends KafkaITBase {
 
         try (final KafkaConsumer<String, VisibilityStatement> consumer = new KafkaConsumer<>(consumerProps)) {
             consumer.subscribe(Arrays.asList(rule.getKafkaTopicName()));
-            final ConsumerRecords<String, VisibilityStatement> records = consumer.poll(1000);
+            final ConsumerRecords<String, VisibilityStatement> records = consumer.poll(2000);
 
             assertEquals(3, records.count());
             final Iterator<ConsumerRecord<String, VisibilityStatement>> iter = records.iterator();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a95fe5ad/extras/rya.streams/kafka/src/test/resources/invalid.INVALID
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/resources/invalid.INVALID b/extras/rya.streams/kafka/src/test/resources/invalid.INVALID
index f71c8c6..52d18b4 100644
--- a/extras/rya.streams/kafka/src/test/resources/invalid.INVALID
+++ b/extras/rya.streams/kafka/src/test/resources/invalid.INVALID
@@ -1 +1,18 @@
+# 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.
+
 this file should not be parsed due to invalid extension
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a95fe5ad/extras/rya.streams/kafka/src/test/resources/statements.ttl
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/resources/statements.ttl b/extras/rya.streams/kafka/src/test/resources/statements.ttl
index 3456cc2..c19e22d 100644
--- a/extras/rya.streams/kafka/src/test/resources/statements.ttl
+++ b/extras/rya.streams/kafka/src/test/resources/statements.ttl
@@ -1,3 +1,19 @@
+# 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.
 @prefix example: <http://example#> .
 
 example:alice example:talksTo example:bob . 


[48/50] [abbrv] incubator-rya git commit: RYA-377 Fixed tests where the Temporal functions were not being referenced correctly.

Posted by ca...@apache.org.
RYA-377 Fixed tests where the Temporal functions were not being referenced correctly.


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

Branch: refs/heads/master
Commit: acd7654667444a64a833a6c4dec39b1c224f5550
Parents: 92c85ee
Author: kchilton2 <ke...@gmail.com>
Authored: Thu Jan 4 15:47:43 2018 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:02 2018 -0500

----------------------------------------------------------------------
 .../streams/kafka/processors/filter/TemporalFilterIT.java   | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/acd76546/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
index 22a883b..6e27669 100644
--- a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
@@ -30,6 +30,7 @@ import java.util.UUID;
 
 import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.function.temporal.TemporalURIs;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.kafka.KafkaTopics;
@@ -89,7 +90,7 @@ public class TemporalFilterIT {
         // Get the RDF model objects that will be used to build the query.
         final String sparql =
                 "PREFIX time: <http://www.w3.org/2006/time/> \n"
-                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "PREFIX tempf: <" + TemporalURIs.NAMESPACE + ">\n"
                         + "SELECT * \n"
                         + "WHERE { \n"
                         + "  <urn:time> time:atTime ?date .\n"
@@ -123,7 +124,7 @@ public class TemporalFilterIT {
         // Get the RDF model objects that will be used to build the query.
         final String sparql =
                 "PREFIX time: <http://www.w3.org/2006/time/> \n"
-                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "PREFIX tempf: <" + TemporalURIs.NAMESPACE + ">\n"
                         + "SELECT * \n"
                         + "WHERE { \n"
                         + "  <urn:time> time:atTime ?date .\n"
@@ -157,7 +158,7 @@ public class TemporalFilterIT {
         // Get the RDF model objects that will be used to build the query.
         final String sparql =
                 "PREFIX time: <http://www.w3.org/2006/time/> \n"
-                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "PREFIX tempf: <" + TemporalURIs.NAMESPACE + ">\n"
                         + "SELECT * \n"
                         + "WHERE { \n"
                         + "  <urn:time> time:atTime ?date .\n"
@@ -191,7 +192,7 @@ public class TemporalFilterIT {
         // Get the RDF model objects that will be used to build the query.
         final String sparql =
                 "PREFIX time: <http://www.w3.org/2006/time/> \n"
-                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "PREFIX tempf: <" + TemporalURIs.NAMESPACE + ">\n"
                         + "SELECT * \n"
                         + "WHERE { \n"
                         + "  <urn:time> time:atTime ?date .\n"


[47/50] [abbrv] incubator-rya git commit: RYA-377 Fixing flakey integration test.

Posted by ca...@apache.org.
RYA-377 Fixing flakey integration test.


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

Branch: refs/heads/master
Commit: f3655213cf4a9f2c36e5de31a2570fe0579415f5
Parents: acd7654
Author: kchilton2 <ke...@gmail.com>
Authored: Thu Jan 4 18:35:58 2018 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:02 2018 -0500

----------------------------------------------------------------------
 .../aggregation/AggregationProcessorIT.java     | 13 ---------
 .../src/test/resources/log4j.properties         | 29 ++++++++++++++++++++
 .../rya/streams/kafka/RyaStreamsTestUtil.java   |  5 +++-
 3 files changed, 33 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3655213/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
index 072469a..866fc2b 100644
--- a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
@@ -332,13 +332,6 @@ public class AggregationProcessorIT {
                 vf.createStatement(vf.createURI("urn:timecard3"), vf.createURI("urn:hours"), vf.createLiteral(28)), ""));
 
         statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard4")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard4")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:timecard4"), vf.createURI("urn:hours"), vf.createLiteral(28)), ""));
-
-        statements.add(new VisibilityStatement(
                 vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), ""));
         statements.add(new VisibilityStatement(
                 vf.createStatement(vf.createURI("urn:CoffeeShop"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard5")), ""));
@@ -368,12 +361,6 @@ public class AggregationProcessorIT {
         expected.add(new VisibilityBindingSet(bs, ""));
 
         bs = new MapBindingSet();
-        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("totalHours", vf.createLiteral("56", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
         bs.addBinding("business", vf.createURI("urn:CoffeeShop"));
         bs.addBinding("employee", vf.createURI("urn:Alice"));
         bs.addBinding("totalHours", vf.createLiteral("12", XMLSchema.INTEGER));

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3655213/extras/rya.streams/integration/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/resources/log4j.properties b/extras/rya.streams/integration/src/test/resources/log4j.properties
new file mode 100644
index 0000000..0bf0bcf
--- /dev/null
+++ b/extras/rya.streams/integration/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Root logger option
+log4j.rootLogger=INFO, stdout
+
+# Direct log messages to stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
+
+log4j.logger.org.apache.rya.streams.kafka.processors=debug

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3655213/extras/rya.streams/kafka-test/src/main/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka-test/src/main/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java b/extras/rya.streams/kafka-test/src/main/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
index ee25f8c..65ad840 100644
--- a/extras/rya.streams/kafka-test/src/main/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
+++ b/extras/rya.streams/kafka-test/src/main/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
@@ -25,8 +25,10 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
 import java.util.Set;
+import java.util.UUID;
 
 import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.StringDeserializer;
@@ -88,7 +90,8 @@ public class RyaStreamsTestUtil {
 
         // Start the streams program.
         final Properties props = kafka.createBootstrapServerConfig();
-        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "StatementPatternProcessorIT");
+        props.put(StreamsConfig.APPLICATION_ID_CONFIG, UUID.randomUUID().toString());
+        props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
 
         final KafkaStreams streams = new KafkaStreams(builder, new StreamsConfig(props));
         streams.cleanUp();


[20/50] [abbrv] incubator-rya git commit: RYA-377 Added executable jar for CLI tool

Posted by ca...@apache.org.
RYA-377 Added executable jar for CLI tool


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

Branch: refs/heads/master
Commit: 6056528b23a4d1762a078a5915c4607606840996
Parents: 7deb0c0
Author: Andrew Smith <sm...@gmail.com>
Authored: Tue Oct 31 15:49:04 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 extras/rya.streams/client/pom.xml | 27 +++++++++++++++++++++++++++
 1 file changed, 27 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/6056528b/extras/rya.streams/client/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/pom.xml b/extras/rya.streams/client/pom.xml
index 0dc640b..2e49952 100644
--- a/extras/rya.streams/client/pom.xml
+++ b/extras/rya.streams/client/pom.xml
@@ -75,4 +75,31 @@ under the License.
             <scope>test</scope>
         </dependency>
     </dependencies>
+    
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <descriptorRefs>
+                        <descriptorRef>jar-with-dependencies</descriptorRef>
+                    </descriptorRefs>
+                    <archive>
+                        <manifest>
+                            <mainClass>org.apache.rya.streams.client.CLIDriver</mainClass>
+                        </manifest>
+                    </archive>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+     </build>
 </project>


[35/50] [abbrv] incubator-rya git commit: RYA-377 Fixed a bug that made kafka integration tests fail.

Posted by ca...@apache.org.
RYA-377 Fixed a bug that made kafka integration tests fail.


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

Branch: refs/heads/master
Commit: 5603a10611de6dc8dde5b3f0b230d1ad31422074
Parents: 2957c75
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Dec 5 17:39:55 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java      | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5603a106/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java
----------------------------------------------------------------------
diff --git a/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java b/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java
index 4b41f1a..fa1e0a0 100644
--- a/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java
+++ b/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java
@@ -87,6 +87,7 @@ public final class KafkaTestUtil {
 
         final Properties props = kafka.createBootstrapServerConfig();
         props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
+        props.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, UUID.randomUUID().toString());
         props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
         props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass.getName());
         props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass.getName());


[09/50] [abbrv] incubator-rya git commit: RYA-377 Adding a Kafka Streams Processor that outputs VisibilityStatements from VisibilityBindingSets.

Posted by ca...@apache.org.
RYA-377 Adding a Kafka Streams Processor that outputs VisibilityStatements from VisibilityBindingSets.


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

Branch: refs/heads/master
Commit: b8b0a1269d7b2ae195dab01d4305b36c258bd2e4
Parents: 98af7aa
Author: kchilton2 <ke...@gmail.com>
Authored: Wed Nov 15 18:18:08 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 extras/rya.streams/kafka/pom.xml                |   5 +
 .../RyaStreamsSinkFormatterSupplier.java        |  83 --------
 .../BindingSetOutputFormatterSupplier.java      |  84 ++++++++
 .../StatementOutputFormatterSupplier.java       | 114 +++++++++++
 .../kafka/processors/ProjectionProcessorIT.java |   6 +-
 .../processors/StatementPatternProcessorIT.java |  10 +-
 .../kafka/processors/join/JoinProcessorIT.java  |  12 +-
 .../output/BindingSetOutputFormatterTest.java   |  84 ++++++++
 .../output/StatementOutputFormatterTest.java    | 203 +++++++++++++++++++
 9 files changed, 504 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/b8b0a126/extras/rya.streams/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/pom.xml b/extras/rya.streams/kafka/pom.xml
index d3f6891..16a8b8e 100644
--- a/extras/rya.streams/kafka/pom.xml
+++ b/extras/rya.streams/kafka/pom.xml
@@ -79,5 +79,10 @@ under the License.
             <artifactId>rya.test.kafka</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/b8b0a126/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsSinkFormatterSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsSinkFormatterSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsSinkFormatterSupplier.java
deleted file mode 100644
index d6a8d2d..0000000
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsSinkFormatterSupplier.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors;
-
-import org.apache.kafka.streams.processor.Processor;
-import org.apache.kafka.streams.processor.ProcessorContext;
-import org.apache.kafka.streams.processor.ProcessorSupplier;
-import org.apache.rya.api.model.VisibilityBindingSet;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-
-/**
- * Supplies {@link RyaStreamsSinkFormatter} instances.
- */
-@DefaultAnnotation(NonNull.class)
-public class RyaStreamsSinkFormatterSupplier implements ProcessorSupplier<Object, ProcessorResult> {
-
-    @Override
-    public Processor<Object, ProcessorResult> get() {
-        return new RyaStreamsSinkFormatter();
-    }
-
-    /**
-     * Accepts {@link ProcessorResult}s and forwards just their {@link VisibilityBindingSet} so that it may be
-     * written to a sink.
-     */
-    @DefaultAnnotation(NonNull.class)
-    public static final class RyaStreamsSinkFormatter implements Processor<Object, ProcessorResult> {
-
-        private ProcessorContext processorContext;
-
-        @Override
-        public void init(final ProcessorContext context) {
-            processorContext = context;
-        }
-
-        @Override
-        public void process(final Object key, final ProcessorResult value) {
-
-            VisibilityBindingSet result = null;
-            switch(value.getType()) {
-                case UNARY:
-                    result = value.getUnary().getResult();
-                    break;
-
-                case BINARY:
-                    result = value.getBinary().getResult();
-                    break;
-            }
-
-            if(result != null) {
-                processorContext.forward(key, result);
-            }
-        }
-
-        @Override
-        public void punctuate(final long timestamp) {
-            // Does nothing.
-        }
-
-        @Override
-        public void close() {
-            // Does nothing.
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/b8b0a126/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/output/BindingSetOutputFormatterSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/output/BindingSetOutputFormatterSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/output/BindingSetOutputFormatterSupplier.java
new file mode 100644
index 0000000..89f5c6c
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/output/BindingSetOutputFormatterSupplier.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.output;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.ProcessorSupplier;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Supplies {@link BindingSetOutputFormatter} instances.
+ */
+@DefaultAnnotation(NonNull.class)
+public class BindingSetOutputFormatterSupplier implements ProcessorSupplier<Object, ProcessorResult> {
+
+    @Override
+    public Processor<Object, ProcessorResult> get() {
+        return new BindingSetOutputFormatter();
+    }
+
+    /**
+     * Accepts {@link ProcessorResult}s and forwards just their {@link VisibilityBindingSet} so that it may be
+     * written to a sink.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static final class BindingSetOutputFormatter implements Processor<Object, ProcessorResult> {
+
+        private ProcessorContext processorContext;
+
+        @Override
+        public void init(final ProcessorContext context) {
+            processorContext = context;
+        }
+
+        @Override
+        public void process(final Object key, final ProcessorResult value) {
+
+            VisibilityBindingSet result = null;
+            switch(value.getType()) {
+                case UNARY:
+                    result = value.getUnary().getResult();
+                    break;
+
+                case BINARY:
+                    result = value.getBinary().getResult();
+                    break;
+            }
+
+            if(result != null) {
+                processorContext.forward(key, result);
+            }
+        }
+
+        @Override
+        public void punctuate(final long timestamp) {
+            // Does nothing.
+        }
+
+        @Override
+        public void close() {
+            // Does nothing.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/b8b0a126/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/output/StatementOutputFormatterSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/output/StatementOutputFormatterSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/output/StatementOutputFormatterSupplier.java
new file mode 100644
index 0000000..74a6f96
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/output/StatementOutputFormatterSupplier.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.output;
+
+import java.util.Collection;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.ProcessorSupplier;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+import com.google.common.collect.Sets;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Supplies instance of {@link StatementOutputFormatter}
+ */
+@DefaultAnnotation(NonNull.class)
+public class StatementOutputFormatterSupplier implements ProcessorSupplier<Object, ProcessorResult> {
+
+    @Override
+    public Processor<Object, ProcessorResult> get() {
+        return new StatementOutputFormatter();
+    }
+
+    /**
+     * Converts {@link VisiblityBindingSet}s that contain a "subject", "predicate", and "object" binding into a
+     * {@link VisibilityStatement} and then forwards that to the downstream processor.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static final class StatementOutputFormatter implements Processor<Object, ProcessorResult> {
+
+        private static final ValueFactory VF = new ValueFactoryImpl();
+        private static final Collection<String> REQURIED_BINDINGS = Sets.newHashSet("subject", "predicate", "object");
+
+        private ProcessorContext processorContext;
+
+        @Override
+        public void init(final ProcessorContext context) {
+            processorContext = context;
+        }
+
+        @Override
+        public void process(final Object key, final ProcessorResult value) {
+            VisibilityBindingSet result = null;
+            switch(value.getType()) {
+                case UNARY:
+                    result = value.getUnary().getResult();
+                    break;
+
+                case BINARY:
+                    result = value.getBinary().getResult();
+                    break;
+            }
+
+            if(result != null && result.getBindingNames().containsAll(REQURIED_BINDINGS)) {
+                // Make sure the Subject is the correct type.
+                final Value subjVal = result.getValue("subject");
+                if(!(subjVal instanceof Resource)) {
+                    return;
+                }
+
+                // Make sure the Predicate is the correct type.
+                final Value predVal = result.getValue("predicate");
+                if(!(predVal instanceof URI)) {
+                    return;
+                }
+
+                // Forward the visibility statement.
+                final Statement statement = VF.createStatement(
+                        (Resource) subjVal,
+                        (URI) predVal,
+                        result.getValue("object"));
+                processorContext.forward(key, new VisibilityStatement(statement, result.getVisibility()));
+            }
+        }
+
+        @Override
+        public void punctuate(final long timestamp) {
+            // Do nothing.
+        }
+
+        @Override
+        public void close() {
+            // Do nothing.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/b8b0a126/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
index 53d1765..f58387e 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
@@ -33,8 +33,8 @@ import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.RdfTestUtil;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
-import org.apache.rya.streams.kafka.processors.RyaStreamsSinkFormatterSupplier.RyaStreamsSinkFormatter;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
+import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
 import org.apache.rya.streams.kafka.processors.projection.ProjectionProcessorSupplier;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
@@ -85,7 +85,7 @@ public class ProjectionProcessorIT {
         builder.addProcessor("P1", new ProjectionProcessorSupplier(elems, result -> ProcessorResult.make(new UnaryResult(result))), "SP1");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "P1");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "P1");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
@@ -131,7 +131,7 @@ public class ProjectionProcessorIT {
         builder.addProcessor("P1", new ProjectionProcessorSupplier(elems, result -> ProcessorResult.make(new UnaryResult(result))), "SP1");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "P1");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "P1");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/b8b0a126/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
index 371fd0b..0b2ff60 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
@@ -33,8 +33,8 @@ import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.RdfTestUtil;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
-import org.apache.rya.streams.kafka.processors.RyaStreamsSinkFormatterSupplier.RyaStreamsSinkFormatter;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
+import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
@@ -74,7 +74,7 @@ public class StatementPatternProcessorIT {
         builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "SP1");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "SP1");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
@@ -117,7 +117,7 @@ public class StatementPatternProcessorIT {
         builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "SP1");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "SP1");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
@@ -172,7 +172,7 @@ public class StatementPatternProcessorIT {
         builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "SP1", "SP2");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "SP1", "SP2");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
@@ -224,7 +224,7 @@ public class StatementPatternProcessorIT {
         builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "SP1", "SP2");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "SP1", "SP2");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/b8b0a126/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
index 14a559f..7051efa 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
@@ -40,9 +40,9 @@ import org.apache.rya.streams.kafka.processors.ProcessorResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
-import org.apache.rya.streams.kafka.processors.RyaStreamsSinkFormatterSupplier.RyaStreamsSinkFormatter;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier.JoinProcessor;
+import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerde;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
@@ -118,7 +118,7 @@ public class JoinProcessorIT {
         builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "NATURAL_JOIN");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "NATURAL_JOIN");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
@@ -201,7 +201,7 @@ public class JoinProcessorIT {
         builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "NATURAL_JOIN");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "NATURAL_JOIN");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
@@ -284,7 +284,7 @@ public class JoinProcessorIT {
         builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "NATURAL_JOIN");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "NATURAL_JOIN");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
@@ -394,7 +394,7 @@ public class JoinProcessorIT {
         builder.addStateStore(join2StoreSupplier, "JOIN2");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "JOIN2");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "JOIN2");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
@@ -466,7 +466,7 @@ public class JoinProcessorIT {
         builder.addStateStore(joinStoreSupplier, "LEFT_JOIN");
 
         // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "LEFT_JOIN");
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "LEFT_JOIN");
 
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/b8b0a126/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/output/BindingSetOutputFormatterTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/output/BindingSetOutputFormatterTest.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/output/BindingSetOutputFormatterTest.java
new file mode 100644
index 0000000..3c810b1
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/output/BindingSetOutputFormatterTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.output;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Unit tests the methods of {@link BindingSetOutputFormatter}.
+ */
+public class BindingSetOutputFormatterTest {
+
+    @Test
+    public void unaryResult() {
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bindingSet = new MapBindingSet();
+        bindingSet.addBinding("person", vf.createURI("urn:Alice"));
+        bindingSet.addBinding("age", vf.createLiteral(34));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bindingSet, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final BindingSetOutputFormatter formatter = new BindingSetOutputFormatter();
+        formatter.init(context);
+        formatter.process("key", ProcessorResult.make(new UnaryResult(visBs)));
+
+        // Verify the mock was invoked with the expected output.
+        verify(context, times(1)).forward(eq("key"), eq(visBs));
+    }
+
+    @Test
+    public void binaryResult() {
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bindingSet = new MapBindingSet();
+        bindingSet.addBinding("person", vf.createURI("urn:Alice"));
+        bindingSet.addBinding("age", vf.createLiteral(34));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bindingSet, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final BindingSetOutputFormatter formatter = new BindingSetOutputFormatter();
+        formatter.init(context);
+        formatter.process("key", ProcessorResult.make(new BinaryResult(Side.LEFT, visBs)));
+
+        // Verify the mock was invoked with the expected output.
+        verify(context, times(1)).forward(eq("key"), eq(visBs));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/b8b0a126/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/output/StatementOutputFormatterTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/output/StatementOutputFormatterTest.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/output/StatementOutputFormatterTest.java
new file mode 100644
index 0000000..5f7e9a6
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/output/StatementOutputFormatterTest.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.output;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.output.StatementOutputFormatterSupplier.StatementOutputFormatter;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Unit tests the methods of {@link StatementOutputFormatter}.
+ */
+public class StatementOutputFormatterTest {
+
+    @Test
+    public void unaryResult() {
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bindingSet = new MapBindingSet();
+        bindingSet.addBinding("subject", vf.createURI("urn:Alice"));
+        bindingSet.addBinding("predicate", vf.createURI("urn:age"));
+        bindingSet.addBinding("object", vf.createLiteral(34));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bindingSet, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final StatementOutputFormatter formatter = new StatementOutputFormatter();
+        formatter.init(context);
+        formatter.process("key", ProcessorResult.make(new UnaryResult(visBs)));
+
+        // Verify the mock was invoked with the expected output.
+        final VisibilityStatement expectedStmt = new VisibilityStatement(vf.createStatement(
+                vf.createURI("urn:Alice"),
+                vf.createURI("urn:age"),
+                vf.createLiteral(34)), "a");
+        verify(context, times(1)).forward(eq("key"), eq(expectedStmt));
+    }
+
+    @Test
+    public void binaryResult() {
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bindingSet = new MapBindingSet();
+        bindingSet.addBinding("subject", vf.createURI("urn:Alice"));
+        bindingSet.addBinding("predicate", vf.createURI("urn:age"));
+        bindingSet.addBinding("object", vf.createLiteral(34));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bindingSet, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final StatementOutputFormatter formatter = new StatementOutputFormatter();
+        formatter.init(context);
+        formatter.process("key", ProcessorResult.make(new BinaryResult(Side.LEFT, visBs)));
+
+        // Verify the mock was invoked with the expected output.
+        final VisibilityStatement expectedStmt = new VisibilityStatement(vf.createStatement(
+                vf.createURI("urn:Alice"),
+                vf.createURI("urn:age"),
+                vf.createLiteral(34)), "a");
+        verify(context, times(1)).forward(eq("key"), eq(expectedStmt));
+    }
+
+    @Test
+    public void missingSubject() {
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bindingSet = new MapBindingSet();
+        bindingSet.addBinding("predicate", vf.createURI("urn:age"));
+        bindingSet.addBinding("object", vf.createLiteral(34));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bindingSet, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final StatementOutputFormatter formatter = new StatementOutputFormatter();
+        formatter.init(context);
+        formatter.process("key", ProcessorResult.make(new UnaryResult(visBs)));
+
+        // Verify the mock was never invoked.
+        verify(context, times(0)).forward(any(), any());
+    }
+
+    @Test
+    public void subjectWrongType() {
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bindingSet = new MapBindingSet();
+        bindingSet.addBinding("subject", vf.createLiteral("Alice"));
+        bindingSet.addBinding("predicate", vf.createURI("urn:age"));
+        bindingSet.addBinding("object", vf.createLiteral(34));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bindingSet, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final StatementOutputFormatter formatter = new StatementOutputFormatter();
+        formatter.init(context);
+        formatter.process("key", ProcessorResult.make(new UnaryResult(visBs)));
+
+        // Verify the mock was never invoked.
+        verify(context, times(0)).forward(any(), any());
+    }
+
+    @Test
+    public void missingPredicate() {
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bindingSet = new MapBindingSet();
+        bindingSet.addBinding("subject", vf.createURI("urn:Alice"));
+        bindingSet.addBinding("object", vf.createLiteral(34));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bindingSet, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final StatementOutputFormatter formatter = new StatementOutputFormatter();
+        formatter.init(context);
+        formatter.process("key", ProcessorResult.make(new UnaryResult(visBs)));
+
+        // Verify the mock was never invoked.
+        verify(context, times(0)).forward(any(), any());
+    }
+
+    @Test
+    public void predicateWrongType() {
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bindingSet = new MapBindingSet();
+        bindingSet.addBinding("subject", vf.createURI("urn:Alice"));
+        bindingSet.addBinding("predicate", vf.createLiteral("age"));
+        bindingSet.addBinding("object", vf.createLiteral(34));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bindingSet, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final StatementOutputFormatter formatter = new StatementOutputFormatter();
+        formatter.init(context);
+        formatter.process("key", ProcessorResult.make(new UnaryResult(visBs)));
+
+        // Verify the mock was never invoked.
+        verify(context, times(0)).forward(any(), any());
+    }
+
+    @Test
+    public void missingObject() {
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bindingSet = new MapBindingSet();
+        bindingSet.addBinding("subject", vf.createURI("urn:Alice"));
+        bindingSet.addBinding("predicate", vf.createURI("urn:age"));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bindingSet, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final StatementOutputFormatter formatter = new StatementOutputFormatter();
+        formatter.init(context);
+        formatter.process("key", ProcessorResult.make(new UnaryResult(visBs)));
+
+        // Verify the mock was never invoked.
+        verify(context, times(0)).forward(any(), any());
+    }
+}
\ No newline at end of file



[50/50] [abbrv] incubator-rya git commit: RYA-377 Ignoring flakey test. Closes #377. Closes #427.

Posted by ca...@apache.org.
RYA-377 Ignoring flakey test. Closes #377. Closes #427.


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

Branch: refs/heads/master
Commit: 9b8162ab7b729dbff574dba0b7fe63298477712c
Parents: f0725df
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Jan 9 15:07:30 2018 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 16:47:16 2018 -0500

----------------------------------------------------------------------
 .../kafka/processors/aggregation/AggregationProcessorIT.java      | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9b8162ab/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
index 866fc2b..a4e4a3e 100644
--- a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
@@ -34,6 +34,7 @@ import org.apache.rya.streams.kafka.processors.aggregation.AggregationProcessorS
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.topology.TopologyFactory;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;
@@ -379,7 +380,9 @@ public class AggregationProcessorIT {
         RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
+    // Ignored because this test is kind of flakey.
     @Test
+    @Ignore
     public void multipleAggregations() throws Exception {
         // A query that figures out what the youngest and oldest ages are across all people.
         final String sparql =


[19/50] [abbrv] incubator-rya git commit: RYA-377 Implemented integration tests for the client commands.

Posted by ca...@apache.org.
RYA-377 Implemented integration tests for the client commands.


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

Branch: refs/heads/master
Commit: 7deb0c0026feb71fa33b1c4ea95bf46e4ff3f112
Parents: 27255cc
Author: kchilton2 <ke...@gmail.com>
Authored: Fri Oct 27 20:43:38 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../interactor/defaults/DefaultListQueries.java |   7 +-
 .../rya/streams/client/RyaStreamsCommand.java   |  29 ++-
 .../streams/client/command/AddQueryCommand.java |  17 +-
 .../client/command/DeleteQueryCommand.java      |  19 +-
 .../client/command/ListQueriesCommand.java      |  17 +-
 .../client/command/LoadStatementsCommand.java   |  12 +-
 .../client/command/AddQueryCommandIT.java       | 103 ++++++++++-
 .../client/command/DeleteQueryCommandIT.java    | 153 +++++++++++++---
 .../client/command/ListQueryCommandIT.java      |  97 ++++++++--
 .../client/command/LoadStatementsCommandIT.java | 181 +++++++++++++++++++
 .../client/src/test/resources/statements.ttl    |  21 +++
 .../apache/rya/streams/kafka/KafkaTopics.java   |  52 ++++++
 12 files changed, 619 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultListQueries.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultListQueries.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultListQueries.java
index 82ca691..946944f 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultListQueries.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultListQueries.java
@@ -38,10 +38,9 @@ public class DefaultListQueries implements ListQueries {
     private final QueryRepository repository;
 
     /**
-     * Creates a new {@link DefaultAddQuery}.
+     * Creates a new {@link DefaultListQueries}.
      *
-     * @param repository - The {@link QueryRepository} to add a query to. (not
-     *        null)
+     * @param repository - The {@link QueryRepository} that hosts the listed queries. (not null)
      */
     public DefaultListQueries(final QueryRepository repository) {
         this.repository = requireNonNull(repository);
@@ -51,4 +50,4 @@ public class DefaultListQueries implements ListQueries {
     public Set<StreamsQuery> all() throws RyaStreamsException {
         return repository.list();
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
index 967b79e..5d64785 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
@@ -26,23 +26,22 @@ import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
 
 /**
- * A command that may be executed by the {@link PcjAdminClient}.
+ * A command that may be executed by the Rya Streams {@link CLIDriver}.
  */
 @DefaultAnnotation(NonNull.class)
 public interface RyaStreamsCommand {
+
     /**
-     * Command line parameters that are used by this command to configure
-     * itself.
+     * Command line parameters that are used by all commands that interact with Kafka.
      */
-    class Parameters {
-        @Parameter(names = { "--topic",
-        "-t" }, required = true, description = "The kafka topic to load the statements into.")
-        public String topicName;
-        @Parameter(names = { "--kafkaPort",
-        "-p" }, required = true, description = "The port to use to connect to Kafka.")
+    class KafkaParameters {
+        @Parameter(names = {"--ryaInstance", "-r"}, required = true, description = "The name of the Rya Instance the Rya Streams is a part of.")
+        public String ryaInstance;
+
+        @Parameter(names = { "--kafkaPort", "-p" }, required = true, description = "The port to use to connect to Kafka.")
         public String kafkaPort;
-        @Parameter(names = { "--kafkaHostname",
-        "-i" }, required = true, description = "The IP or Hostname to use to connect to Kafka.")
+
+        @Parameter(names = { "--kafkaHostname", "-i" }, required = true, description = "The IP or Hostname to use to connect to Kafka.")
         public String kafkaIP;
 
         @Override
@@ -51,8 +50,8 @@ public interface RyaStreamsCommand {
             parameters.append("Parameters");
             parameters.append("\n");
 
-            if (!Strings.isNullOrEmpty(topicName)) {
-                parameters.append("\tTopic: " + topicName);
+            if(!Strings.isNullOrEmpty(ryaInstance)) {
+                parameters.append("\tRya Instance Name: " + ryaInstance + "\n");
             }
 
             if (!Strings.isNullOrEmpty(kafkaIP)) {
@@ -82,7 +81,7 @@ public interface RyaStreamsCommand {
      * @return Describes what arguments may be provided to the command.
      */
     default public String getUsage() {
-        final JCommander parser = new JCommander(new Parameters());
+        final JCommander parser = new JCommander(new KafkaParameters());
 
         final StringBuilder usage = new StringBuilder();
         parser.usage(usage);
@@ -127,4 +126,4 @@ public interface RyaStreamsCommand {
             super(message, cause);
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java
index dfaa6c6..8439f20 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java
@@ -40,6 +40,7 @@ import org.apache.rya.streams.api.queries.QueryChange;
 import org.apache.rya.streams.api.queries.QueryChangeLog;
 import org.apache.rya.streams.api.queries.QueryRepository;
 import org.apache.rya.streams.client.RyaStreamsCommand;
+import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
@@ -62,12 +63,10 @@ public class AddQueryCommand implements RyaStreamsCommand {
     private static final Logger log = LoggerFactory.getLogger(AddQueryCommand.class);
 
     /**
-     * Command line parameters that are used by this command to configure
-     * itself.
+     * Command line parameters that are used by this command to configure itself.
      */
-    private class AddParameters extends RyaStreamsCommand.Parameters {
-        @Parameter(names = { "--query",
-        "-q" }, required = true, description = "The SPARQL query to add to Rya Streams.")
+    private class AddParameters extends RyaStreamsCommand.KafkaParameters {
+        @Parameter(names = { "--query", "-q" }, required = true, description = "The SPARQL query to add to Rya Streams.")
         private String query;
 
         @Override
@@ -121,15 +120,19 @@ public class AddQueryCommand implements RyaStreamsCommand {
         producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
         producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
         producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
+
         final Properties consumerProperties = new Properties();
         consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
         consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
         consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
+
         final Producer<?, QueryChange> queryProducer = new KafkaProducer<>(producerProperties);
         final Consumer<?, QueryChange> queryConsumer = new KafkaConsumer<>(consumerProperties);
 
-        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, params.topicName);
+        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, KafkaTopics.queryChangeLogTopic(params.ryaInstance));
         final QueryRepository repo = new InMemoryQueryRepository(changeLog);
+
+        // Execute the add query command.
         final AddQuery addQuery = new DefaultAddQuery(repo);
         try {
             final StreamsQuery query = addQuery.addQuery(params.query);
@@ -140,4 +143,4 @@ public class AddQueryCommand implements RyaStreamsCommand {
 
         log.trace("Finished executing the Add Query Command.");
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java
index 65a7017..b101a0f 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java
@@ -40,6 +40,7 @@ import org.apache.rya.streams.api.queries.QueryChange;
 import org.apache.rya.streams.api.queries.QueryChangeLog;
 import org.apache.rya.streams.api.queries.QueryRepository;
 import org.apache.rya.streams.client.RyaStreamsCommand;
+import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
@@ -62,10 +63,9 @@ public class DeleteQueryCommand implements RyaStreamsCommand {
     private static final Logger log = LoggerFactory.getLogger(DeleteQueryCommand.class);
 
     /**
-     * Command line parameters that are used by this command to configure
-     * itself.
+     * Command line parameters that are used by this command to configure itself.
      */
-    private class RemoveParameters extends RyaStreamsCommand.Parameters {
+    private class RemoveParameters extends RyaStreamsCommand.KafkaParameters {
         @Parameter(names = { "--queryID", "-q" }, required = true, description = "The ID of the query to remove from Rya Streams.")
         private String queryId;
 
@@ -73,7 +73,6 @@ public class DeleteQueryCommand implements RyaStreamsCommand {
         public String toString() {
             final StringBuilder parameters = new StringBuilder();
             parameters.append(super.toString());
-            parameters.append("\n");
 
             if (!Strings.isNullOrEmpty(queryId)) {
                 parameters.append("\tQueryID: " + queryId);
@@ -120,23 +119,27 @@ public class DeleteQueryCommand implements RyaStreamsCommand {
         producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
         producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
         producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
+
         final Properties consumerProperties = new Properties();
         consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
         consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
         consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
+
         final Producer<?, QueryChange> queryProducer = new KafkaProducer<>(producerProperties);
         final Consumer<?, QueryChange> queryConsumer = new KafkaConsumer<>(consumerProperties);
 
-        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, params.topicName);
+        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, KafkaTopics.queryChangeLogTopic(params.ryaInstance));
         final QueryRepository repo = new InMemoryQueryRepository(changeLog);
+
+        // Execute the delete query command.
         final DeleteQuery deleteQuery = new DefaultDeleteQuery(repo);
         try {
             deleteQuery.delete(UUID.fromString(params.queryId));
             log.trace("Deleted query: " + params.queryId);
         } catch (final RyaStreamsException e) {
-            log.error("Unable to parse query: " + params.queryId, e);
+            log.error("Unable to delete query with ID: " + params.queryId, e);
         }
 
-        log.trace("Finished executing the Add Query Command.");
+        log.trace("Finished executing the Delete Query Command.");
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
index ec40b50..c4e5de6 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
@@ -41,6 +41,7 @@ import org.apache.rya.streams.api.queries.QueryChange;
 import org.apache.rya.streams.api.queries.QueryChangeLog;
 import org.apache.rya.streams.api.queries.QueryRepository;
 import org.apache.rya.streams.client.RyaStreamsCommand;
+import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
@@ -75,7 +76,7 @@ public class ListQueriesCommand implements RyaStreamsCommand {
         requireNonNull(args);
 
         // Parse the command line arguments.
-        final Parameters params = new Parameters();
+        final KafkaParameters params = new KafkaParameters();
         try {
             new JCommander(params, args);
         } catch (final ParameterException e) {
@@ -88,25 +89,29 @@ public class ListQueriesCommand implements RyaStreamsCommand {
         producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
         producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
         producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
+
         final Properties consumerProperties = new Properties();
         consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
         consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
         consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
+
         final Producer<?, QueryChange> queryProducer = new KafkaProducer<>(producerProperties);
         final Consumer<?, QueryChange> queryConsumer = new KafkaConsumer<>(consumerProperties);
 
-        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, params.topicName);
+        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, KafkaTopics.queryChangeLogTopic(params.ryaInstance));
         final QueryRepository repo = new InMemoryQueryRepository(changeLog);
+
+        // Execute the list queries command.
         final ListQueries listQueries = new DefaultListQueries(repo);
         try {
             final Set<StreamsQuery> queries = listQueries.all();
-            logQueries(queries);
+            System.out.println( formatQueries(queries) );
         } catch (final RyaStreamsException e) {
             log.error("Unable to retrieve the queries.", e);
         }
     }
 
-    private void logQueries(final Set<StreamsQuery> queries) {
+    private String formatQueries(final Set<StreamsQuery> queries) {
         final StringBuilder sb = new StringBuilder();
         sb.append("\n");
         sb.append("Queries in Rya Streams:\n");
@@ -119,6 +124,6 @@ public class ListQueriesCommand implements RyaStreamsCommand {
             sb.append(query.getSparql());
             sb.append("\n");
         });
-        log.trace(sb.toString());
+        return sb.toString();
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
index 057de77..4763bd8 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
@@ -27,10 +27,13 @@ import java.util.Properties;
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.api.interactor.LoadStatements;
 import org.apache.rya.streams.client.RyaStreamsCommand;
+import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,9 +56,11 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
     /**
      * Command line parameters that are used by this command to configure itself.
      */
-    private static final class LoadStatementsParameters extends RyaStreamsCommand.Parameters {
+    private static final class LoadStatementsParameters extends RyaStreamsCommand.KafkaParameters {
+
         @Parameter(names = {"--statementsFile", "-f"}, required = true, description = "The file of RDF statements to load into Rya Streams.")
         private String statementsFile;
+
         @Parameter(names= {"--visibilities", "-v"}, required = true, description = "The visibilities to assign to the statements being loaded in.")
         private String visibilities;
 
@@ -63,7 +68,6 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
         public String toString() {
             final StringBuilder parameters = new StringBuilder();
             parameters.append(super.toString());
-            parameters.append("\n");
 
             if (!Strings.isNullOrEmpty(statementsFile)) {
                 parameters.append("\tStatements File: " + statementsFile);
@@ -117,7 +121,7 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
 
         final Properties producerProps = buildProperties(params);
         try (final Producer<Object, VisibilityStatement> producer = new KafkaProducer<>(producerProps)) {
-            final LoadStatements statements = new KafkaLoadStatements(params.topicName, producer);
+            final LoadStatements statements = new KafkaLoadStatements(KafkaTopics.statementsTopic(params.ryaInstance), producer);
             statements.load(statementsPath, params.visibilities);
         } catch (final Exception e) {
             log.error("Unable to parse statements file: " + statementsPath.toString(), e);
@@ -130,6 +134,8 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
         requireNonNull(params);
         final Properties props = new Properties();
         props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
+        props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+        props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VisibilityStatementSerializer.class.getName());
         return props;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
index 6b13b46..09e874c 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
@@ -21,10 +21,30 @@ package org.apache.rya.streams.client.command;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Properties;
+import java.util.Set;
+import java.util.UUID;
 
 import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.rya.streams.api.entity.StreamsQuery;
+import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.api.queries.QueryChangeLog;
+import org.apache.rya.streams.api.queries.QueryRepository;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
 import org.apache.rya.test.kafka.KafkaITBase;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -33,7 +53,15 @@ import org.junit.Test;
  * integration Test for adding a new query through a command.
  */
 public class AddQueryCommandIT extends KafkaITBase {
-    private String[] args;
+
+    private final String ryaInstance = UUID.randomUUID().toString();
+
+    private String kafkaIp;
+    private String kafkaPort;
+    private QueryRepository queryRepo;
+
+    private Producer<?, QueryChange> queryProducer = null;
+    private Consumer<?, QueryChange> queryConsumer = null;
 
     @Rule
     public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
@@ -43,19 +71,74 @@ public class AddQueryCommandIT extends KafkaITBase {
         final Properties props = rule.createBootstrapServerConfig();
         final String location = props.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
         final String[] tokens = location.split(":");
-        args = new String[] {
-                "-q", "Some sparql query",
-                "-t", rule.getKafkaTopicName(),
-                "-p", tokens[1],
-                "-i", tokens[0]
+
+        kafkaIp = tokens[0];
+        kafkaPort = tokens[1];
+
+        // Initialize the QueryRepository.
+        final Properties producerProperties = new Properties();
+        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
+        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
+
+        final Properties consumerProperties = new Properties();
+        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
+        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
+
+        queryProducer = new KafkaProducer<>(producerProperties);
+        queryConsumer = new KafkaConsumer<>(consumerProperties);
+
+        final String changeLogTopic = KafkaTopics.queryChangeLogTopic("" + ryaInstance);
+        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, changeLogTopic);
+        queryRepo = new InMemoryQueryRepository(changeLog);
+    }
+
+    @After
+    public void cleanup() {
+        queryProducer.close();
+        queryConsumer.close();
+    }
+
+    @Test
+    public void shortParams() throws Exception {
+        // Arguments that add a query to Rya Streams.
+        final String query = "SELECT * WHERE { ?person <urn:name> ?name }";
+        final String[] args = new String[] {
+                "-r", "" + ryaInstance,
+                "-i", kafkaIp,
+                "-p", kafkaPort,
+                "-q", query
         };
+
+        // Execute the command.
+        final AddQueryCommand command = new AddQueryCommand();
+        command.execute(args);
+
+        // Show that the query was added to the Query Repository.
+        final Set<StreamsQuery> queries = queryRepo.list();
+        assertEquals(1, queries.size());
+        assertEquals(query, queries.iterator().next().getSparql());
     }
 
     @Test
-    public void happyAddQueryTest() throws Exception {
+    public void longParams() throws Exception {
+        // Arguments that add a query to Rya Streams.
+        final String query = "SELECT * WHERE { ?person <urn:name> ?name }";
+        final String[] args = new String[] {
+                "--ryaInstance", "" + ryaInstance,
+                "--kafkaHostname", kafkaIp,
+                "--kafkaPort", kafkaPort,
+                "--query", query
+        };
+
+        // Execute the command.
         final AddQueryCommand command = new AddQueryCommand();
         command.execute(args);
-        // not sure what to assert here.
-        assertEquals(true, true);
+
+        // Show that the query was added to the Query Repository.
+        final Set<StreamsQuery> queries = queryRepo.list();
+        assertEquals(1, queries.size());
+        assertEquals(query, queries.iterator().next().getSparql());
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
index db8c200..0079371 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
@@ -19,25 +19,49 @@
 package org.apache.rya.streams.client.command;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Properties;
+import java.util.Set;
+import java.util.UUID;
 
 import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.rya.streams.api.entity.StreamsQuery;
+import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.api.queries.QueryChangeLog;
+import org.apache.rya.streams.api.queries.QueryRepository;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
 import org.apache.rya.test.kafka.KafkaITBase;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
-
 /**
  * Integration Test for deleting a query from Rya Streams through a command.
  */
 public class DeleteQueryCommandIT extends KafkaITBase {
-    private List<String> args;
+
+    private final String ryaInstance = UUID.randomUUID().toString();
+
+    private String kafkaIp;
+    private String kafkaPort;
+
+    private Producer<?, QueryChange> queryProducer = null;
+    private Consumer<?, QueryChange> queryConsumer = null;
 
     @Rule
     public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
@@ -48,28 +72,107 @@ public class DeleteQueryCommandIT extends KafkaITBase {
         final String location = props.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
         final String[] tokens = location.split(":");
 
-        args = Lists.newArrayList(
-                "-t", rule.getKafkaTopicName(),
-                "-p", tokens[1],
-                "-i", tokens[0]
-                );
+        kafkaIp = tokens[0];
+        kafkaPort = tokens[1];
+    }
+
+    /**
+     * This test simulates executing many commands and each of them use their own InMemoryQueryRepository. We need
+     * to re-create the repo outside of the command to ensure it has the most up to date values inside of it.
+     */
+    private QueryRepository makeQueryRepository() {
+        final Properties producerProperties = new Properties();
+        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
+        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
+
+        final Properties consumerProperties = new Properties();
+        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
+        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
+
+        cleanup();
+        queryProducer = new KafkaProducer<>(producerProperties);
+        queryConsumer = new KafkaConsumer<>(consumerProperties);
+
+        final String changeLogTopic = KafkaTopics.queryChangeLogTopic("" + ryaInstance);
+        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, changeLogTopic);
+        return new InMemoryQueryRepository(changeLog);
+    }
+
+    @After
+    public void cleanup() {
+        if(queryProducer != null) {
+            queryProducer.close();
+        }
+        if(queryConsumer != null) {
+            queryConsumer.close();
+        }
     }
 
     @Test
-    public void happyDeleteQueryTest() throws Exception {
-        // add a query so that it can be removed.
-        final List<String> addArgs = new ArrayList<>(args);
-        addArgs.add("-q");
-        addArgs.add("Some sparql query");
-        final AddQueryCommand addCommand = new AddQueryCommand();
-        addCommand.execute(addArgs.toArray(new String[] {}));
-
-        final List<String> deleteArgs = new ArrayList<>(args);
-        addArgs.add("-q");
-        addArgs.add("12345");
+    public void shortParams() throws Exception {
+        // Add a few queries to Rya Streams.
+        QueryRepository repo = makeQueryRepository();
+        repo.add("query1");
+        final UUID query2Id = repo.add("query2").getQueryId();
+        repo.add("query3");
+
+        // Show that all three of the queries were added.
+        Set<StreamsQuery> queries = repo.list();
+        assertEquals(3, queries.size());
+
+        // Delete query 2 using the delete query command.
+        final String[] deleteArgs = new String[] {
+                "-r", "" + ryaInstance,
+                "-i", kafkaIp,
+                "-p", kafkaPort,
+                "-q", query2Id.toString()
+        };
+
+        final DeleteQueryCommand deleteCommand = new DeleteQueryCommand();
+        deleteCommand.execute(deleteArgs);
+
+        // Show query2 was deleted.
+        repo = makeQueryRepository();
+        queries = repo.list();
+        assertEquals(2, queries.size());
+
+        for(final StreamsQuery query : queries) {
+            assertNotEquals(query2Id, query.getQueryId());
+        }
+    }
+
+    @Test
+    public void longParams() throws Exception {
+        // Add a few queries to Rya Streams.
+        QueryRepository repo = makeQueryRepository();
+        repo.add("query1");
+        final UUID query2Id = repo.add("query2").getQueryId();
+        repo.add("query3");
+
+        // Show that all three of the queries were added.
+        Set<StreamsQuery> queries = repo.list();
+        assertEquals(3, queries.size());
+
+        // Delete query 2 using the delete query command.
+        final String[] deleteArgs = new String[] {
+                "--ryaInstance", "" + ryaInstance,
+                "--kafkaHostname", kafkaIp,
+                "--kafkaPort", kafkaPort,
+                "--queryID", query2Id.toString()
+        };
+
         final DeleteQueryCommand deleteCommand = new DeleteQueryCommand();
-        deleteCommand.execute(deleteArgs.toArray(new String[] {}));
-        // not sure what to assert here.
-        assertEquals(true, true);
+        deleteCommand.execute(deleteArgs);
+
+        // Show query2 was deleted.
+        repo = makeQueryRepository();
+        queries = repo.list();
+        assertEquals(2, queries.size());
+
+        for(final StreamsQuery query : queries) {
+            assertNotEquals(query2Id, query.getQueryId());
+        }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
index be90c5f..eb759ba 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
@@ -18,13 +18,29 @@
  */
 package org.apache.rya.streams.client.command;
 
-import static org.junit.Assert.assertEquals;
-
 import java.util.Properties;
+import java.util.UUID;
 
 import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.api.queries.QueryChangeLog;
+import org.apache.rya.streams.api.queries.QueryRepository;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
 import org.apache.rya.test.kafka.KafkaITBase;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -33,7 +49,15 @@ import org.junit.Test;
  * integration Test for listing queries through a command.
  */
 public class ListQueryCommandIT extends KafkaITBase {
-    private String[] args;
+
+    private final String ryaInstance = UUID.randomUUID().toString();
+
+    private String kafkaIp;
+    private String kafkaPort;
+    private QueryRepository queryRepo;
+
+    private Producer<?, QueryChange> queryProducer = null;
+    private Consumer<?, QueryChange> queryConsumer = null;
 
     @Rule
     public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
@@ -43,18 +67,69 @@ public class ListQueryCommandIT extends KafkaITBase {
         final Properties props = rule.createBootstrapServerConfig();
         final String location = props.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
         final String[] tokens = location.split(":");
-        args = new String[] {
-                "-t", rule.getKafkaTopicName(),
-                "-p", tokens[1],
-                "-i", tokens[0]
+
+        kafkaIp = tokens[0];
+        kafkaPort = tokens[1];
+
+        // Initialize the QueryRepository.
+        final Properties producerProperties = new Properties();
+        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
+        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
+
+        final Properties consumerProperties = new Properties();
+        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
+        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
+
+        queryProducer = new KafkaProducer<>(producerProperties);
+        queryConsumer = new KafkaConsumer<>(consumerProperties);
+
+        final String changeLogTopic = KafkaTopics.queryChangeLogTopic("" + ryaInstance);
+        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, changeLogTopic);
+        queryRepo = new InMemoryQueryRepository(changeLog);
+    }
+
+    @After
+    public void cleanup() {
+        queryProducer.close();
+        queryConsumer.close();
+    }
+
+
+    @Test
+    public void shortParams() throws Exception {
+        // Add a few queries to Rya Streams.
+        queryRepo.add("query1");
+        queryRepo.add("query2");
+        queryRepo.add("query3");
+
+        // Execute the List Queries command.
+        final String[] args = new String[] {
+                "-r", "" + ryaInstance,
+                "-i", kafkaIp,
+                "-p", kafkaPort
         };
+
+        final ListQueriesCommand command = new ListQueriesCommand();
+        command.execute(args);
     }
 
     @Test
-    public void happyListQueriesTest() throws Exception {
+    public void longParams() throws Exception {
+        // Add a few queries to Rya Streams.
+        queryRepo.add("query1");
+        queryRepo.add("query2");
+        queryRepo.add("query3");
+
+        // Execute the List Queries command.
+        final String[] args = new String[] {
+                "--ryaInstance", "" + ryaInstance,
+                "--kafkaHostname", kafkaIp,
+                "--kafkaPort", kafkaPort
+        };
+
         final ListQueriesCommand command = new ListQueriesCommand();
         command.execute(args);
-        // not sure what to assert here.
-        assertEquals(true, true);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/LoadStatementsCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/LoadStatementsCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/LoadStatementsCommandIT.java
new file mode 100644
index 0000000..95a4876
--- /dev/null
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/LoadStatementsCommandIT.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.client.command;
+
+import static org.junit.Assert.assertEquals;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+/**
+ * Integration tests the methods of {@link LoadStatementsCommand}.
+ */
+public class LoadStatementsCommandIT {
+
+    private static final Path TURTLE_FILE = Paths.get("src/test/resources/statements.ttl");
+
+    private final String ryaInstance = UUID.randomUUID().toString();
+
+    private String kafkaIp;
+    private String kafkaPort;
+
+    @Rule
+    public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
+
+    @Before
+    public void setup() {
+        final Properties props = rule.createBootstrapServerConfig();
+        final String location = props.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
+        final String[] tokens = location.split(":");
+
+        kafkaIp = tokens[0];
+        kafkaPort = tokens[1];
+    }
+
+    @Test
+    public void shortParams() throws Exception {
+        // Load a file of statements into Kafka.
+        final String visibilities = "a|b|c";
+        final String[] args = new String[] {
+                "-r", "" + ryaInstance,
+                "-i", kafkaIp,
+                "-p", kafkaPort,
+                "-f", TURTLE_FILE.toString(),
+                "-v", visibilities
+        };
+
+        new LoadStatementsCommand().execute(args);
+
+        // Show that the statements were loaded into the topic.
+        // Read a VisibilityBindingSet from the test topic.
+        final List<VisibilityStatement> read = new ArrayList<>();
+
+        final Properties consumerProps = new Properties();
+        consumerProps.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
+        consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
+        consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+        consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VisibilityStatementDeserializer.class.getName());
+
+        try(final Consumer<String, VisibilityStatement> consumer = new KafkaConsumer<>(consumerProps)) {
+            final String topic = KafkaTopics.statementsTopic(ryaInstance);
+            consumer.subscribe(Arrays.asList(topic));
+            final ConsumerRecords<String, VisibilityStatement> records = consumer.poll(3000);
+
+            assertEquals(3, records.count());
+            final Iterator<ConsumerRecord<String, VisibilityStatement>> iter = records.iterator();
+            while(iter.hasNext()) {
+                final VisibilityStatement visiSet = iter.next().value();
+                read.add(visiSet);
+            }
+        }
+
+        final ValueFactory VF = ValueFactoryImpl.getInstance();
+        final List<VisibilityStatement> expected = new ArrayList<>();
+        expected.add(new VisibilityStatement(
+                VF.createStatement(VF.createURI("http://example#alice"), VF.createURI("http://example#talksTo"), VF.createURI("http://example#bob")),
+                visibilities));
+        expected.add(new VisibilityStatement(
+                VF.createStatement(VF.createURI("http://example#bob"), VF.createURI("http://example#talksTo"), VF.createURI("http://example#charlie")),
+                visibilities));
+        expected.add(new VisibilityStatement(
+                VF.createStatement(VF.createURI("http://example#charlie"), VF.createURI("http://example#likes"), VF.createURI("http://example#icecream")),
+                visibilities));
+
+        // Show the written statements matches the read ones.
+        assertEquals(expected, read);
+    }
+
+    @Test
+    public void longParams() throws Exception {
+        // Load a file of statements into Kafka.
+        final String visibilities = "a|b|c";
+        final String[] args = new String[] {
+                "--ryaInstance", "" + ryaInstance,
+                "--kafkaHostname", kafkaIp,
+                "--kafkaPort", kafkaPort,
+                "--statementsFile", TURTLE_FILE.toString(),
+                "--visibilities", visibilities
+        };
+
+        new LoadStatementsCommand().execute(args);
+
+        // Show that the statements were loaded into the topic.
+        // Read a VisibilityBindingSet from the test topic.
+        final List<VisibilityStatement> read = new ArrayList<>();
+
+        final Properties consumerProps = new Properties();
+        consumerProps.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaIp + ":" + kafkaPort);
+        consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
+        consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+        consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VisibilityStatementDeserializer.class.getName());
+
+        try(final Consumer<String, VisibilityStatement> consumer = new KafkaConsumer<>(consumerProps)) {
+            final String topic = KafkaTopics.statementsTopic(ryaInstance);
+            consumer.subscribe(Arrays.asList(topic));
+            final ConsumerRecords<String, VisibilityStatement> records = consumer.poll(3000);
+
+            assertEquals(3, records.count());
+            final Iterator<ConsumerRecord<String, VisibilityStatement>> iter = records.iterator();
+            while(iter.hasNext()) {
+                final VisibilityStatement visiSet = iter.next().value();
+                read.add(visiSet);
+            }
+        }
+
+        final ValueFactory VF = ValueFactoryImpl.getInstance();
+        final List<VisibilityStatement> expected = new ArrayList<>();
+        expected.add(new VisibilityStatement(
+                VF.createStatement(VF.createURI("http://example#alice"), VF.createURI("http://example#talksTo"), VF.createURI("http://example#bob")),
+                visibilities));
+        expected.add(new VisibilityStatement(
+                VF.createStatement(VF.createURI("http://example#bob"), VF.createURI("http://example#talksTo"), VF.createURI("http://example#charlie")),
+                visibilities));
+        expected.add(new VisibilityStatement(
+                VF.createStatement(VF.createURI("http://example#charlie"), VF.createURI("http://example#likes"), VF.createURI("http://example#icecream")),
+                visibilities));
+
+        // Show the written statements matches the read ones.
+        assertEquals(expected, read);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/client/src/test/resources/statements.ttl
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/resources/statements.ttl b/extras/rya.streams/client/src/test/resources/statements.ttl
new file mode 100644
index 0000000..c19e22d
--- /dev/null
+++ b/extras/rya.streams/client/src/test/resources/statements.ttl
@@ -0,0 +1,21 @@
+# 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.
+@prefix example: <http://example#> .
+
+example:alice example:talksTo example:bob . 
+example:bob example:talksTo example:charlie . 
+example:charlie example:likes example:icecream .

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/7deb0c00/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
new file mode 100644
index 0000000..dfc4c9d
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka;
+
+import org.apache.rya.streams.api.queries.QueryChangeLog;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Creates the Kafka topic names that are used for Rya Streams systems.
+ */
+@DefaultAnnotation(NonNull.class)
+public class KafkaTopics {
+
+    /**
+     * Creates the Kafka topic that will be used for a specific instance of Rya's {@link QueryChangeLog}.
+     *
+     * @param ryaInstance - The Rya instance the change log is for. (not null)
+     * @return The name of the Kafka topic.
+     */
+    public static String queryChangeLogTopic(final String ryaInstance) {
+        return ryaInstance + "-QueryChangeLog";
+    }
+
+    /**
+     * Creates the Kafka topic that will be used to load statements into the Rya Streams system for a specific
+     * instance of Rya.
+     *
+     * @param ryaInstance - The Rya instance the statements are for. (not null)
+     * @return The name of the Kafka topic.
+     */
+    public static String statementsTopic(final String ryaInstance) {
+        return ryaInstance + "-Statements";
+    }
+}
\ No newline at end of file


[24/50] [abbrv] incubator-rya git commit: RYA-377 Implement the Filter processors for Rya Streams.

Posted by ca...@apache.org.
RYA-377 Implement the Filter processors for Rya Streams.


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

Branch: refs/heads/master
Commit: a5e361806faeae26ff5a4acfcf2dd4bb5f74a75f
Parents: da63fd1
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Nov 21 12:40:54 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../api/function/Filter/FilterEvaluator.java    | 117 ++++++++++++++++++
 .../function/filter/FilterEvaluatorTest.java    | 108 +++++++++++++++++
 .../filter/FilterProcessorSupplier.java         | 120 +++++++++++++++++++
 .../streams/kafka/topology/TopologyFactory.java |  18 +++
 .../apache/rya/streams/kafka/RdfTestUtil.java   |  23 ++++
 .../processors/filter/FilterProcessorIT.java    |  86 +++++++++++++
 .../processors/filter/FilterProcessorTest.java  |  75 ++++++++++++
 7 files changed, 547 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a5e36180/common/rya.api.function/src/main/java/org/apache/rya/api/function/Filter/FilterEvaluator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/Filter/FilterEvaluator.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/Filter/FilterEvaluator.java
new file mode 100644
index 0000000..d1e1776
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/Filter/FilterEvaluator.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.Filter;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.algebra.Filter;
+import org.openrdf.query.algebra.ValueExpr;
+import org.openrdf.query.algebra.evaluation.TripleSource;
+import org.openrdf.query.algebra.evaluation.impl.EvaluationStrategyImpl;
+import org.openrdf.query.algebra.evaluation.util.QueryEvaluationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * Processes a {@link Filter} node from a SPARQL query.
+ */
+@DefaultAnnotation(NonNull.class)
+public class FilterEvaluator {
+    private static final Logger log = LoggerFactory.getLogger(FilterEvaluator.class);
+
+    /**
+     * Is used to evaluate the conditions of a {@link Filter}.
+     */
+    private static final EvaluationStrategyImpl EVALUATOR = new EvaluationStrategyImpl(
+            new TripleSource() {
+                private final ValueFactory valueFactory = new ValueFactoryImpl();
+
+                @Override
+                public ValueFactory getValueFactory() {
+                    return valueFactory;
+                }
+
+                @Override
+                public CloseableIteration<? extends Statement, QueryEvaluationException> getStatements(
+                        final Resource arg0,
+                        final URI arg1,
+                        final Value arg2,
+                        final Resource... arg3) throws QueryEvaluationException {
+                    throw new UnsupportedOperationException();
+                }
+            });
+
+    private final ValueExpr condition;
+
+    /**
+     * Constructs an instance of {@link FilterEvaluator}.
+     *
+     * @param condition - The condition that defines what passes the filter function. (not null)
+     */
+    public FilterEvaluator(final ValueExpr condition) {
+        this.condition = requireNonNull(condition);
+    }
+
+    /**
+     * Make a {@link FilterEvaluator} that processes the logic of a {@link Filter}.
+     *
+     * @param filter - Defines the Filter that will be processed. (not null)
+     * @return The {@link FilterEvaluator} for the provided {@link Filter}.
+     */
+    public static FilterEvaluator make(final Filter filter) {
+        requireNonNull(filter);
+        final ValueExpr condition = filter.getCondition();
+        return new FilterEvaluator(condition);
+    }
+
+    /**
+     * Checks to see if a {@link VisibilityBindingSet} should be included in the results or not.
+     *
+     * @param bs - The value that will be evaluated against the filter. (not null)
+     * @return {@code true} if the binding set matches the filter and it should be included in the node's results,
+     *   otherwise {@code false} and it should be excluded.
+     */
+    public boolean filter(final VisibilityBindingSet bs) {
+        requireNonNull(bs);
+
+        try {
+            final Value result = EVALUATOR.evaluate(condition, bs);
+            return QueryEvaluationUtil.getEffectiveBooleanValue(result);
+        } catch (final QueryEvaluationException e) {
+            //False returned because for whatever reason, the ValueExpr could not be evaluated.
+            //In the event that the ValueExpr is a FunctionCall, this Exception will be generated if
+            //the Function URI is a valid URI that was found in the FunctionRegistry, but the arguments
+            //for that Function could not be parsed.
+            log.error("Could not evaluate a Filter.", e);
+            return false;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a5e36180/common/rya.api.function/src/test/java/org/apache/rya/api/function/filter/FilterEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/test/java/org/apache/rya/api/function/filter/FilterEvaluatorTest.java b/common/rya.api.function/src/test/java/org/apache/rya/api/function/filter/FilterEvaluatorTest.java
new file mode 100644
index 0000000..8bbf005
--- /dev/null
+++ b/common/rya.api.function/src/test/java/org/apache/rya/api/function/filter/FilterEvaluatorTest.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.filter;
+
+import static java.util.Objects.requireNonNull;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.rya.api.function.Filter.FilterEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.Filter;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.impl.MapBindingSet;
+import org.openrdf.query.parser.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
+
+/**
+ * Unit tests the methods of {@link FilterEvaluator}.
+ */
+public class FilterEvaluatorTest {
+
+    @Test
+    public void matches() throws Exception {
+        // Read the filter object from a SPARQL query.
+        final Filter filter = getFilter(
+                "SELECT * " +
+                "WHERE { " +
+                    "FILTER(?age < 10)" +
+                    "?person <urn:age> ?age " +
+                "}");
+
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("age", vf.createLiteral(9));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bs);
+
+        // Test the evaluator.
+        assertTrue( FilterEvaluator.make(filter).filter(visBs) );
+    }
+
+    @Test
+    public void doesNotMatch() throws Exception {
+        // Read the filter object from a SPARQL query.
+        final Filter filter = getFilter(
+                "SELECT * " +
+                "WHERE { " +
+                    "FILTER(?age < 10)" +
+                    "?person <urn:age> ?age " +
+                "}");
+
+        // Create the input binding set.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("age", vf.createLiteral(11));
+        final VisibilityBindingSet visBs = new VisibilityBindingSet(bs);
+
+        // Test the evaluator.
+        assertFalse( FilterEvaluator.make(filter).filter(visBs) );
+    }
+
+    /**
+     * Get the first {@link Filter} node from a SPARQL query.
+     *
+     * @param sparql - The query that contains a single Projection node.
+     * @return The first {@link Filter} that is encountered.
+     * @throws Exception The query could not be parsed.
+     */
+    public static @Nullable Filter getFilter(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<Filter> filter = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final Filter node) throws Exception {
+                filter.set(node);
+            }
+        });
+
+        return filter.get();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a5e36180/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorSupplier.java
new file mode 100644
index 0000000..5997237
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorSupplier.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.filter;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.function.Filter.FilterEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.ResultType;
+import org.apache.rya.streams.kafka.processors.ProcessorResultFactory;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessor;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessorSupplier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Supplies {@link FilterProcessor} instances.
+ */
+@DefaultAnnotation(NonNull.class)
+public class FilterProcessorSupplier extends RyaStreamsProcessorSupplier {
+    private static final Logger log = LoggerFactory.getLogger(FilterProcessorSupplier.class);
+
+    private final FilterEvaluator filter;
+
+    /**
+     * Constructs an instance of {@link FilterProcessorSupplier}.
+     *
+     * @param filter - Defines the filter the supplied processors will evaluate. (not null)
+     * @param resultFactory - The factory that the supplied processors will use to create results. (not null)
+     */
+    public FilterProcessorSupplier(
+            final FilterEvaluator filter,
+            final ProcessorResultFactory resultFactory) {
+        super(resultFactory);
+        this.filter = requireNonNull(filter);
+    }
+
+    @Override
+    public Processor<Object, ProcessorResult> get() {
+        return new FilterProcessor(filter, super.getResultFactory());
+    }
+
+    /**
+     * Evaluates {@link ProcessorResult}s against a {@link FilterEvaluator} and forwards the original result
+     * to a downstream processor if it passes the filter's condition.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static class FilterProcessor extends RyaStreamsProcessor {
+
+        private final FilterEvaluator filter;
+        private ProcessorContext context;
+
+        /**
+         * Constructs an instance of {@link FilterProcessor}.
+         *
+         * @param filter - Defines the filter the supplied processor will evaluate. (not null)
+         * @param resultFactory - The factory that the processor will use to create results. (not null)
+         */
+        public FilterProcessor(
+                final FilterEvaluator filter,
+                final ProcessorResultFactory resultFactory) {
+            super(resultFactory);
+            this.filter = requireNonNull(filter);
+        }
+
+        @Override
+        public void init(final ProcessorContext context) {
+            this.context = context;
+        }
+
+        @Override
+        public void process(final Object key, final ProcessorResult value) {
+            // Filters can only be unary.
+            if (value.getType() != ResultType.UNARY) {
+                throw new RuntimeException("The ProcessorResult to be processed must be Unary.");
+            }
+
+            // If the value's binding set passes the filter, then forward it to the downstream processor.
+            final VisibilityBindingSet bindingSet = value.getUnary().getResult();
+            log.debug("\nINPUT:\n{}", bindingSet);
+            if(filter.filter(bindingSet)) {
+                log.debug("\nOUTPUT:\n{}", bindingSet);
+                final ProcessorResult result = super.getResultFactory().make(bindingSet);
+                context.forward(key, result);
+            }
+        }
+
+        @Override
+        public void punctuate(final long timestamp) {
+            // Do nothing.
+        }
+
+        @Override
+        public void close() {
+            // Do nothing.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a5e36180/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
index 08f3625..426b041 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
@@ -38,6 +38,7 @@ import org.apache.kafka.streams.processor.ProcessorSupplier;
 import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.kafka.streams.state.Stores;
+import org.apache.rya.api.function.Filter.FilterEvaluator;
 import org.apache.rya.api.function.join.IterativeJoin;
 import org.apache.rya.api.function.join.LeftOuterJoin;
 import org.apache.rya.api.function.join.NaturalJoin;
@@ -50,6 +51,7 @@ import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier;
 import org.apache.rya.streams.kafka.processors.output.StatementOutputFormatterSupplier;
@@ -62,6 +64,7 @@ import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
 import org.openrdf.query.MalformedQueryException;
 import org.openrdf.query.algebra.BinaryTupleOperator;
 import org.openrdf.query.algebra.Extension;
+import org.openrdf.query.algebra.Filter;
 import org.openrdf.query.algebra.Join;
 import org.openrdf.query.algebra.LeftJoin;
 import org.openrdf.query.algebra.MultiProjection;
@@ -90,6 +93,7 @@ public class TopologyFactory implements TopologyBuilderFactory {
     private static final String STATEMENT_PATTERN_PREFIX = "SP_";
     private static final String JOIN_PREFIX = "JOIN_";
     private static final String PROJECTION_PREFIX = "PROJECTION_";
+    private static final String FILTER_PREFIX = "FILTER_";
     private static final String SINK = "SINK";
 
     private List<ProcessorEntry> processorEntryList;
@@ -428,6 +432,20 @@ public class TopologyFactory implements TopologyBuilderFactory {
         }
 
         @Override
+        public void meet(final Filter node) throws TopologyBuilderException {
+            final String id = FILTER_PREFIX + UUID.randomUUID();
+            final Optional<Side> side = getSide(node);
+
+            final FilterProcessorSupplier supplier = new FilterProcessorSupplier(
+                    FilterEvaluator.make(node),
+                    result -> getResult(side, result));
+
+            entries.add(new ProcessorEntry(node, id, side, supplier, Lists.newArrayList(node.getArg())));
+            idMap.put(node, id);
+            super.meet(node);
+        }
+
+        @Override
         public void meet(final Join node) throws TopologyBuilderException {
             final String id = JOIN_PREFIX + UUID.randomUUID();
             meetJoin(id, new NaturalJoin(), node);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a5e36180/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
index 190bad3..b4388c3 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
@@ -22,6 +22,7 @@ import static java.util.Objects.requireNonNull;
 
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.openrdf.query.algebra.Filter;
 import org.openrdf.query.algebra.MultiProjection;
 import org.openrdf.query.algebra.Projection;
 import org.openrdf.query.algebra.StatementPattern;
@@ -105,4 +106,26 @@ public final class RdfTestUtil {
 
         return multiProjection.get();
     }
+
+    /**
+     * Get the first {@link Filter} node from a SPARQL query.
+     *
+     * @param sparql - The query that contains a single Projection node.
+     * @return The first {@link Filter} that is encountered.
+     * @throws Exception The query could not be parsed.
+     */
+    public static @Nullable Filter getFilter(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<Filter> filter = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final Filter node) throws Exception {
+                filter.set(node);
+            }
+        });
+
+        return filter.get();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a5e36180/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
new file mode 100644
index 0000000..0348dcd
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.filter;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Integration tests the methods of {@link FilterProcessor}.
+ */
+public class FilterProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void showProcessorWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "SELECT * " +
+                "WHERE { " +
+                    "FILTER(?age < 10)" +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(11)), "a"));
+        statements.add(new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(9)), "a"));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("age", vf.createLiteral(9));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a5e36180/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
new file mode 100644
index 0000000..4c44bae
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.filter;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.function.Filter.FilterEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.RdfTestUtil;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.Filter;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Unit tests the methods of {@link FilterProcessor}.
+ */
+public class FilterProcessorTest {
+
+    @Test
+    public void showFilterFunctionIsCalled() throws Exception {
+        // Read the filter object from a SPARQL query.
+        final Filter filter = RdfTestUtil.getFilter(
+                "SELECT * " +
+                "WHERE { " +
+                    "FILTER(?age < 10)" +
+                    "?person <urn:age> ?age " +
+                "}");
+
+        // Create a Binding Set that will be passed into the Filter function based on the where clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("age", vf.createLiteral(9));
+        final VisibilityBindingSet inputVisBs = new VisibilityBindingSet(bs, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final FilterProcessor processor = new FilterProcessor(
+                FilterEvaluator.make(filter),
+                result -> ProcessorResult.make(new UnaryResult(result)));
+        processor.init(context);
+        processor.process("key", ProcessorResult.make(new UnaryResult(inputVisBs)));
+
+        // Verify the binding set was passed through.
+        verify(context, times(1)).forward(eq("key"), eq(ProcessorResult.make(new UnaryResult(inputVisBs))));
+
+    }
+}
\ No newline at end of file


[31/50] [abbrv] incubator-rya git commit: RYA-377 Fixed the project layout so that it builds even when the geoindexing profile is not enabled.

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
deleted file mode 100644
index 22a883b..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors.filter;
-
-import static org.junit.Assert.assertEquals;
-
-import java.time.ZonedDateTime;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.function.projection.RandomUUIDFactory;
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
-import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
-import org.apache.rya.streams.kafka.topology.TopologyFactory;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.StatementImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.algebra.evaluation.function.Function;
-import org.openrdf.query.algebra.evaluation.function.FunctionRegistry;
-import org.openrdf.query.impl.MapBindingSet;
-
-/**
- * Integration tests the temporal methods of {@link FilterProcessor}.
- */
-public class TemporalFilterIT {
-    private static final ValueFactory vf = new ValueFactoryImpl();
-    private static final String TEMPORAL = "http://rya.apache.org/ns/temporal";
-    private static final ZonedDateTime TIME = ZonedDateTime.parse("2015-12-30T12:00:00Z");
-    private static final ZonedDateTime TIME_10 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
-    private static final ZonedDateTime TIME_20 = ZonedDateTime.parse("2015-12-30T12:00:20Z");
-
-    @Rule
-    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(false);
-
-    @Test
-    public void temporalFunctionsRegistered() {
-        int count = 0;
-        final Collection<Function> funcs = FunctionRegistry.getInstance().getAll();
-        for (final Function fun : funcs) {
-            if (fun.getURI().startsWith(TEMPORAL)) {
-                count++;
-            }
-        }
-
-        // There are 4 temporal functions registered, ensure that there are 4.
-        assertEquals(4, count);
-    }
-
-    @Test
-    public void showEqualsWorks() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Get the RDF model objects that will be used to build the query.
-        final String sparql =
-                "PREFIX time: <http://www.w3.org/2006/time/> \n"
-                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
-                        + "SELECT * \n"
-                        + "WHERE { \n"
-                        + "  <urn:time> time:atTime ?date .\n"
-                        + " FILTER(tempf:equals(?date, \"" + TIME.toString() + "\")) "
-                        + "}";
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create the statements that will be input into the query.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = getStatements();
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        final MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("date", vf.createLiteral(TIME.toString()));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void showBeforeWorks() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Get the RDF model objects that will be used to build the query.
-        final String sparql =
-                "PREFIX time: <http://www.w3.org/2006/time/> \n"
-                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
-                        + "SELECT * \n"
-                        + "WHERE { \n"
-                        + "  <urn:time> time:atTime ?date .\n"
-                        + " FILTER(tempf:before(?date, \"" + TIME_10.toString() + "\")) "
-                        + "}";
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create the statements that will be input into the query.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = getStatements();
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        final MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("date", vf.createLiteral(TIME.toString()));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void showAfterWorks() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Get the RDF model objects that will be used to build the query.
-        final String sparql =
-                "PREFIX time: <http://www.w3.org/2006/time/> \n"
-                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
-                        + "SELECT * \n"
-                        + "WHERE { \n"
-                        + "  <urn:time> time:atTime ?date .\n"
-                        + " FILTER(tempf:after(?date, \"" + TIME_10.toString() + "\")) "
-                        + "}";
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create the statements that will be input into the query.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = getStatements();
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        final MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("date", vf.createLiteral(TIME_20.toString()));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void showWithinWorks() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Get the RDF model objects that will be used to build the query.
-        final String sparql =
-                "PREFIX time: <http://www.w3.org/2006/time/> \n"
-                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
-                        + "SELECT * \n"
-                        + "WHERE { \n"
-                        + "  <urn:time> time:atTime ?date .\n"
-                        + " FILTER(tempf:within(?date, \"" + TIME.toString() + "/" + TIME_20.toString() + "\")) "
-                        + "}";
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create the statements that will be input into the query.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = getStatements();
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        final MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("date", vf.createLiteral(TIME_10.toString()));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    private List<VisibilityStatement> getStatements() throws Exception {
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(statement(TIME), "a"));
-        statements.add(new VisibilityStatement(statement(TIME_10), "a"));
-        statements.add(new VisibilityStatement(statement(TIME_20), "a"));
-        return statements;
-    }
-
-    private static Statement statement(final ZonedDateTime time) {
-        final Resource subject = vf.createURI("urn:time");
-        final URI predicate = vf.createURI("http://www.w3.org/2006/time/atTime");
-        final Value object = vf.createLiteral(time.toString());
-        return new StatementImpl(subject, predicate, object);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
deleted file mode 100644
index bdb9be6..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
+++ /dev/null
@@ -1,316 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors.join;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.function.join.NaturalJoin;
-import org.apache.rya.api.function.projection.RandomUUIDFactory;
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
-import org.apache.rya.streams.kafka.processors.ProcessorResult;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
-import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier.JoinProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
-import org.apache.rya.streams.kafka.topology.TopologyFactory;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.impl.MapBindingSet;
-
-import com.google.common.collect.Lists;
-
-/**
- * Integration tests the methods of {@link JoinProcessor}.
- */
-public class JoinProcessorIT {
-
-    @Rule
-    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
-
-    @Test(expected = IllegalArgumentException.class)
-    public void badAllVars() throws IllegalArgumentException {
-        new JoinProcessorSupplier(
-                "NATURAL_JOIN",
-                new NaturalJoin(),
-                Lists.newArrayList("employee"),
-                Lists.newArrayList("person", "employee", "business"),
-                result -> ProcessorResult.make( new UnaryResult(result) ));
-    }
-
-    @Test
-    public void newLeftResult() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final String query =
-                "SELECT * WHERE { " +
-                    "?person <urn:talksTo> ?employee ." +
-                    "?employee <urn:worksAt> ?business" +
-                " }";
-        final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create some statements that generate a bunch of right SP results.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "a&b") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), "b") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "b|c") );
-
-        // Add a statement that will generate a left result that joins with some of those right results.
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "c") );
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
-        expected.add( new VisibilityBindingSet(bs, "a&b&c") );
-
-        bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
-        expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void newRightResult() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final String query =
-                "SELECT * WHERE { " +
-                    "?person <urn:talksTo> ?employee ." +
-                    "?employee <urn:worksAt> ?business" +
-                " }";
-        final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create some statements that generate a bunch of right SP results.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "a&b") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), "b") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "b|c") );
-
-        // Add a statement that will generate a left result that joins with some of those right results.
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "c") );
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
-        expected.add( new VisibilityBindingSet(bs, "a&b&c") );
-
-        bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
-        expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void newResultsBothSides() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final String query =
-                "SELECT * WHERE { " +
-                    "?person <urn:talksTo> ?employee ." +
-                    "?employee <urn:worksAt> ?business" +
-                " }";
-        final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create some statements that generate a bunch of right SP results.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "a&b") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "c") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), "b") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "b|c") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:talksTo"), vf.createURI("urn:Charlie")), "c") );
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
-        expected.add( new VisibilityBindingSet(bs, "a&b&c") );
-
-        bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
-        expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
-
-        bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Bob"));
-        bs.addBinding("employee", vf.createURI("urn:Charlie"));
-        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
-        expected.add( new VisibilityBindingSet(bs, "a&c") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void manyJoins() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final String query =
-                "SELECT * WHERE { " +
-                    "?person <urn:talksTo> ?employee ." +
-                    "?employee <urn:worksAt> ?business ." +
-                    "?employee <urn:hourlyWage> ?wage ." +
-                " }";
-        final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create some statements that generate a bunch of right SP results.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hourlyWage"), vf.createLiteral(7.25)), "a") );
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        final MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
-        bs.addBinding("wage", vf.createLiteral(7.25));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void leftJoin() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final String query =
-                "SELECT * WHERE { " +
-                    "?person <urn:talksTo> ?employee ." +
-                    "OPTIONAL{ ?employee <urn:worksAt> ?business } " +
-                " }";
-        final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create some statements that generate a result that includes the optional value as well as one that does not.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "b") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:talksTo"), vf.createURI("urn:Charlie")), "c") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "d") );
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
-        expected.add( new VisibilityBindingSet(bs, "a&b") );
-
-        bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Bob"));
-        bs.addBinding("employee", vf.createURI("urn:Charlie"));
-        expected.add( new VisibilityBindingSet(bs, "c") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
deleted file mode 100644
index a8de401..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors.projection;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
-import org.apache.rya.streams.kafka.processors.projection.MultiProjectionProcessorSupplier.MultiProjectionProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
-import org.apache.rya.streams.kafka.topology.TopologyFactory;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.BNode;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.model.vocabulary.RDF;
-
-/**
- * Integration tests the methods of {@link MultiProjectionProcessor}.
- */
-public class MultiProjectionProcessorIT {
-
-    @Rule
-    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
-
-    @Test
-    public void showProcessorWorks() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Create a topology for the Query that will be tested.
-        final String sparql =
-                "CONSTRUCT {" +
-                    "_:b a <urn:movementObservation> ; " +
-                    "<urn:location> ?location ; " +
-                    "<urn:direction> ?direction ; " +
-                "}" +
-                "WHERE {" +
-                    "?thing <urn:corner> ?location ." +
-                    "?thing <urn:compass> ?direction." +
-                "}";
-
-        final String bNodeId = UUID.randomUUID().toString();
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, () -> bNodeId);
-
-        // Create the statements that will be input into the query.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:car1"), vf.createURI("urn:compass"), vf.createURI("urn:NW")), "a") );
-        statements.add( new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:car1"), vf.createURI("urn:corner"), vf.createURI("urn:corner1")), "a") );
-
-        // Make the expected results.
-        final Set<VisibilityStatement> expected = new HashSet<>();
-        final BNode blankNode = vf.createBNode(bNodeId);
-
-        expected.add(new VisibilityStatement(vf.createStatement(blankNode, RDF.TYPE, vf.createURI("urn:movementObservation")), "a"));
-        expected.add(new VisibilityStatement(vf.createStatement(blankNode, vf.createURI("urn:direction"), vf.createURI("urn:NW")), "a"));
-        expected.add(new VisibilityStatement(vf.createStatement(blankNode, vf.createURI("urn:location"), vf.createURI("urn:corner1")), "a"));
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityStatementDeserializer.class);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
deleted file mode 100644
index 2af3a49..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors.projection;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.function.projection.RandomUUIDFactory;
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
-import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
-import org.apache.rya.streams.kafka.topology.TopologyFactory;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.impl.MapBindingSet;
-
-import com.google.common.collect.Sets;
-
-/**
- * Integration tests the methods of {@link StatementPatternProcessor}.
- */
-public class ProjectionProcessorIT {
-
-    @Rule
-    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
-
-    @Test
-    public void showProcessorWorks() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Create a topology for the Query that will be tested.
-        final String sparql =
-                "SELECT (?person AS ?p) ?otherPerson " +
-                "WHERE { " +
-                    "?person <urn:talksTo> ?otherPerson . " +
-                "}";
-
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Load some data into the input topic.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
-
-        // Show the correct binding set results from the job.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-
-        final MapBindingSet expectedBs = new MapBindingSet();
-        expectedBs.addBinding("p", vf.createURI("urn:Alice"));
-        expectedBs.addBinding("otherPerson", vf.createURI("urn:Bob"));
-        expected.add(new VisibilityBindingSet(expectedBs, "a"));
-
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, Sets.newHashSet(expected), VisibilityBindingSetDeserializer.class);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/pom.xml b/extras/rya.streams/pom.xml
index bd559d2..dd876a0 100644
--- a/extras/rya.streams/pom.xml
+++ b/extras/rya.streams/pom.xml
@@ -35,7 +35,10 @@
 
     <modules>
         <module>kafka</module>
+        <module>kafka-test</module>
         <module>api</module>
         <module>client</module>
+        <module>geo</module>
+        <module>integration</module>
     </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 58cc7c5..31b17f8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -329,6 +329,11 @@ under the License.
                 <artifactId>rya.streams.kafka</artifactId>
                 <version>${project.version}</version>
             </dependency>
+            <dependency>
+                <groupId>org.apache.rya</groupId>
+                <artifactId>rya.streams.kafka-test</artifactId>
+                <version>${project.version}</version>
+            </dependency>
 
             <dependency>
                 <groupId>org.apache.thrift</groupId>
@@ -357,6 +362,11 @@ under the License.
             </dependency>
             <dependency>
                 <groupId>org.apache.rya</groupId>
+                <artifactId>rya.test.rdf</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.rya</groupId>
                 <artifactId>rya.test.kafka</artifactId>
                 <version>${project.version}</version>
             </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index d458f8f..d13ea0b 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -35,5 +35,6 @@
 
     <modules>
         <module>kafka</module>
+        <module>rdf</module>
     </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/test/rdf/pom.xml
----------------------------------------------------------------------
diff --git a/test/rdf/pom.xml b/test/rdf/pom.xml
new file mode 100644
index 0000000..f8dae1d
--- /dev/null
+++ b/test/rdf/pom.xml
@@ -0,0 +1,59 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+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">
+
+    <parent>
+        <groupId>org.apache.rya</groupId>
+        <artifactId>rya.test.parent</artifactId>
+        <version>3.2.12-incubating-SNAPSHOT</version>
+    </parent>
+    
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>rya.test.rdf</artifactId>
+    
+    <name>Apache Rya Test RDF</name>
+    <description>
+        This module contains the Rya Test RDF components that help write RDF based tests.
+    </description>
+
+    <dependencies>
+        <!-- Third Party Dependencies -->
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-queryalgebra-model</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-queryparser-sparql</artifactId>
+        </dependency>
+    
+        <dependency>
+            <groupId>com.github.stephenc.findbugs</groupId>
+            <artifactId>findbugs-annotations</artifactId>
+        </dependency>
+    
+        <!-- Testing dependencies. -->
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/test/rdf/src/main/java/org/apache/rya/streams/kafka/RdfTestUtil.java
----------------------------------------------------------------------
diff --git a/test/rdf/src/main/java/org/apache/rya/streams/kafka/RdfTestUtil.java b/test/rdf/src/main/java/org/apache/rya/streams/kafka/RdfTestUtil.java
new file mode 100644
index 0000000..b4388c3
--- /dev/null
+++ b/test/rdf/src/main/java/org/apache/rya/streams/kafka/RdfTestUtil.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.openrdf.query.algebra.Filter;
+import org.openrdf.query.algebra.MultiProjection;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.parser.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+
+/**
+ * A set of utility functions that are useful when writing tests RDF functions.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class RdfTestUtil {
+
+    private RdfTestUtil() { }
+
+    /**
+     * Fetch the {@link StatementPattern} from a SPARQL string.
+     *
+     * @param sparql - A SPARQL query that contains only a single Statement Patern. (not nul)
+     * @return The {@link StatementPattern} that was in the query, if it could be found. Otherwise {@code null}
+     * @throws Exception The statement pattern could not be found in the parsed SPARQL query.
+     */
+    public static @Nullable StatementPattern getSp(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<StatementPattern> statementPattern = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visitChildren(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final StatementPattern node) throws Exception {
+                statementPattern.set(node);
+            }
+        });
+        return statementPattern.get();
+    }
+
+    /**
+     * Get the first {@link Projection} node from a SPARQL query.
+     *
+     * @param sparql - The query that contains a single Projection node.
+     * @return The first {@link Projection} that is encountered.
+     * @throws Exception The query could not be parsed.
+     */
+    public static @Nullable Projection getProjection(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<Projection> projection = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final Projection node) throws Exception {
+                projection.set(node);
+            }
+        });
+
+        return projection.get();
+    }
+
+    /**
+     * Get the first {@link MultiProjection} node from a SPARQL query.
+     *
+     * @param sparql - The query that contains a single Projection node.
+     * @return The first {@link MultiProjection} that is encountered.
+     * @throws Exception The query could not be parsed.
+     */
+    public static @Nullable MultiProjection getMultiProjection(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<MultiProjection> multiProjection = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final MultiProjection node) throws Exception {
+                multiProjection.set(node);
+            }
+        });
+
+        return multiProjection.get();
+    }
+
+    /**
+     * Get the first {@link Filter} node from a SPARQL query.
+     *
+     * @param sparql - The query that contains a single Projection node.
+     * @return The first {@link Filter} that is encountered.
+     * @throws Exception The query could not be parsed.
+     */
+    public static @Nullable Filter getFilter(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<Filter> filter = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final Filter node) throws Exception {
+                filter.set(node);
+            }
+        });
+
+        return filter.get();
+    }
+}
\ No newline at end of file


[13/50] [abbrv] incubator-rya git commit: RYA-377 Abstract Join into api.function

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/VisibilitySimplifier.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/VisibilitySimplifier.java b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/VisibilitySimplifier.java
new file mode 100644
index 0000000..a9d7468
--- /dev/null
+++ b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/VisibilitySimplifier.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.model.visibility;
+
+import static java.util.Objects.requireNonNull;
+
+import com.google.common.base.Charsets;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Simplifies Accumulo visibility expressions.
+ * 
+ * XXX
+ * This class has been copied over because Rya has decided to use the Accumulo
+ * implementation of visibilities to control who is able to access what data
+ * within a Rya instance. Until we implement an Accumulo agnostic method for
+ * handling those visibility expressions, we have chosen to pull the Accumulo
+ * code into our API.
+ *
+ * Copied from rya accumulo's org.apache.accumulo.core.util.VisibilitySimplifier
+ *   <dependancy>
+ *     <groupId>org.apache.rya.accumulo</groupId>
+ *     <artifactId>accumulo.rya</artifactId>
+ *     <version>3.2.12-incubating-SNAPSHOT</version>
+ *   </dependancy>
+ */
+@DefaultAnnotation(NonNull.class)
+public class VisibilitySimplifier {
+
+    /**
+     * Unions two visibility equations and then simplifies the result.
+     *
+     * @param vis1 - The first visibility equation that will be unioned. (not null)
+     * @param vis2 - The other visibility equation that will be unioned. (not null)
+     * @return A simplified form of the unioned visibility equations.
+     */
+    public static String unionAndSimplify(final String vis1, final String vis2) {
+        requireNonNull(vis1);
+        requireNonNull(vis2);
+
+        if(vis1.isEmpty()) {
+            return vis2;
+        }
+
+        if(vis2.isEmpty()) {
+            return vis1;
+        }
+
+        return simplify("(" + vis1 + ")&(" + vis2 + ")");
+    }
+
+    /**
+     * Simplifies an Accumulo visibility expression.
+     *
+     * @param visibility - The expression to simplify. (not null)
+     * @return A simplified form of {@code visibility}.
+     */
+    public static String simplify(final String visibility) {
+        requireNonNull(visibility);
+
+        String last = visibility;
+        String simplified = new String(new ColumnVisibility(visibility).flatten(), Charsets.UTF_8);
+
+        while(!simplified.equals(last)) {
+            last = simplified;
+            simplified = new String(new ColumnVisibility(simplified).flatten(), Charsets.UTF_8);
+        }
+
+        return simplified;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/WritableComparator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/WritableComparator.java b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/WritableComparator.java
new file mode 100644
index 0000000..121da20
--- /dev/null
+++ b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/WritableComparator.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rya.api.model.visibility;
+
+/**
+ * A Comparator for {@link WritableComparable}s.
+ *
+ * <p>
+ * This base implemenation uses the natural ordering. To define alternate
+ * orderings, override {@link #compare(WritableComparable,WritableComparable)}.
+ *
+ * <p>
+ * One may optimize compare-intensive operations by overriding
+ * {@link #compare(byte[],int,int,byte[],int,int)}. Static utility methods are
+ * provided to assist in optimized implementations of this method.
+ *
+ * XXX
+ * This class has been copied over because Rya has decided to use the Accumulo
+ * implementation of visibilities to control who is able to access what data
+ * within a Rya instance. Until we implement an Accumulo agnostic method for
+ * handling those visibility expressions, we have chosen to pull the Accumulo
+ * code into our API.
+ *
+ * Copied from accumulo's org.apache.hadoop.io.WritableComparator
+ *   <dependancy>
+ *     <groupId>org.apache.hadoop</groupId>
+ *     <artifactId>hadoop-commons</artifactId>
+ *     <version>2.5</version>
+ *   </dependancy>
+ */
+public class WritableComparator {
+    /** Lexicographic order of binary data. */
+    public static int compareBytes(final byte[] b1, final int s1, final int l1, final byte[] b2, final int s2,
+            final int l2) {
+        return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/src/test/java/org/apache/rya/api/model/visibility/VisibilitySimplifierTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/test/java/org/apache/rya/api/model/visibility/VisibilitySimplifierTest.java b/common/rya.api.model/src/test/java/org/apache/rya/api/model/visibility/VisibilitySimplifierTest.java
new file mode 100644
index 0000000..30949ad
--- /dev/null
+++ b/common/rya.api.model/src/test/java/org/apache/rya/api/model/visibility/VisibilitySimplifierTest.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.model.visibility;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+/**
+ * Tests the methods of {@link VisibilitySimplifier}.
+ * 
+ * XXX
+ * This class has been copied over because Rya has decided to use the Accumulo
+ * implementation of visibilities to control who is able to access what data
+ * within a Rya instance. Until we implement an Accumulo agnostic method for
+ * handling those visibility expressions, we have chosen to pull the Accumulo
+ * code into our API.
+ *
+ * Copied from accumulo's org.apache.rya.accumulo.utils.VisibilitySimplifierTest
+ *   <dependancy>
+ *     <groupId>org.apache.rya.accumulo</groupId>
+ *     <artifactId>accumulo.rya</artifactId>
+ *     <version>3.2.12-incubating-SNAPSHOT</version>
+ *   </dependancy>
+ */
+public class VisibilitySimplifierTest {
+
+    @Test
+    public void noneRequired() {
+        final String simplified = new VisibilitySimplifier().simplify("u");
+        assertEquals("u", simplified);
+    }
+
+    @Test
+    public void parenthesis() {
+        final String simplified = new VisibilitySimplifier().simplify("(u&u)&u");
+        assertEquals("u", simplified);
+    }
+
+    @Test
+    public void manyAnds() {
+        final String simplified = new VisibilitySimplifier().simplify("u&u&u");
+        assertEquals("u", simplified);
+    }
+
+    @Test
+    public void complex() {
+        final String simplified = new VisibilitySimplifier().simplify("(a|b)|(a|b)|a|b");
+        assertEquals("a|b", simplified);
+    }
+
+    @Test
+    public void unionAndSimplify() {
+        final String simplified = new VisibilitySimplifier().unionAndSimplify("u&b", "u");
+        assertEquals("b&u", simplified);
+    }
+
+    @Test
+    public void unionAndSimplify_firstIsEmpty() {
+        final String simplified = new VisibilitySimplifier().unionAndSimplify("", "u");
+        assertEquals("u", simplified);
+    }
+
+    @Test
+    public void unionAndSimplify_secondIsEmpty() {
+        final String simplified = new VisibilitySimplifier().unionAndSimplify("u", "");
+        assertEquals("u", simplified);
+    }
+
+    @Test
+    public void unionAndSimplify_bothAreEmpty() {
+        final String simplified = new VisibilitySimplifier().unionAndSimplify("", "");
+        assertEquals("", simplified);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml b/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml
index 4839c04..f2e8cf9 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml
@@ -59,6 +59,11 @@ under the License.
                 </exclusion>
             </exclusions>
         </dependency>
+        
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.api.function</artifactId>
+        </dependency>
 
         <!-- 3rd Party Runtime Dependencies. -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java
index 2dc48f5..2d2bfa7 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/JoinResultUpdater.java
@@ -35,7 +35,10 @@ import org.apache.fluo.api.data.ColumnValue;
 import org.apache.fluo.api.data.RowColumn;
 import org.apache.fluo.api.data.Span;
 import org.apache.log4j.Logger;
-import org.apache.rya.accumulo.utils.VisibilitySimplifier;
+import org.apache.rya.api.function.join.IterativeJoin;
+import org.apache.rya.api.function.join.LazyJoiningIterator.Side;
+import org.apache.rya.api.function.join.LeftOuterJoin;
+import org.apache.rya.api.function.join.NaturalJoin;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.indexing.pcj.fluo.app.batch.AbstractBatchBindingSetUpdater;
 import org.apache.rya.indexing.pcj.fluo.app.batch.BatchInformation.Task;
@@ -47,9 +50,6 @@ import org.apache.rya.indexing.pcj.fluo.app.query.JoinMetadata;
 import org.apache.rya.indexing.pcj.fluo.app.query.MetadataCacheSupplier;
 import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
 import org.apache.rya.indexing.pcj.storage.accumulo.VisibilityBindingSetSerDe;
-import org.openrdf.query.Binding;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.impl.MapBindingSet;
 
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
@@ -90,21 +90,21 @@ public class JoinResultUpdater extends AbstractNodeUpdater {
 
         log.trace(
                 "Transaction ID: " + tx.getStartTimestamp() + "\n" +
-                "Join Node ID: " + joinMetadata.getNodeId() + "\n" +
-                "Child Node ID: " + childNodeId + "\n" +
-                "Child Binding Set:\n" + childBindingSet + "\n");
+                        "Join Node ID: " + joinMetadata.getNodeId() + "\n" +
+                        "Child Node ID: " + childNodeId + "\n" +
+                        "Child Binding Set:\n" + childBindingSet + "\n");
 
         // Figure out which join algorithm we are going to use.
         final IterativeJoin joinAlgorithm;
         switch(joinMetadata.getJoinType()) {
-        case NATURAL_JOIN:
-            joinAlgorithm = new NaturalJoin();
-            break;
-        case LEFT_OUTER_JOIN:
-            joinAlgorithm = new LeftOuterJoin();
-            break;
-        default:
-            throw new RuntimeException("Unsupported JoinType: " + joinMetadata.getJoinType());
+            case NATURAL_JOIN:
+                joinAlgorithm = new NaturalJoin();
+                break;
+            case LEFT_OUTER_JOIN:
+                joinAlgorithm = new LeftOuterJoin();
+                break;
+            default:
+                throw new RuntimeException("Unsupported JoinType: " + joinMetadata.getJoinType());
         }
 
         // Figure out which side of the join the new binding set appeared on.
@@ -120,10 +120,10 @@ public class JoinResultUpdater extends AbstractNodeUpdater {
         }
 
         // Iterates over the sibling node's BindingSets that join with the new binding set.
-        Set<VisibilityBindingSet> siblingBindingSets = new HashSet<>();
-        Span siblingSpan = getSpan(tx, childNodeId, childBindingSet, siblingId);
-        Column siblingColumn = getScanColumnFamily(siblingId);
-        Optional<RowColumn> rowColumn = fillSiblingBatch(tx, siblingSpan, siblingColumn, siblingBindingSets, joinMetadata.getJoinBatchSize());
+        final Set<VisibilityBindingSet> siblingBindingSets = new HashSet<>();
+        final Span siblingSpan = getSpan(tx, childNodeId, childBindingSet, siblingId);
+        final Column siblingColumn = getScanColumnFamily(siblingId);
+        final Optional<RowColumn> rowColumn = fillSiblingBatch(tx, siblingSpan, siblingColumn, siblingBindingSets, joinMetadata.getJoinBatchSize());
 
         // Iterates over the resulting BindingSets from the join.
         final Iterator<VisibilityBindingSet> newJoinResults;
@@ -148,7 +148,7 @@ public class JoinResultUpdater extends AbstractNodeUpdater {
 
                 log.trace(
                         "Transaction ID: " + tx.getStartTimestamp() + "\n" +
-                        "New Join Result:\n" + newJoinResult + "\n");
+                                "New Join Result:\n" + newJoinResult + "\n");
 
                 tx.set(resultRow, FluoQueryColumns.JOIN_BINDING_SET, nodeValueBytes);
             }
@@ -157,29 +157,21 @@ public class JoinResultUpdater extends AbstractNodeUpdater {
         // if batch limit met, there are additional entries to process
         // update the span and register updated batch job
         if (rowColumn.isPresent()) {
-            Span newSpan = AbstractBatchBindingSetUpdater.getNewSpan(rowColumn.get(), siblingSpan);
-            JoinBatchInformation joinBatch = JoinBatchInformation.builder()
-                .setBatchSize(joinMetadata.getJoinBatchSize())
-                .setBs(childBindingSet)
-                .setColumn(siblingColumn)
-                .setJoinType(joinMetadata.getJoinType())
-                .setSide(emittingSide)
-                .setSpan(newSpan)
-                .setTask(Task.Add)
-                .build();
+            final Span newSpan = AbstractBatchBindingSetUpdater.getNewSpan(rowColumn.get(), siblingSpan);
+            final JoinBatchInformation joinBatch = JoinBatchInformation.builder()
+                    .setBatchSize(joinMetadata.getJoinBatchSize())
+                    .setBs(childBindingSet)
+                    .setColumn(siblingColumn)
+                    .setJoinType(joinMetadata.getJoinType())
+                    .setSide(emittingSide)
+                    .setSpan(newSpan)
+                    .setTask(Task.Add)
+                    .build();
             BatchInformationDAO.addBatch(tx, joinMetadata.getNodeId(), joinBatch);
         }
     }
 
     /**
-     * The different sides a new binding set may appear on.
-     */
-    public static enum Side {
-        LEFT, RIGHT;
-    }
-
-
-    /**
      * Fetches batch to be processed by scanning over the Span specified by the
      * {@link JoinBatchInformation}. The number of results is less than or equal
      * to the batch size specified by the JoinBatchInformation.
@@ -190,17 +182,17 @@ public class JoinResultUpdater extends AbstractNodeUpdater {
      * @return Set - containing results of sibling scan.
      * @throws Exception
      */
-    private Optional<RowColumn> fillSiblingBatch(TransactionBase tx, Span siblingSpan, Column siblingColumn, Set<VisibilityBindingSet> bsSet, int batchSize) throws Exception {
+    private Optional<RowColumn> fillSiblingBatch(final TransactionBase tx, final Span siblingSpan, final Column siblingColumn, final Set<VisibilityBindingSet> bsSet, final int batchSize) throws Exception {
 
-        RowScanner rs = tx.scanner().over(siblingSpan).fetch(siblingColumn).byRow().build();
-        Iterator<ColumnScanner> colScannerIter = rs.iterator();
+        final RowScanner rs = tx.scanner().over(siblingSpan).fetch(siblingColumn).byRow().build();
+        final Iterator<ColumnScanner> colScannerIter = rs.iterator();
 
         boolean batchLimitMet = false;
         Bytes row = siblingSpan.getStart().getRow();
         while (colScannerIter.hasNext() && !batchLimitMet) {
-            ColumnScanner colScanner = colScannerIter.next();
+            final ColumnScanner colScanner = colScannerIter.next();
             row = colScanner.getRow();
-            Iterator<ColumnValue> iter = colScanner.iterator();
+            final Iterator<ColumnValue> iter = colScanner.iterator();
             while (iter.hasNext() && !batchLimitMet) {
                 bsSet.add(BS_SERDE.deserialize(iter.next().getValue()));
                 //check if batch size has been met and set flag if it has been met
@@ -271,7 +263,7 @@ public class JoinResultUpdater extends AbstractNodeUpdater {
     private VariableOrder removeBinIdFromVarOrder(VariableOrder varOrder) {
         List<String> varOrderList = varOrder.getVariableOrders();
         if(varOrderList.get(0).equals(IncrementalUpdateConstants.PERIODIC_BIN_ID)) {
-            List<String> updatedVarOrderList = Lists.newArrayList(varOrderList);
+            final List<String> updatedVarOrderList = Lists.newArrayList(varOrderList);
             updatedVarOrderList.remove(0);
             return new VariableOrder(updatedVarOrderList);
         } else {
@@ -350,168 +342,4 @@ public class JoinResultUpdater extends AbstractNodeUpdater {
 
         return column;
     }
-
-    /**
-     * Defines each of the cases that may generate new join results when
-     * iteratively computing a query's join node.
-     */
-    public static interface IterativeJoin {
-
-        /**
-         * Invoked when a new {@link VisibilityBindingSet} is emitted from the left child
-         * node of the join. The Fluo table is scanned for results on the right
-         * side that will be joined with the new result.
-         *
-         * @param newLeftResult - A new VisibilityBindingSet that has been emitted from
-         *   the left child node.
-         * @param rightResults - The right child node's binding sets that will
-         *   be joined with the new left result. (not null)
-         * @return The new BindingSet results for the join.
-         */
-        public Iterator<VisibilityBindingSet> newLeftResult(VisibilityBindingSet newLeftResult, Iterator<VisibilityBindingSet> rightResults);
-
-        /**
-         * Invoked when a new {@link VisibilityBindingSet} is emitted from the right child
-         * node of the join. The Fluo table is scanned for results on the left
-         * side that will be joined with the new result.
-         *
-         * @param leftResults - The left child node's binding sets that will be
-         *   joined with the new right result.
-         * @param newRightResult - A new BindingSet that has been emitted from
-         *   the right child node.
-         * @return The new BindingSet results for the join.
-         */
-        public Iterator<VisibilityBindingSet> newRightResult(Iterator<VisibilityBindingSet> leftResults, VisibilityBindingSet newRightResult);
-    }
-
-    /**
-     * Implements an {@link IterativeJoin} that uses the Natural Join algorithm
-     * defined by Relational Algebra.
-     * <p>
-     * This is how you combine {@code BindnigSet}s that may have common Binding
-     * names. When two Binding Sets are joined, any bindings that appear in both
-     * binding sets are only included once.
-     */
-    public static final class NaturalJoin implements IterativeJoin {
-        @Override
-        public Iterator<VisibilityBindingSet> newLeftResult(final VisibilityBindingSet newLeftResult, final Iterator<VisibilityBindingSet> rightResults) {
-            checkNotNull(newLeftResult);
-            checkNotNull(rightResults);
-
-            // Both sides are required, so if there are no right results, then do not emit anything.
-            return new LazyJoiningIterator(Side.LEFT, newLeftResult, rightResults);
-        }
-
-        @Override
-        public Iterator<VisibilityBindingSet> newRightResult(final Iterator<VisibilityBindingSet> leftResults, final VisibilityBindingSet newRightResult) {
-            checkNotNull(leftResults);
-            checkNotNull(newRightResult);
-
-            // Both sides are required, so if there are no left reuslts, then do not emit anything.
-            return new LazyJoiningIterator(Side.RIGHT, newRightResult, leftResults);
-        }
-    }
-
-    /**
-     * Implements an {@link IterativeJoin} that uses the Left Outer Join
-     * algorithm defined by Relational Algebra.
-     * <p>
-     * This is how you add optional information to a {@link BindingSet}. Left
-     * binding sets are emitted even if they do not join with anything on the right.
-     * However, right binding sets must be joined with a left binding set.
-     */
-    public static final class LeftOuterJoin implements IterativeJoin {
-        @Override
-        public Iterator<VisibilityBindingSet> newLeftResult(final VisibilityBindingSet newLeftResult, final Iterator<VisibilityBindingSet> rightResults) {
-            checkNotNull(newLeftResult);
-            checkNotNull(rightResults);
-
-            // If the required portion does not join with any optional portions,
-            // then emit a BindingSet that matches the new left result.
-            if(!rightResults.hasNext()) {
-                return Lists.<VisibilityBindingSet>newArrayList(newLeftResult).iterator();
-            }
-
-            // Otherwise, return an iterator that holds the new required result
-            // joined with the right results.
-            return new LazyJoiningIterator(Side.LEFT, newLeftResult, rightResults);
-        }
-
-        @Override
-        public Iterator<VisibilityBindingSet> newRightResult(final Iterator<VisibilityBindingSet> leftResults, final VisibilityBindingSet newRightResult) {
-            checkNotNull(leftResults);
-            checkNotNull(newRightResult);
-
-            // The right result is optional, so if it does not join with anything
-            // on the left, then do not emit anything.
-            return new LazyJoiningIterator(Side.RIGHT, newRightResult, leftResults);
-        }
-    }
-
-    /**
-     * Joins a {@link BindingSet} (which is new to the left or right side of a join)
-     * to all binding sets on the other side that join with it.
-     * <p>
-     * This is done lazily so that you don't have to load all of the BindingSets
-     * into memory at once.
-     */
-    private static final class LazyJoiningIterator implements Iterator<VisibilityBindingSet> {
-
-        private final Side newResultSide;
-        private final VisibilityBindingSet newResult;
-        private final Iterator<VisibilityBindingSet> joinedResults;
-
-        /**
-         * Constructs an instance of {@link LazyJoiningIterator}.
-         *
-         * @param newResultSide - Indicates which side of the join the {@code newResult} arrived on. (not null)
-         * @param newResult - A binding set that will be joined with some other binding sets. (not null)
-         * @param joinedResults - The binding sets that will be joined with {@code newResult}. (not null)
-         */
-        public LazyJoiningIterator(final Side newResultSide, final VisibilityBindingSet newResult, final Iterator<VisibilityBindingSet> joinedResults) {
-            this.newResultSide = checkNotNull(newResultSide);
-            this.newResult = checkNotNull(newResult);
-            this.joinedResults = checkNotNull(joinedResults);
-        }
-
-        @Override
-        public boolean hasNext() {
-            return joinedResults.hasNext();
-        }
-
-        @Override
-        public VisibilityBindingSet next() {
-            final MapBindingSet bs = new MapBindingSet();
-
-            for(final Binding binding : newResult) {
-                bs.addBinding(binding);
-            }
-
-            final VisibilityBindingSet joinResult = joinedResults.next();
-            for(final Binding binding : joinResult) {
-                bs.addBinding(binding);
-            }
-
-            // We want to make sure the visibilities are always written the same way,
-            // so figure out which are on the left side and which are on the right side.
-            final String leftVisi;
-            final String rightVisi;
-            if(newResultSide == Side.LEFT) {
-                leftVisi = newResult.getVisibility();
-                rightVisi = joinResult.getVisibility();
-            } else {
-                leftVisi = joinResult.getVisibility();
-                rightVisi = newResult.getVisibility();
-            }
-            final String visibility = VisibilitySimplifier.unionAndSimplify(leftVisi, rightVisi);
-
-            return new VisibilityBindingSet(bs, visibility);
-        }
-
-        @Override
-        public void remove() {
-            throw new UnsupportedOperationException("remove() is unsupported.");
-        }
-    }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchBindingSetUpdater.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchBindingSetUpdater.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchBindingSetUpdater.java
index 0d46e19..98f35cb 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchBindingSetUpdater.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchBindingSetUpdater.java
@@ -31,11 +31,11 @@ import org.apache.fluo.api.data.ColumnValue;
 import org.apache.fluo.api.data.RowColumn;
 import org.apache.fluo.api.data.Span;
 import org.apache.log4j.Logger;
+import org.apache.rya.api.function.join.IterativeJoin;
+import org.apache.rya.api.function.join.LazyJoiningIterator.Side;
+import org.apache.rya.api.function.join.LeftOuterJoin;
+import org.apache.rya.api.function.join.NaturalJoin;
 import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.IterativeJoin;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.LeftOuterJoin;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.NaturalJoin;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.Side;
 import org.apache.rya.indexing.pcj.fluo.app.batch.BatchInformation.Task;
 import org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns;
 import org.apache.rya.indexing.pcj.fluo.app.query.JoinMetadata;
@@ -66,12 +66,12 @@ public class JoinBatchBindingSetUpdater extends AbstractBatchBindingSetUpdater {
      * @throws Exception
      */
     @Override
-    public void processBatch(TransactionBase tx, Bytes row, BatchInformation batch) throws Exception {
+    public void processBatch(final TransactionBase tx, final Bytes row, final BatchInformation batch) throws Exception {
         super.processBatch(tx, row, batch);
-        String nodeId = BatchRowKeyUtil.getNodeId(row);
+        final String nodeId = BatchRowKeyUtil.getNodeId(row);
         Preconditions.checkArgument(batch instanceof JoinBatchInformation);
-        JoinBatchInformation joinBatch = (JoinBatchInformation) batch;
-        Task task = joinBatch.getTask();
+        final JoinBatchInformation joinBatch = (JoinBatchInformation) batch;
+        final Task task = joinBatch.getTask();
 
         // Figure out which join algorithm we are going to use.
         final IterativeJoin joinAlgorithm;
@@ -86,12 +86,12 @@ public class JoinBatchBindingSetUpdater extends AbstractBatchBindingSetUpdater {
             throw new RuntimeException("Unsupported JoinType: " + joinBatch.getJoinType());
         }
 
-        Set<VisibilityBindingSet> bsSet = new HashSet<>();
-        Optional<RowColumn> rowCol = fillSiblingBatch(tx, joinBatch, bsSet);
+        final Set<VisibilityBindingSet> bsSet = new HashSet<>();
+        final Optional<RowColumn> rowCol = fillSiblingBatch(tx, joinBatch, bsSet);
 
         // Iterates over the resulting BindingSets from the join.
         final Iterator<VisibilityBindingSet> newJoinResults;
-        VisibilityBindingSet bs = joinBatch.getBs();
+        final VisibilityBindingSet bs = joinBatch.getBs();
         if (joinBatch.getSide() == Side.LEFT) {
             newJoinResults = joinAlgorithm.newLeftResult(bs, bsSet.iterator());
         } else {
@@ -104,7 +104,7 @@ public class JoinBatchBindingSetUpdater extends AbstractBatchBindingSetUpdater {
         while (newJoinResults.hasNext()) {
             final VisibilityBindingSet newJoinResult = newJoinResults.next();
             //create BindingSet value
-            Bytes bsBytes = BS_SERDE.serialize(newJoinResult);
+            final Bytes bsBytes = BS_SERDE.serialize(newJoinResult);
             //make rowId
             Bytes rowKey = BindingHashShardingFunction.addShard(nodeId, joinVarOrder, newJoinResult);
             final Column col = FluoQueryColumns.JOIN_BINDING_SET;
@@ -114,14 +114,14 @@ public class JoinBatchBindingSetUpdater extends AbstractBatchBindingSetUpdater {
         // if batch limit met, there are additional entries to process
         // update the span and register updated batch job
         if (rowCol.isPresent()) {
-            Span newSpan = getNewSpan(rowCol.get(), joinBatch.getSpan());
+            final Span newSpan = getNewSpan(rowCol.get(), joinBatch.getSpan());
             joinBatch.setSpan(newSpan);
             BatchInformationDAO.addBatch(tx, nodeId, joinBatch);
         }
 
     }
 
-    private void processTask(TransactionBase tx, Task task, Bytes row, Column column, Bytes value) {
+    private void processTask(final TransactionBase tx, final Task task, final Bytes row, final Column column, final Bytes value) {
         switch (task) {
         case Add:
             tx.set(row, column, value);
@@ -149,21 +149,21 @@ public class JoinBatchBindingSetUpdater extends AbstractBatchBindingSetUpdater {
      * @return Set - containing results of sibling scan.
      * @throws Exception
      */
-    private Optional<RowColumn> fillSiblingBatch(TransactionBase tx, JoinBatchInformation batch, Set<VisibilityBindingSet> bsSet) throws Exception {
+    private Optional<RowColumn> fillSiblingBatch(final TransactionBase tx, final JoinBatchInformation batch, final Set<VisibilityBindingSet> bsSet) throws Exception {
 
-        Span span = batch.getSpan();
-        Column column = batch.getColumn();
-        int batchSize = batch.getBatchSize();
+        final Span span = batch.getSpan();
+        final Column column = batch.getColumn();
+        final int batchSize = batch.getBatchSize();
 
-        RowScanner rs = tx.scanner().over(span).fetch(column).byRow().build();
-        Iterator<ColumnScanner> colScannerIter = rs.iterator();
+        final RowScanner rs = tx.scanner().over(span).fetch(column).byRow().build();
+        final Iterator<ColumnScanner> colScannerIter = rs.iterator();
 
         boolean batchLimitMet = false;
         Bytes row = span.getStart().getRow();
         while (colScannerIter.hasNext() && !batchLimitMet) {
-            ColumnScanner colScanner = colScannerIter.next();
+            final ColumnScanner colScanner = colScannerIter.next();
             row = colScanner.getRow();
-            Iterator<ColumnValue> iter = colScanner.iterator();
+            final Iterator<ColumnValue> iter = colScanner.iterator();
             while (iter.hasNext()) {
                 if (bsSet.size() >= batchSize) {
                     batchLimitMet = true;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchInformation.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchInformation.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchInformation.java
index d46cccd..ace9e76 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchInformation.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/JoinBatchInformation.java
@@ -21,8 +21,8 @@ import java.util.Objects;
 
 import org.apache.fluo.api.data.Column;
 import org.apache.fluo.api.data.Span;
+import org.apache.rya.api.function.join.LazyJoiningIterator.Side;
 import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.Side;
 import org.apache.rya.indexing.pcj.fluo.app.query.JoinMetadata.JoinType;
 import org.openrdf.query.Binding;
 
@@ -49,9 +49,9 @@ import org.openrdf.query.Binding;
 public class JoinBatchInformation extends AbstractSpanBatchInformation {
 
     private static final BatchBindingSetUpdater updater = new JoinBatchBindingSetUpdater();
-    private VisibilityBindingSet bs; //update for join child indicated by side
-    private Side side;  //join child that was updated by bs
-    private JoinType join;
+    private final VisibilityBindingSet bs; //update for join child indicated by side
+    private final Side side;  //join child that was updated by bs
+    private final JoinType join;
     /**
      * @param batchSize - batch size that Tasks are performed in
      * @param task - Add, Delete, or Update
@@ -61,14 +61,14 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
      * @param side - The side of the child that the VisibilityBindingSet update occurred at
      * @param join - JoinType (left, right, natural inner)
      */
-    public JoinBatchInformation(int batchSize, Task task, Column column, Span span, VisibilityBindingSet bs, Side side, JoinType join) {
+    public JoinBatchInformation(final int batchSize, final Task task, final Column column, final Span span, final VisibilityBindingSet bs, final Side side, final JoinType join) {
         super(batchSize, task, column, span);
         this.bs = Objects.requireNonNull(bs);
         this.side = Objects.requireNonNull(side);
         this.join = Objects.requireNonNull(join);
     }
 
-    public JoinBatchInformation(Task task, Column column, Span span, VisibilityBindingSet bs, Side side, JoinType join) {
+    public JoinBatchInformation(final Task task, final Column column, final Span span, final VisibilityBindingSet bs, final Side side, final JoinType join) {
         this(DEFAULT_BATCH_SIZE, task, column, span, bs, side, join);
     }
 
@@ -123,7 +123,7 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
     }
 
     @Override
-    public boolean equals(Object other) {
+    public boolean equals(final Object other) {
         if (this == other) {
             return true;
         }
@@ -132,9 +132,9 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
             return false;
         }
 
-        JoinBatchInformation batch = (JoinBatchInformation) other;
-        return super.equals(other) &&  Objects.equals(this.bs, batch.bs) && Objects.equals(this.join, batch.join)
-                && Objects.equals(this.side, batch.side);
+        final JoinBatchInformation batch = (JoinBatchInformation) other;
+        return super.equals(other) &&  Objects.equals(bs, batch.bs) && Objects.equals(join, batch.join)
+                && Objects.equals(side, batch.side);
     }
 
     @Override
@@ -160,7 +160,7 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
         /**
          * @param batchSize - batch size that {@link Task}s are performed in
          */
-        public Builder setBatchSize(int batchSize) {
+        public Builder setBatchSize(final int batchSize) {
             this.batchSize = batchSize;
             return this;
         }
@@ -168,7 +168,7 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
         /**
          * @param task - Task performed (Add, Delete, Update)
          */
-        public Builder setTask(Task task) {
+        public Builder setTask(final Task task) {
             this.task = task;
             return this;
         }
@@ -176,7 +176,7 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
         /**
          * @param column - Column of join child to be scanned
          */
-        public Builder setColumn(Column column) {
+        public Builder setColumn(final Column column) {
             this.column = column;
             return this;
         }
@@ -188,7 +188,7 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
          * the common variables of the left and right join children.
          * @param span - Span over join child to be scanned
          */
-        public Builder setSpan(Span span) {
+        public Builder setSpan(final Span span) {
             this.span = span;
             return this;
         }
@@ -198,7 +198,7 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
          * by Side.
          * @param bs - BindingSet update of join child to be joined with results of scan
          */
-        public Builder setBs(VisibilityBindingSet bs) {
+        public Builder setBs(final VisibilityBindingSet bs) {
             this.bs = bs;
             return this;
         }
@@ -206,7 +206,7 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
         /**
          * @param join - JoinType (left, right, natural inner)
          */
-        public Builder setJoinType(JoinType join) {
+        public Builder setJoinType(final JoinType join) {
             this.join = join;
             return this;
         }
@@ -215,7 +215,7 @@ public class JoinBatchInformation extends AbstractSpanBatchInformation {
          * Indicates the join child corresponding to the VisibilityBindingSet update
          * @param side - side of join the child BindingSet update appeared at
          */
-        public Builder setSide(Side side) {
+        public Builder setSide(final Side side) {
             this.side = side;
             return this;
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/JoinBatchInformationTypeAdapter.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/JoinBatchInformationTypeAdapter.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/JoinBatchInformationTypeAdapter.java
index 0687a3e..1e7eba0 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/JoinBatchInformationTypeAdapter.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/JoinBatchInformationTypeAdapter.java
@@ -22,8 +22,8 @@ import java.lang.reflect.Type;
 import org.apache.fluo.api.data.Column;
 import org.apache.fluo.api.data.RowColumn;
 import org.apache.fluo.api.data.Span;
+import org.apache.rya.api.function.join.LazyJoiningIterator.Side;
 import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.Side;
 import org.apache.rya.indexing.pcj.fluo.app.batch.BatchInformation.Task;
 import org.apache.rya.indexing.pcj.fluo.app.batch.JoinBatchInformation;
 import org.apache.rya.indexing.pcj.fluo.app.query.JoinMetadata.JoinType;
@@ -49,42 +49,42 @@ public class JoinBatchInformationTypeAdapter implements JsonSerializer<JoinBatch
     private static final VisibilityBindingSetStringConverter converter = new VisibilityBindingSetStringConverter();
 
     @Override
-    public JsonElement serialize(JoinBatchInformation batch, Type typeOfSrc, JsonSerializationContext context) {
-        JsonObject result = new JsonObject();
+    public JsonElement serialize(final JoinBatchInformation batch, final Type typeOfSrc, final JsonSerializationContext context) {
+        final JsonObject result = new JsonObject();
         result.add("class", new JsonPrimitive(batch.getClass().getName()));
         result.add("batchSize", new JsonPrimitive(batch.getBatchSize()));
         result.add("task", new JsonPrimitive(batch.getTask().name()));
-        Column column = batch.getColumn();
+        final Column column = batch.getColumn();
         result.add("column", new JsonPrimitive(column.getsFamily() + "\u0000" + column.getsQualifier()));
-        Span span = batch.getSpan();
+        final Span span = batch.getSpan();
         result.add("span", new JsonPrimitive(span.getStart().getsRow() + "\u0000" + span.getEnd().getsRow()));
         result.add("startInc", new JsonPrimitive(span.isStartInclusive()));
         result.add("endInc", new JsonPrimitive(span.isEndInclusive()));
         result.add("side", new JsonPrimitive(batch.getSide().name()));
         result.add("joinType", new JsonPrimitive(batch.getJoinType().name()));
-        String updateVarOrderString = Joiner.on(";").join(batch.getBs().getBindingNames());
-        VariableOrder updateVarOrder = new VariableOrder(updateVarOrderString);
+        final String updateVarOrderString = Joiner.on(";").join(batch.getBs().getBindingNames());
+        final VariableOrder updateVarOrder = new VariableOrder(updateVarOrderString);
         result.add("bindingSet", new JsonPrimitive(converter.convert(batch.getBs(), updateVarOrder)));
         result.add("updateVarOrder", new JsonPrimitive(updateVarOrderString));
         return result;
     }
 
     @Override
-    public JoinBatchInformation deserialize(JsonElement element, Type typeOfT, JsonDeserializationContext context)
+    public JoinBatchInformation deserialize(final JsonElement element, final Type typeOfT, final JsonDeserializationContext context)
             throws JsonParseException {
-        JsonObject json = element.getAsJsonObject();
-        int batchSize = json.get("batchSize").getAsInt();
-        Task task = Task.valueOf(json.get("task").getAsString());
-        String[] colArray = json.get("column").getAsString().split("\u0000");
-        Column column = new Column(colArray[0], colArray[1]);
-        String[] rows = json.get("span").getAsString().split("\u0000");
-        boolean startInc = json.get("startInc").getAsBoolean();
-        boolean endInc = json.get("endInc").getAsBoolean();
-        Span span = new Span(new RowColumn(rows[0]), startInc, new RowColumn(rows[1]), endInc);
-        VariableOrder updateVarOrder = new VariableOrder(json.get("updateVarOrder").getAsString());
-        VisibilityBindingSet bs = converter.convert(json.get("bindingSet").getAsString(), updateVarOrder);
-        Side side = Side.valueOf(json.get("side").getAsString());
-        JoinType join = JoinType.valueOf(json.get("joinType").getAsString());
+        final JsonObject json = element.getAsJsonObject();
+        final int batchSize = json.get("batchSize").getAsInt();
+        final Task task = Task.valueOf(json.get("task").getAsString());
+        final String[] colArray = json.get("column").getAsString().split("\u0000");
+        final Column column = new Column(colArray[0], colArray[1]);
+        final String[] rows = json.get("span").getAsString().split("\u0000");
+        final boolean startInc = json.get("startInc").getAsBoolean();
+        final boolean endInc = json.get("endInc").getAsBoolean();
+        final Span span = new Span(new RowColumn(rows[0]), startInc, new RowColumn(rows[1]), endInc);
+        final VariableOrder updateVarOrder = new VariableOrder(json.get("updateVarOrder").getAsString());
+        final VisibilityBindingSet bs = converter.convert(json.get("bindingSet").getAsString(), updateVarOrder);
+        final Side side = Side.valueOf(json.get("side").getAsString());
+        final JoinType join = JoinType.valueOf(json.get("joinType").getAsString());
         return JoinBatchInformation.builder().setBatchSize(batchSize).setTask(task).setSpan(span).setColumn(column).setBs(bs)
                .setSide(side).setJoinType(join).build();
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/ExporterManager.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/ExporterManager.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/ExporterManager.java
index 7cfa73c..c58a27b 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/ExporterManager.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/export/ExporterManager.java
@@ -49,11 +49,11 @@ public class ExporterManager implements AutoCloseable {
 
     private static final VisibilityBindingSetSerDe BS_SERDE = new VisibilityBindingSetSerDe();
     private static final RyaSubGraphKafkaSerDe SG_SERDE = new RyaSubGraphKafkaSerDe();
-    private Map<String, String> simplifiedVisibilities = new HashMap<>();
+    private final Map<String, String> simplifiedVisibilities = new HashMap<>();
     
-    private Map<QueryType, Map<ExportStrategy, IncrementalResultExporter>> exporters;
+    private final Map<QueryType, Map<ExportStrategy, IncrementalResultExporter>> exporters;
     
-    private ExporterManager(Map<QueryType, Map<ExportStrategy, IncrementalResultExporter>> exporters) {
+    private ExporterManager(final Map<QueryType, Map<ExportStrategy, IncrementalResultExporter>> exporters) {
         this.exporters = Preconditions.checkNotNull(exporters);
     }
     
@@ -73,9 +73,9 @@ public class ExporterManager implements AutoCloseable {
      * @param data - Serialized result to be exported
      * @throws ResultExportException 
      */
-    public void export(QueryType type, Set<ExportStrategy> strategies, String queryId, Bytes data) throws ResultExportException {
+    public void export(final QueryType type, final Set<ExportStrategy> strategies, final String queryId, final Bytes data) throws ResultExportException {
         
-        String pcjId = FluoQueryUtils.convertFluoQueryIdToPcjId(queryId);
+        final String pcjId = FluoQueryUtils.convertFluoQueryIdToPcjId(queryId);
         
         if(type == QueryType.CONSTRUCT) {
             exportSubGraph(exporters.get(type), strategies, pcjId, data);
@@ -93,21 +93,21 @@ public class ExporterManager implements AutoCloseable {
      * @param data - serialized BindingSet result
      * @throws ResultExportException
      */
-    private void exportBindingSet(Map<ExportStrategy, IncrementalResultExporter> exporters, Set<ExportStrategy> strategies, String pcjId, Bytes data) throws ResultExportException {
+    private void exportBindingSet(final Map<ExportStrategy, IncrementalResultExporter> exporters, final Set<ExportStrategy> strategies, final String pcjId, final Bytes data) throws ResultExportException {
         VisibilityBindingSet bs;
         try {
             bs = BS_SERDE.deserialize(data);
             simplifyVisibilities(bs);
-        } catch (Exception e) {
+        } catch (final Exception e) {
             throw new ResultExportException("Unable to deserialize the given BindingSet.", e);
         }
             
         try{
-            for(ExportStrategy strategy: strategies) {
-                IncrementalBindingSetExporter exporter = (IncrementalBindingSetExporter) exporters.get(strategy);
+            for(final ExportStrategy strategy: strategies) {
+                final IncrementalBindingSetExporter exporter = (IncrementalBindingSetExporter) exporters.get(strategy);
                 exporter.export(pcjId, bs);
             }
-        } catch (Exception e) {
+        } catch (final Exception e) {
             throw new ResultExportException("Unable to export the given BindingSet " + bs + " with the given set of ExportStrategies " + strategies, e);
         }
     }
@@ -120,27 +120,27 @@ public class ExporterManager implements AutoCloseable {
      * @param data - serialized RyaSubGraph result
      * @throws ResultExportException
      */
-    private void exportSubGraph(Map<ExportStrategy, IncrementalResultExporter> exporters, Set<ExportStrategy> strategies, String pcjId, Bytes data) throws ResultExportException {
-        RyaSubGraph subGraph = SG_SERDE.fromBytes(data.toArray());
+    private void exportSubGraph(final Map<ExportStrategy, IncrementalResultExporter> exporters, final Set<ExportStrategy> strategies, final String pcjId, final Bytes data) throws ResultExportException {
+        final RyaSubGraph subGraph = SG_SERDE.fromBytes(data.toArray());
         
         try {
             simplifyVisibilities(subGraph);
-        } catch (UnsupportedEncodingException e) {
+        } catch (final UnsupportedEncodingException e) {
             throw new ResultExportException("Undable to deserialize provided RyaSubgraph", e);
         }
         
         try {
-            for (ExportStrategy strategy : strategies) {
-                IncrementalRyaSubGraphExporter exporter = (IncrementalRyaSubGraphExporter) exporters.get(strategy);
+            for (final ExportStrategy strategy : strategies) {
+                final IncrementalRyaSubGraphExporter exporter = (IncrementalRyaSubGraphExporter) exporters.get(strategy);
                 exporter.export(pcjId, subGraph);
             }
-        } catch (Exception e) {
+        } catch (final Exception e) {
             throw new ResultExportException(
                     "Unable to export the given subgraph " + subGraph + " using all of the ExportStrategies " + strategies);
         }
     }
     
-    private void simplifyVisibilities(VisibilityBindingSet result) {
+    private void simplifyVisibilities(final VisibilityBindingSet result) {
         // Simplify the result's visibilities.
         final String visibility = result.getVisibility();
         if(!simplifiedVisibilities.containsKey(visibility)) {
@@ -150,19 +150,19 @@ public class ExporterManager implements AutoCloseable {
         result.setVisibility( simplifiedVisibilities.get(visibility) );
     }
     
-    private void simplifyVisibilities(RyaSubGraph subgraph) throws UnsupportedEncodingException {
-        Set<RyaStatement> statements = subgraph.getStatements();
+    private void simplifyVisibilities(final RyaSubGraph subgraph) throws UnsupportedEncodingException {
+        final Set<RyaStatement> statements = subgraph.getStatements();
         if (statements.size() > 0) {
-            byte[] visibilityBytes = statements.iterator().next().getColumnVisibility();
+            final byte[] visibilityBytes = statements.iterator().next().getColumnVisibility();
             // Simplify the result's visibilities and cache new simplified
             // visibilities
-            String visibility = new String(visibilityBytes, "UTF-8");
+            final String visibility = new String(visibilityBytes, "UTF-8");
             if (!simplifiedVisibilities.containsKey(visibility)) {
-                String simplified = VisibilitySimplifier.simplify(visibility);
+                final String simplified = VisibilitySimplifier.simplify(visibility);
                 simplifiedVisibilities.put(visibility, simplified);
             }
 
-            for (RyaStatement statement : statements) {
+            for (final RyaStatement statement : statements) {
                 statement.setColumnVisibility(simplifiedVisibilities.get(visibility).getBytes("UTF-8"));
             }
             
@@ -172,25 +172,25 @@ public class ExporterManager implements AutoCloseable {
     
     public static class Builder {
         
-        private Map<QueryType, Map<ExportStrategy, IncrementalResultExporter>> exporters = new HashMap<>();
+        private final Map<QueryType, Map<ExportStrategy, IncrementalResultExporter>> exporters = new HashMap<>();
         
         /**
          * Add an {@link IncrementalResultExporter} to be used by this ExporterManager for exporting results
          * @param exporter - IncrementalResultExporter for exporting query results
          * @return - Builder for chaining method calls
          */
-        public Builder addIncrementalResultExporter(IncrementalResultExporter exporter) {
+        public Builder addIncrementalResultExporter(final IncrementalResultExporter exporter) {
             
-            Set<QueryType> types = exporter.getQueryTypes();
-            ExportStrategy strategy = exporter.getExportStrategy();
+            final Set<QueryType> types = exporter.getQueryTypes();
+            final ExportStrategy strategy = exporter.getExportStrategy();
             
-            for (QueryType type : types) {
+            for (final QueryType type : types) {
                 if (!exporters.containsKey(type)) {
-                    Map<ExportStrategy, IncrementalResultExporter> exportMap = new HashMap<>();
+                    final Map<ExportStrategy, IncrementalResultExporter> exportMap = new HashMap<>();
                     exportMap.put(strategy, exporter);
                     exporters.put(type, exportMap);
                 } else {
-                    Map<ExportStrategy, IncrementalResultExporter> exportMap = exporters.get(type);
+                    final Map<ExportStrategy, IncrementalResultExporter> exportMap = exporters.get(type);
                     if (!exportMap.containsKey(strategy)) {
                         exportMap.put(strategy, exporter);
                     }
@@ -212,10 +212,10 @@ public class ExporterManager implements AutoCloseable {
     @Override
     public void close() throws Exception {
         
-        Collection<Map<ExportStrategy, IncrementalResultExporter>> values = exporters.values();
+        final Collection<Map<ExportStrategy, IncrementalResultExporter>> values = exporters.values();
         
-        for(Map<ExportStrategy, IncrementalResultExporter> map: values) {
-            for(IncrementalResultExporter exporter: map.values()) {
+        for(final Map<ExportStrategy, IncrementalResultExporter> map: values) {
+            for(final IncrementalResultExporter exporter: map.values()) {
                 exporter.close();
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/IterativeJoinTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/IterativeJoinTest.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/IterativeJoinTest.java
deleted file mode 100644
index e1324c7..0000000
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/IterativeJoinTest.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.indexing.pcj.fluo.app;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.IterativeJoin;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.LeftOuterJoin;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.NaturalJoin;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameter;
-import org.junit.runners.Parameterized.Parameters;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.impl.MapBindingSet;
-
-/**
- * Tests the methods of {@link IterativeJoin}.
- */
-@RunWith(Parameterized.class)
-public class IterativeJoinTest {
-
-    @Parameters
-    public static Collection<Object[]> data() {
-        return Arrays.asList(new Object[][] {
-            {new NaturalJoin()},
-            {new LeftOuterJoin()}
-           });
-    }
-
-    @Parameter
-    public IterativeJoin join;
-
-    /**
-     * This test ensures the same binding sets are created as the result of a
-     * {@link IterativeJoin} regardless of which side the notification is triggered on.
-     */
-    @Test
-    public void naturalJoin_sideDoesNotMatter() {
-        // Create the binding sets that will be joined.
-        final ValueFactory vf = new ValueFactoryImpl();
-
-        final MapBindingSet bs1 = new MapBindingSet();
-        bs1.addBinding("id", vf.createLiteral("some_uid"));
-        bs1.addBinding("name", vf.createLiteral("Alice"));
-        final VisibilityBindingSet vbs1 = new VisibilityBindingSet(bs1, "a");
-
-        final MapBindingSet bs2 = new MapBindingSet();
-        bs2.addBinding("id", vf.createLiteral("some_uid"));
-        bs2.addBinding("hair", vf.createLiteral("brown"));
-        final VisibilityBindingSet vbs2 = new VisibilityBindingSet(bs2, "b");
-
-        // new vbs1 shows up on the left, matches vbs2 on the right
-        final Iterator<VisibilityBindingSet> newLeftIt = join.newLeftResult(vbs1, Collections.singleton(vbs2).iterator());
-        final VisibilityBindingSet newLeftResult = newLeftIt.next();
-
-        // new vbs2 shows up on the right, matches vbs1 on the left
-        final Iterator<VisibilityBindingSet> newRightIt = join.newRightResult(Collections.singleton(vbs1).iterator(), vbs2);
-        final VisibilityBindingSet newRightResult = newRightIt.next();
-
-        // Ensure those two results are the same.
-        assertEquals(newLeftResult, newRightResult);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/LeftOuterJoinTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/LeftOuterJoinTest.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/LeftOuterJoinTest.java
deleted file mode 100644
index 54051ab..0000000
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/LeftOuterJoinTest.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.indexing.pcj.fluo.app;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.IterativeJoin;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.LeftOuterJoin;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.impl.MapBindingSet;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
-/**
- * Tests the methods of {@link LeftOuterJoin}.
- */
-public class LeftOuterJoinTest {
-
-    private final ValueFactory vf = new ValueFactoryImpl();
-
-    @Test
-    public void newLeftResult_noRightMatches() {
-        final IterativeJoin leftOuterJoin = new LeftOuterJoin();
-
-        // There is a new left result.
-        final MapBindingSet mapLeftResult = new MapBindingSet();
-        mapLeftResult.addBinding("name", vf.createLiteral("Bob"));
-        final VisibilityBindingSet newLeftResult = new VisibilityBindingSet(mapLeftResult);
-
-        // There are no right results that join with the left result.
-        final Iterator<VisibilityBindingSet> rightResults= new ArrayList<VisibilityBindingSet>().iterator();
-
-        // Therefore, the left result is a new join result.
-        final Iterator<VisibilityBindingSet> newJoinResultsIt = leftOuterJoin.newLeftResult(newLeftResult, rightResults);
-
-        final Set<BindingSet> newJoinResults = new HashSet<>();
-        while(newJoinResultsIt.hasNext()) {
-            newJoinResults.add( newJoinResultsIt.next() );
-        }
-
-        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet( newLeftResult );
-
-        assertEquals(expected, newJoinResults);
-    }
-
-    @Test
-    public void newLeftResult_joinsWithRightResults() {
-        final IterativeJoin leftOuterJoin = new LeftOuterJoin();
-
-        // There is a new left result.
-        final MapBindingSet mapLeftResult = new MapBindingSet();
-        mapLeftResult.addBinding("name", vf.createLiteral("Bob"));
-        mapLeftResult.addBinding("height", vf.createLiteral("5'9\""));
-        final VisibilityBindingSet newLeftResult = new VisibilityBindingSet(mapLeftResult);
-
-        // There are a few right results that join with the left result.
-        final MapBindingSet nameAge = new MapBindingSet();
-        nameAge.addBinding("name", vf.createLiteral("Bob"));
-        nameAge.addBinding("age", vf.createLiteral(56));
-        final VisibilityBindingSet visiAge = new VisibilityBindingSet(nameAge);
-
-        final MapBindingSet nameHair = new MapBindingSet();
-        nameHair.addBinding("name", vf.createLiteral("Bob"));
-        nameHair.addBinding("hairColor", vf.createLiteral("Brown"));
-        final VisibilityBindingSet visiHair = new VisibilityBindingSet(nameHair);
-
-        final Iterator<VisibilityBindingSet> rightResults = Lists.<VisibilityBindingSet>newArrayList(visiAge, visiHair).iterator();
-
-        // Therefore, there are a few new join results that mix the two together.
-        final Iterator<VisibilityBindingSet> newJoinResultsIt = leftOuterJoin.newLeftResult(newLeftResult, rightResults);
-
-        final Set<BindingSet> newJoinResults = new HashSet<>();
-        while(newJoinResultsIt.hasNext()) {
-            newJoinResults.add( newJoinResultsIt.next() );
-        }
-
-        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet();
-        final MapBindingSet nameHeightAge = new MapBindingSet();
-        nameHeightAge.addBinding("name", vf.createLiteral("Bob"));
-        nameHeightAge.addBinding("height", vf.createLiteral("5'9\""));
-        nameHeightAge.addBinding("age", vf.createLiteral(56));
-        expected.add(new VisibilityBindingSet(nameHeightAge));
-
-        final MapBindingSet nameHeightHair = new MapBindingSet();
-        nameHeightHair.addBinding("name", vf.createLiteral("Bob"));
-        nameHeightHair.addBinding("height", vf.createLiteral("5'9\""));
-        nameHeightHair.addBinding("hairColor", vf.createLiteral("Brown"));
-        expected.add(new VisibilityBindingSet(nameHeightHair));
-
-        assertEquals(expected, newJoinResults);
-    }
-
-    @Test
-    public void newRightResult_noLeftMatches() {
-        final IterativeJoin leftOuterJoin = new LeftOuterJoin();
-
-        // There are no left results.
-        final Iterator<VisibilityBindingSet> leftResults= new ArrayList<VisibilityBindingSet>().iterator();
-
-        // There is a new right result.
-        final MapBindingSet newRightResult = new MapBindingSet();
-        newRightResult.addBinding("name", vf.createLiteral("Bob"));
-
-        // Therefore, there are no new join results.
-        final Iterator<VisibilityBindingSet> newJoinResultsIt = leftOuterJoin.newRightResult(leftResults, new VisibilityBindingSet(newRightResult));
-        assertFalse( newJoinResultsIt.hasNext() );
-    }
-
-    @Test
-    public void newRightResult_joinsWithLeftResults() {
-        final IterativeJoin leftOuterJoin = new LeftOuterJoin();
-
-        // There are a few left results that join with the new right result.
-        final MapBindingSet nameAge = new MapBindingSet();
-        nameAge.addBinding("name", vf.createLiteral("Bob"));
-        nameAge.addBinding("age", vf.createLiteral(56));
-
-        final MapBindingSet nameHair = new MapBindingSet();
-        nameHair.addBinding("name", vf.createLiteral("Bob"));
-        nameHair.addBinding("hairColor", vf.createLiteral("Brown"));
-
-        final Iterator<VisibilityBindingSet> leftResults = Lists.<VisibilityBindingSet>newArrayList(
-                new VisibilityBindingSet(nameAge),
-                new VisibilityBindingSet(nameHair)).iterator();
-
-        // There is a new right result.
-        final MapBindingSet newRightResult = new MapBindingSet();
-        newRightResult.addBinding("name", vf.createLiteral("Bob"));
-        newRightResult.addBinding("height", vf.createLiteral("5'9\""));
-
-        // Therefore, there are a few new join results that mix the two together.
-        final Iterator<VisibilityBindingSet> newJoinResultsIt = leftOuterJoin.newRightResult(leftResults, new VisibilityBindingSet(newRightResult));
-
-        final Set<BindingSet> newJoinResults = new HashSet<>();
-        while(newJoinResultsIt.hasNext()) {
-            newJoinResults.add( newJoinResultsIt.next() );
-        }
-
-        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet();
-        final MapBindingSet nameHeightAge = new MapBindingSet();
-        nameHeightAge.addBinding("name", vf.createLiteral("Bob"));
-        nameHeightAge.addBinding("height", vf.createLiteral("5'9\""));
-        nameHeightAge.addBinding("age", vf.createLiteral(56));
-        expected.add(new VisibilityBindingSet(nameHeightAge));
-
-        final MapBindingSet nameHeightHair = new MapBindingSet();
-        nameHeightHair.addBinding("name", vf.createLiteral("Bob"));
-        nameHeightHair.addBinding("height", vf.createLiteral("5'9\""));
-        nameHeightHair.addBinding("hairColor", vf.createLiteral("Brown"));
-        expected.add(new VisibilityBindingSet(nameHeightHair));
-
-        assertEquals(expected, newJoinResults);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/NaturalJoinTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/NaturalJoinTest.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/NaturalJoinTest.java
deleted file mode 100644
index c41f637..0000000
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/NaturalJoinTest.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.indexing.pcj.fluo.app;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.IterativeJoin;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.NaturalJoin;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.impl.MapBindingSet;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
-/**
- * Tests the methods of {@link NaturalJoin}.
- */
-public class NaturalJoinTest {
-
-    private final ValueFactory vf = new ValueFactoryImpl();
-
-    @Test
-    public void newLeftResult_noRightMatches() {
-        final IterativeJoin naturalJoin = new NaturalJoin();
-
-        // There is a new left result.
-        final MapBindingSet newLeftResult = new MapBindingSet();
-        newLeftResult.addBinding("name", vf.createLiteral("Bob"));
-
-        // There are no right results that join with the left result.
-        final Iterator<VisibilityBindingSet> rightResults= new ArrayList<VisibilityBindingSet>().iterator();
-
-        // Therefore, the left result is a new join result.
-        final Iterator<VisibilityBindingSet> newJoinResultsIt = naturalJoin.newLeftResult(new VisibilityBindingSet(newLeftResult), rightResults);
-        assertFalse( newJoinResultsIt.hasNext() );
-    }
-
-    @Test
-    public void newLeftResult_joinsWithRightResults() {
-        final IterativeJoin naturalJoin = new NaturalJoin();
-
-        // There is a new left result.
-        final MapBindingSet newLeftResult = new MapBindingSet();
-        newLeftResult.addBinding("name", vf.createLiteral("Bob"));
-        newLeftResult.addBinding("height", vf.createLiteral("5'9\""));
-
-        // There are a few right results that join with the left result.
-        final MapBindingSet nameAge = new MapBindingSet();
-        nameAge.addBinding("name", vf.createLiteral("Bob"));
-        nameAge.addBinding("age", vf.createLiteral(56));
-
-        final MapBindingSet nameHair = new MapBindingSet();
-        nameHair.addBinding("name", vf.createLiteral("Bob"));
-        nameHair.addBinding("hairColor", vf.createLiteral("Brown"));
-
-        final Iterator<VisibilityBindingSet> rightResults = Lists.<VisibilityBindingSet>newArrayList(
-                new VisibilityBindingSet(nameAge),
-                new VisibilityBindingSet(nameHair)).iterator();
-
-        // Therefore, there are a few new join results that mix the two together.
-        final Iterator<VisibilityBindingSet> newJoinResultsIt = naturalJoin.newLeftResult(new VisibilityBindingSet(newLeftResult), rightResults);
-
-        final Set<BindingSet> newJoinResults = new HashSet<>();
-        while(newJoinResultsIt.hasNext()) {
-            newJoinResults.add( newJoinResultsIt.next() );
-        }
-
-        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet();
-        final MapBindingSet nameHeightAge = new MapBindingSet();
-        nameHeightAge.addBinding("name", vf.createLiteral("Bob"));
-        nameHeightAge.addBinding("height", vf.createLiteral("5'9\""));
-        nameHeightAge.addBinding("age", vf.createLiteral(56));
-        expected.add(new VisibilityBindingSet(nameHeightAge));
-
-        final MapBindingSet nameHeightHair = new MapBindingSet();
-        nameHeightHair.addBinding("name", vf.createLiteral("Bob"));
-        nameHeightHair.addBinding("height", vf.createLiteral("5'9\""));
-        nameHeightHair.addBinding("hairColor", vf.createLiteral("Brown"));
-        expected.add(new VisibilityBindingSet(nameHeightHair));
-
-        assertEquals(expected, newJoinResults);
-    }
-
-    @Test
-    public void newRightResult_noLeftMatches() {
-        final IterativeJoin naturalJoin = new NaturalJoin();
-
-        // There are no left results.
-        final Iterator<VisibilityBindingSet> leftResults= new ArrayList<VisibilityBindingSet>().iterator();
-
-        // There is a new right result.
-        final MapBindingSet newRightResult = new MapBindingSet();
-        newRightResult.addBinding("name", vf.createLiteral("Bob"));
-
-        // Therefore, there are no new join results.
-        final Iterator<VisibilityBindingSet> newJoinResultsIt = naturalJoin.newRightResult(leftResults, new VisibilityBindingSet(newRightResult));
-        assertFalse( newJoinResultsIt.hasNext() );
-    }
-
-    @Test
-    public void newRightResult_joinsWithLeftResults() {
-        final IterativeJoin naturalJoin = new NaturalJoin();
-
-        // There are a few left results that join with the new right result.
-        final MapBindingSet nameAge = new MapBindingSet();
-        nameAge.addBinding("name", vf.createLiteral("Bob"));
-        nameAge.addBinding("age", vf.createLiteral(56));
-
-        final MapBindingSet nameHair = new MapBindingSet();
-        nameHair.addBinding("name", vf.createLiteral("Bob"));
-        nameHair.addBinding("hairColor", vf.createLiteral("Brown"));
-
-        final Iterator<VisibilityBindingSet> leftResults = Lists.<VisibilityBindingSet>newArrayList(
-                new VisibilityBindingSet(nameAge),
-                new VisibilityBindingSet(nameHair)).iterator();
-
-        // There is a new right result.
-        final MapBindingSet newRightResult = new MapBindingSet();
-        newRightResult.addBinding("name", vf.createLiteral("Bob"));
-        newRightResult.addBinding("height", vf.createLiteral("5'9\""));
-
-        // Therefore, there are a few new join results that mix the two together.
-        final Iterator<VisibilityBindingSet> newJoinResultsIt = naturalJoin.newRightResult(leftResults, new VisibilityBindingSet(newRightResult));
-
-        final Set<BindingSet> newJoinResults = new HashSet<>();
-        while(newJoinResultsIt.hasNext()) {
-            newJoinResults.add( newJoinResultsIt.next() );
-        }
-
-        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet();
-        final MapBindingSet nameHeightAge = new MapBindingSet();
-        nameHeightAge.addBinding("name", vf.createLiteral("Bob"));
-        nameHeightAge.addBinding("height", vf.createLiteral("5'9\""));
-        nameHeightAge.addBinding("age", vf.createLiteral(56));
-        expected.add(new VisibilityBindingSet(nameHeightAge));
-
-        final MapBindingSet nameHeightHair = new MapBindingSet();
-        nameHeightHair.addBinding("name", vf.createLiteral("Bob"));
-        nameHeightHair.addBinding("height", vf.createLiteral("5'9\""));
-        nameHeightHair.addBinding("hairColor", vf.createLiteral("Brown"));
-        expected.add(new VisibilityBindingSet(nameHeightHair));
-
-        assertEquals(expected, newJoinResults);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/BatchInformationSerializerTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/BatchInformationSerializerTest.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/BatchInformationSerializerTest.java
index cc120a1..2c37462 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/BatchInformationSerializerTest.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/test/java/org/apache/rya/indexing/pcj/fluo/app/batch/serializer/BatchInformationSerializerTest.java
@@ -24,8 +24,8 @@ import java.util.Optional;
 
 import org.apache.fluo.api.data.Bytes;
 import org.apache.fluo.api.data.Span;
+import org.apache.rya.api.function.join.LazyJoiningIterator.Side;
 import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.Side;
 import org.apache.rya.indexing.pcj.fluo.app.batch.BatchInformation;
 import org.apache.rya.indexing.pcj.fluo.app.batch.BatchInformation.Task;
 import org.apache.rya.indexing.pcj.fluo.app.batch.JoinBatchInformation;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/BatchIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/BatchIT.java b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/BatchIT.java
index 1007f68..5c8a1be 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/BatchIT.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/BatchIT.java
@@ -39,11 +39,11 @@ import org.apache.fluo.core.client.FluoClientImpl;
 import org.apache.log4j.Logger;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.api.function.join.LazyJoiningIterator.Side;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.indexing.pcj.fluo.api.CreateFluoPcj;
 import org.apache.rya.indexing.pcj.fluo.api.InsertTriples;
 import org.apache.rya.indexing.pcj.fluo.app.BindingSetRow;
-import org.apache.rya.indexing.pcj.fluo.app.JoinResultUpdater.Side;
 import org.apache.rya.indexing.pcj.fluo.app.NodeType;
 import org.apache.rya.indexing.pcj.fluo.app.batch.BatchInformation;
 import org.apache.rya.indexing.pcj.fluo.app.batch.BatchInformation.Task;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e79ef75..6b2ad61 100644
--- a/pom.xml
+++ b/pom.xml
@@ -217,6 +217,11 @@ under the License.
             </dependency>
             <dependency>
                 <groupId>org.apache.rya</groupId>
+                <artifactId>rya.api.function</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.rya</groupId>
                 <artifactId>rya.export.api</artifactId>
                 <version>${project.version}</version>
             </dependency>


[15/50] [abbrv] incubator-rya git commit: RYA-377 interactor for starting query processing

Posted by ca...@apache.org.
RYA-377 interactor for starting query processing


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

Branch: refs/heads/master
Commit: 5cb9a2796c6c7d1aef54ff6cd42175731ed5e5a9
Parents: adc44fd
Author: Andrew Smith <sm...@gmail.com>
Authored: Mon Oct 30 18:11:35 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../rya/streams/api/interactor/StartQuery.java  | 41 ++++++++++++++++++++
 1 file changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5cb9a279/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/StartQuery.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/StartQuery.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/StartQuery.java
new file mode 100644
index 0000000..29c29e8
--- /dev/null
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/StartQuery.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.api.interactor;
+
+import java.util.UUID;
+
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Starts processing a query in Rya Streams.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface StartQuery {
+
+    /**
+     * Starts processing a query in Rya Streams.
+     *
+     * @param queryID - The UUID for the SPARQL query that will be started. (not null)
+     * @throws RyaStreamsException The query could not be processed by Rya Streams.
+     */
+    public void start(final UUID queryID) throws RyaStreamsException;
+}
\ No newline at end of file


[22/50] [abbrv] incubator-rya git commit: RYA-377 Implement the stream query command in the client and fix a bunch of client bugs.

Posted by ca...@apache.org.
RYA-377 Implement the stream query command in the client and fix a bunch of client bugs.


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

Branch: refs/heads/master
Commit: f3ac7df1e9e707eb2e28e1c1c70212b89b716163
Parents: fc9775e
Author: kchilton2 <ke...@gmail.com>
Authored: Thu Nov 2 12:10:31 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../api/queries/InMemoryQueryChangeLog.java     |   5 +
 .../api/queries/InMemoryQueryRepository.java    |  10 ++
 .../rya/streams/api/queries/QueryChangeLog.java |   2 +-
 .../streams/api/queries/QueryRepository.java    |   2 +-
 .../apache/rya/streams/client/CLIDriver.java    |  40 ++++--
 .../rya/streams/client/RyaStreamsCommand.java   |   8 ++
 .../streams/client/command/AddQueryCommand.java |  77 ++++------
 .../client/command/DeleteQueryCommand.java      |  72 ++++------
 .../client/command/ListQueriesCommand.java      |  70 ++++-----
 .../client/command/LoadStatementsCommand.java   |  25 ++--
 .../client/command/StreamResultsCommand.java    | 141 +++++++++++++++++++
 .../client/src/main/resources/log4j.properties  |  27 ++++
 .../client/command/AddQueryCommandIT.java       |   3 +-
 .../client/command/DeleteQueryCommandIT.java    |   3 +-
 .../client/command/ListQueryCommandIT.java      |   3 +-
 .../kafka/queries/KafkaQueryChangeLog.java      |  30 +++-
 .../queries/KafkaQueryChangeLogFactory.java     |  74 ++++++++++
 17 files changed, 427 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryChangeLog.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryChangeLog.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryChangeLog.java
index 71af850..f0f628e 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryChangeLog.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryChangeLog.java
@@ -71,6 +71,11 @@ public class InMemoryQueryChangeLog implements QueryChangeLog {
         }
     }
 
+    @Override
+    public void close() throws Exception {
+        // Nothing to do here.
+    }
+
     /**
      * A {@link CloseableIteration} that iterates over a list.
      *

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java
index 652b16f..c1048fc 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/InMemoryQueryRepository.java
@@ -131,6 +131,16 @@ public class InMemoryQueryRepository implements QueryRepository {
         }
     }
 
+    @Override
+    public void close() throws Exception {
+        lock.lock();
+        try {
+            changeLog.close();
+        } finally {
+            lock.unlock();
+        }
+    }
+
     /**
      * A {@link Map} from query id to the {@link StreamsQuery} that is represented by that id based on what
      * is already in a {@link QueryChangeLog}.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java
index 824eebc..5765366 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryChangeLog.java
@@ -26,7 +26,7 @@ import info.aduna.iteration.CloseableIteration;
  * An ordered log of all of the changes that have been applied to the SPARQL Queries that are managed by Rya Streams.
  */
 @DefaultAnnotation(NonNull.class)
-public interface QueryChangeLog {
+public interface QueryChangeLog extends AutoCloseable {
 
     /**
      * Write a new {@link QueryChange} to the end of the change log.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java
index 54f98fc..850b2bc 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/queries/QueryRepository.java
@@ -31,7 +31,7 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * Repository for adding, deleting, and listing active queries in Rya Streams.
  */
 @DefaultAnnotation(NonNull.class)
-public interface QueryRepository {
+public interface QueryRepository extends AutoCloseable {
     /**
      * Adds a new query to Rya Streams.
      *

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java
index 93df2ae..5c0816f 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/CLIDriver.java
@@ -26,9 +26,11 @@ import java.util.Set;
 
 import org.apache.rya.streams.client.RyaStreamsCommand.ArgumentsException;
 import org.apache.rya.streams.client.RyaStreamsCommand.ExecutionException;
+import org.apache.rya.streams.client.command.AddQueryCommand;
+import org.apache.rya.streams.client.command.DeleteQueryCommand;
+import org.apache.rya.streams.client.command.ListQueriesCommand;
 import org.apache.rya.streams.client.command.LoadStatementsCommand;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.rya.streams.client.command.StreamResultsCommand;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
@@ -37,23 +39,31 @@ import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
 
 /**
- * CLI tool for interacting with rya streams.
- * <p>
+ * CLI tool for interacting with Rya Streams.
+ * </p>
  * This tool can be used to:
  * <ul>
- * <li>Load a file of statements into rya streams</li>
+ *   <li>Add a Query to Rya Streams</li>
+ *   <li>Delete a Query from Rya Streams</li>
+ *   <li>List the Queries that are being managed by Rya Streams</li>
+ *   <li>Load a file of RDF Statements into Rya Streams</li>
+ *   <li>Stream the results of a Query to the console</li>
  * </ul>
  */
 @DefaultAnnotation(NonNull.class)
 public class CLIDriver {
-    private static final Logger LOG = LoggerFactory.getLogger(CLIDriver.class);
+
     /**
      * Maps from command strings to the object that performs the command.
      */
     private static final ImmutableMap<String, RyaStreamsCommand> COMMANDS;
     static {
         final Set<Class<? extends RyaStreamsCommand>> commandClasses = new HashSet<>();
+        commandClasses.add(AddQueryCommand.class);
+        commandClasses.add(DeleteQueryCommand.class);
+        commandClasses.add(ListQueriesCommand.class);
         commandClasses.add(LoadStatementsCommand.class);
+        commandClasses.add(StreamResultsCommand.class);
         final ImmutableMap.Builder<String, RyaStreamsCommand> builder = ImmutableMap.builder();
         for(final Class<? extends RyaStreamsCommand> commandClass : commandClasses) {
             try {
@@ -70,10 +80,7 @@ public class CLIDriver {
     private static final String USAGE = makeUsage(COMMANDS);
 
     public static void main(final String[] args) {
-        LOG.trace("Starting up the Rya Streams Client.");
-
-        // If no command provided or the command isn't recognized, then print
-        // the usage.
+        // If no command provided or the command isn't recognized, then print the usage.
         if (args.length == 0 || !COMMANDS.containsKey(args[0])) {
             System.out.println(USAGE);
             System.exit(1);
@@ -84,14 +91,19 @@ public class CLIDriver {
         final String[] commandArgs = Arrays.copyOfRange(args, 1, args.length);
         final RyaStreamsCommand streamsCommand = COMMANDS.get(command);
 
+        // Print usage if the arguments are invalid for the command.
+        if(!streamsCommand.validArguments(commandArgs)) {
+            System.out.println(streamsCommand.getUsage());
+            System.exit(1);
+        }
+
         // Execute the command.
         try {
             streamsCommand.execute(commandArgs);
         } catch (ArgumentsException | ExecutionException e) {
-            LOG.error("The command: " + command + " failed to execute properly.", e);
+            System.err.println("The command: " + command + " failed to execute properly.");
+            e.printStackTrace();
             System.exit(2);
-        } finally {
-            LOG.trace("Shutting down the Rya Streams Client.");
         }
     }
 
@@ -119,4 +131,4 @@ public class CLIDriver {
 
         return usage.toString();
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
index 5d64785..5b05d0a 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
@@ -89,6 +89,14 @@ public interface RyaStreamsCommand {
     }
 
     /**
+     * Validates a set of arguments that may be passed into the command.
+     *
+     * @param args - The arguments that will be validated. (not null)
+     * @return {@code true} if the arguments are valid, otherwise {@code false}.
+     */
+    public boolean validArguments(String[] args);
+
+    /**
      * Execute the command using the command line arguments.
      *
      * @param args - Command line arguments that configure how the command will execute. (not null)

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java
index 8439f20..c72e6a2 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/AddQueryCommand.java
@@ -20,32 +20,16 @@ package org.apache.rya.streams.client.command;
 
 import static java.util.Objects.requireNonNull;
 
-import java.util.Properties;
-
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.streams.api.entity.StreamsQuery;
 import org.apache.rya.streams.api.exception.RyaStreamsException;
 import org.apache.rya.streams.api.interactor.AddQuery;
 import org.apache.rya.streams.api.interactor.defaults.DefaultAddQuery;
 import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
-import org.apache.rya.streams.api.queries.QueryChange;
 import org.apache.rya.streams.api.queries.QueryChangeLog;
 import org.apache.rya.streams.api.queries.QueryRepository;
 import org.apache.rya.streams.client.RyaStreamsCommand;
 import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
-import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
-import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLogFactory;
 
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
@@ -60,7 +44,6 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  */
 @DefaultAnnotation(NonNull.class)
 public class AddQueryCommand implements RyaStreamsCommand {
-    private static final Logger log = LoggerFactory.getLogger(AddQueryCommand.class);
 
     /**
      * Command line parameters that are used by this command to configure itself.
@@ -73,11 +56,9 @@ public class AddQueryCommand implements RyaStreamsCommand {
         public String toString() {
             final StringBuilder parameters = new StringBuilder();
             parameters.append(super.toString());
-            parameters.append("\n");
 
             if (!Strings.isNullOrEmpty(query)) {
-                parameters.append("\tQuery: " + query);
-                parameters.append("\n");
+                parameters.append("\tQuery: " + query + "\n");
             }
             return parameters.toString();
         }
@@ -103,6 +84,17 @@ public class AddQueryCommand implements RyaStreamsCommand {
     }
 
     @Override
+    public boolean validArguments(final String[] args) {
+        boolean valid = true;
+        try {
+            new JCommander(new AddParameters(), args);
+        } catch(final ParameterException e) {
+            valid = false;
+        }
+        return valid;
+    }
+
+    @Override
     public void execute(final String[] args) throws ArgumentsException, ExecutionException {
         requireNonNull(args);
 
@@ -113,34 +105,27 @@ public class AddQueryCommand implements RyaStreamsCommand {
         } catch(final ParameterException e) {
             throw new ArgumentsException("Could not add a new query because of invalid command line parameters.", e);
         }
-        log.trace("Executing the Add Query Command\n" + params.toString());
 
-        // Create properties for interacting with Kafka.
-        final Properties producerProperties = new Properties();
-        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
-        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
-
-        final Properties consumerProperties = new Properties();
-        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
-        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
-
-        final Producer<?, QueryChange> queryProducer = new KafkaProducer<>(producerProperties);
-        final Consumer<?, QueryChange> queryConsumer = new KafkaConsumer<>(consumerProperties);
-
-        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, KafkaTopics.queryChangeLogTopic(params.ryaInstance));
-        final QueryRepository repo = new InMemoryQueryRepository(changeLog);
+        // Create the Kafka backed QueryChangeLog.
+        final String bootstrapServers = params.kafkaIP + ":" + params.kafkaPort;
+        final String topic = KafkaTopics.queryChangeLogTopic(params.ryaInstance);
+        final QueryChangeLog queryChangeLog = KafkaQueryChangeLogFactory.make(bootstrapServers, topic);
 
         // Execute the add query command.
-        final AddQuery addQuery = new DefaultAddQuery(repo);
-        try {
-            final StreamsQuery query = addQuery.addQuery(params.query);
-            log.trace("Added query: " + query.getSparql());
-        } catch (final RyaStreamsException e) {
-            log.error("Unable to parse query: " + params.query, e);
+        try(QueryRepository queryRepo = new InMemoryQueryRepository(queryChangeLog)) {
+            final AddQuery addQuery = new DefaultAddQuery(queryRepo);
+            try {
+                final StreamsQuery query = addQuery.addQuery(params.query);
+                System.out.println("Added query: " + query.getSparql());
+            } catch (final RyaStreamsException e) {
+                System.err.println("Unable to parse query: " + params.query);
+                e.printStackTrace();
+                System.exit(1);
+            }
+        } catch (final Exception e) {
+            System.err.println("Problem encountered while closing the QueryRepository.");
+            e.printStackTrace();
+            System.exit(1);
         }
-
-        log.trace("Finished executing the Add Query Command.");
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java
index b101a0f..2aeb90c 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/DeleteQueryCommand.java
@@ -20,32 +20,17 @@ package org.apache.rya.streams.client.command;
 
 import static java.util.Objects.requireNonNull;
 
-import java.util.Properties;
 import java.util.UUID;
 
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.streams.api.exception.RyaStreamsException;
 import org.apache.rya.streams.api.interactor.DeleteQuery;
 import org.apache.rya.streams.api.interactor.defaults.DefaultDeleteQuery;
 import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
-import org.apache.rya.streams.api.queries.QueryChange;
 import org.apache.rya.streams.api.queries.QueryChangeLog;
 import org.apache.rya.streams.api.queries.QueryRepository;
 import org.apache.rya.streams.client.RyaStreamsCommand;
 import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
-import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
-import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLogFactory;
 
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
@@ -60,7 +45,6 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  */
 @DefaultAnnotation(NonNull.class)
 public class DeleteQueryCommand implements RyaStreamsCommand {
-    private static final Logger log = LoggerFactory.getLogger(DeleteQueryCommand.class);
 
     /**
      * Command line parameters that are used by this command to configure itself.
@@ -102,6 +86,17 @@ public class DeleteQueryCommand implements RyaStreamsCommand {
     }
 
     @Override
+    public boolean validArguments(final String[] args) {
+        boolean valid = true;
+        try {
+            new JCommander(new RemoveParameters(), args);
+        } catch(final ParameterException e) {
+            valid = false;
+        }
+        return valid;
+    }
+
+    @Override
     public void execute(final String[] args) throws ArgumentsException, ExecutionException {
         requireNonNull(args);
 
@@ -112,34 +107,27 @@ public class DeleteQueryCommand implements RyaStreamsCommand {
         } catch(final ParameterException e) {
             throw new ArgumentsException("Could not add a new query because of invalid command line parameters.", e);
         }
-        log.trace("Executing the Add Query Command\n" + params.toString());
 
-        // Create properties for interacting with Kafka.
-        final Properties producerProperties = new Properties();
-        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
-        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
-
-        final Properties consumerProperties = new Properties();
-        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
-        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
-
-        final Producer<?, QueryChange> queryProducer = new KafkaProducer<>(producerProperties);
-        final Consumer<?, QueryChange> queryConsumer = new KafkaConsumer<>(consumerProperties);
-
-        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, KafkaTopics.queryChangeLogTopic(params.ryaInstance));
-        final QueryRepository repo = new InMemoryQueryRepository(changeLog);
+        // Create the Kafka backed QueryChangeLog.
+        final String bootstrapServers = params.kafkaIP + ":" + params.kafkaPort;
+        final String topic = KafkaTopics.queryChangeLogTopic(params.ryaInstance);
+        final QueryChangeLog queryChangeLog = KafkaQueryChangeLogFactory.make(bootstrapServers, topic);
 
         // Execute the delete query command.
-        final DeleteQuery deleteQuery = new DefaultDeleteQuery(repo);
-        try {
-            deleteQuery.delete(UUID.fromString(params.queryId));
-            log.trace("Deleted query: " + params.queryId);
-        } catch (final RyaStreamsException e) {
-            log.error("Unable to delete query with ID: " + params.queryId, e);
+        try(QueryRepository queryRepo = new InMemoryQueryRepository(queryChangeLog)) {
+            final DeleteQuery deleteQuery = new DefaultDeleteQuery(queryRepo);
+            try {
+                deleteQuery.delete(UUID.fromString(params.queryId));
+                System.out.println("Deleted query: " + params.queryId);
+            } catch (final RyaStreamsException e) {
+                System.err.println("Unable to delete query with ID: " + params.queryId);
+                e.printStackTrace();
+                System.exit(1);
+            }
+        } catch (final Exception e) {
+            System.err.println("Problem encountered while closing the QueryRepository.");
+            e.printStackTrace();
+            System.exit(1);
         }
-
-        log.trace("Finished executing the Delete Query Command.");
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
index c4e5de6..670007b 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
@@ -20,33 +20,18 @@ package org.apache.rya.streams.client.command;
 
 import static java.util.Objects.requireNonNull;
 
-import java.util.Properties;
 import java.util.Set;
 
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.streams.api.entity.StreamsQuery;
 import org.apache.rya.streams.api.exception.RyaStreamsException;
 import org.apache.rya.streams.api.interactor.ListQueries;
 import org.apache.rya.streams.api.interactor.defaults.DefaultListQueries;
 import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
-import org.apache.rya.streams.api.queries.QueryChange;
 import org.apache.rya.streams.api.queries.QueryChangeLog;
 import org.apache.rya.streams.api.queries.QueryRepository;
 import org.apache.rya.streams.client.RyaStreamsCommand;
 import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
-import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
-import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLogFactory;
 
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.ParameterException;
@@ -59,7 +44,6 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  */
 @DefaultAnnotation(NonNull.class)
 public class ListQueriesCommand implements RyaStreamsCommand {
-    private static final Logger log = LoggerFactory.getLogger(ListQueriesCommand.class);
 
     @Override
     public String getCommand() {
@@ -72,6 +56,17 @@ public class ListQueriesCommand implements RyaStreamsCommand {
     }
 
     @Override
+    public boolean validArguments(final String[] args) {
+        boolean valid = true;
+        try {
+            new JCommander(new KafkaParameters(), args);
+        } catch(final ParameterException e) {
+            valid = false;
+        }
+        return valid;
+    }
+
+    @Override
     public void execute(final String[] args) throws ArgumentsException, ExecutionException {
         requireNonNull(args);
 
@@ -82,32 +77,27 @@ public class ListQueriesCommand implements RyaStreamsCommand {
         } catch (final ParameterException e) {
             throw new ArgumentsException("Could not list the queries because of invalid command line parameters.", e);
         }
-        log.trace("Executing the List Query Command.\n" + params.toString());
 
-        // Create properties for interacting with Kafka.
-        final Properties producerProperties = new Properties();
-        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
-        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
-
-        final Properties consumerProperties = new Properties();
-        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
-        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
-
-        final Producer<?, QueryChange> queryProducer = new KafkaProducer<>(producerProperties);
-        final Consumer<?, QueryChange> queryConsumer = new KafkaConsumer<>(consumerProperties);
-
-        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, KafkaTopics.queryChangeLogTopic(params.ryaInstance));
-        final QueryRepository repo = new InMemoryQueryRepository(changeLog);
+        // Create the Kafka backed QueryChangeLog.
+        final String bootstrapServers = params.kafkaIP + ":" + params.kafkaPort;
+        final String topic = KafkaTopics.queryChangeLogTopic(params.ryaInstance);
+        final QueryChangeLog queryChangeLog = KafkaQueryChangeLogFactory.make(bootstrapServers, topic);
 
         // Execute the list queries command.
-        final ListQueries listQueries = new DefaultListQueries(repo);
-        try {
-            final Set<StreamsQuery> queries = listQueries.all();
-            System.out.println( formatQueries(queries) );
-        } catch (final RyaStreamsException e) {
-            log.error("Unable to retrieve the queries.", e);
+        try(QueryRepository queryRepo = new InMemoryQueryRepository(queryChangeLog)) {
+            final ListQueries listQueries = new DefaultListQueries(queryRepo);
+            try {
+                final Set<StreamsQuery> queries = listQueries.all();
+                System.out.println( formatQueries(queries) );
+            } catch (final RyaStreamsException e) {
+                System.err.println("Unable to retrieve the queries.");
+                e.printStackTrace();
+                System.exit(1);
+            }
+        } catch (final Exception e) {
+            System.err.println("Problem encountered while closing the QueryRepository.");
+            e.printStackTrace();
+            System.exit(1);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
index 4763bd8..6ae63da 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
@@ -34,8 +34,6 @@ import org.apache.rya.streams.client.RyaStreamsCommand;
 import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
@@ -51,7 +49,6 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  */
 @DefaultAnnotation(NonNull.class)
 public class LoadStatementsCommand implements RyaStreamsCommand {
-    private static final Logger log = LoggerFactory.getLogger(LoadStatementsCommand.class);
 
     /**
      * Command line parameters that are used by this command to configure itself.
@@ -90,7 +87,7 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
 
     @Override
     public String getDescription() {
-        return "Load RDF Statements into Rya Streams";
+        return "Load RDF Statements into Rya Streams.";
     }
 
     @Override
@@ -103,10 +100,20 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
     }
 
     @Override
+    public boolean validArguments(final String[] args) {
+        boolean valid = true;
+        try {
+            new JCommander(new LoadStatementsParameters(), args);
+        } catch(final ParameterException e) {
+            valid = false;
+        }
+        return valid;
+    }
+
+    @Override
     public void execute(final String[] args) throws ArgumentsException, ExecutionException {
         requireNonNull(args);
 
-
         // Parse the command line arguments.
         final LoadStatementsParameters params = new LoadStatementsParameters();
         try {
@@ -114,9 +121,7 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
         } catch(final ParameterException e) {
             throw new ArgumentsException("Could not load the Statements file because of invalid command line parameters.", e);
         }
-        log.trace("Executing the Load Statements Command\n" + params.toString());
 
-        log.trace("Loading Statements from the file '" + params.statementsFile + "'.");
         final Path statementsPath = Paths.get(params.statementsFile);
 
         final Properties producerProps = buildProperties(params);
@@ -124,10 +129,8 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
             final LoadStatements statements = new KafkaLoadStatements(KafkaTopics.statementsTopic(params.ryaInstance), producer);
             statements.load(statementsPath, params.visibilities);
         } catch (final Exception e) {
-            log.error("Unable to parse statements file: " + statementsPath.toString(), e);
+            System.err.println("Unable to parse statements file: " + statementsPath.toString());
         }
-
-        log.trace("Finished executing the Load Statements Command.");
     }
 
     private Properties buildProperties(final LoadStatementsParameters params) {
@@ -138,4 +141,4 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
         props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VisibilityStatementSerializer.class.getName());
         return props;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
new file mode 100644
index 0000000..9de978b
--- /dev/null
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.client.command;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.api.entity.QueryResultStream;
+import org.apache.rya.streams.api.interactor.GetQueryResultStream;
+import org.apache.rya.streams.client.RyaStreamsCommand;
+import org.apache.rya.streams.kafka.interactor.KafkaGetQueryResultStream;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.google.common.base.Strings;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A command that streams the results of a query to the console.
+ */
+@DefaultAnnotation(NonNull.class)
+public class StreamResultsCommand implements RyaStreamsCommand {
+
+    /**
+     * Command line parameters that are used by this command to configure itself.
+     */
+    private static final class StreamResultsParameters extends RyaStreamsCommand.KafkaParameters {
+
+        @Parameter(names = {"--queryId", "-q"}, required = true, description = "The query whose results will be streamed to the console.")
+        private String queryId;
+
+        @Override
+        public String toString() {
+            final StringBuilder parameters = new StringBuilder();
+            parameters.append(super.toString());
+
+            if (!Strings.isNullOrEmpty(queryId)) {
+                parameters.append("\tQuery ID: " + queryId);
+                parameters.append("\n");
+            }
+
+            return parameters.toString();
+        }
+    }
+
+    @Override
+    public String getCommand() {
+        return "stream-results";
+    }
+
+    @Override
+    public String getDescription() {
+        return "Stream the results of a query to the console.";
+    }
+
+    @Override
+    public String getUsage() {
+        final JCommander parser = new JCommander(new StreamResultsParameters());
+
+        final StringBuilder usage = new StringBuilder();
+        parser.usage(usage);
+        return usage.toString();
+    }
+
+    @Override
+    public boolean validArguments(final String[] args) {
+        boolean valid = true;
+        try {
+            new JCommander(new StreamResultsParameters(), args);
+        } catch(final ParameterException e) {
+            valid = false;
+        }
+        return valid;
+    }
+
+    @Override
+    public void execute(final String[] args) throws ArgumentsException, ExecutionException {
+        requireNonNull(args);
+
+        // Parse the command line arguments.
+        final StreamResultsParameters params = new StreamResultsParameters();
+        try {
+            new JCommander(params, args);
+        } catch(final ParameterException e) {
+            throw new ArgumentsException("Could not stream the query's results because of invalid command line parameters.", e);
+        }
+
+        final UUID queryId;
+        try {
+            queryId = UUID.fromString( params.queryId );
+        } catch(final IllegalArgumentException e) {
+            throw new ArgumentsException("Invalid Query ID " + params.queryId);
+        }
+
+        // This command executes until the application is killed, so create a kill boolean.
+        final AtomicBoolean finished = new AtomicBoolean(false);
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                finished.set(true);
+            }
+        });
+
+        // Execute the command.
+        final GetQueryResultStream getQueryResultStream = new KafkaGetQueryResultStream(params.kafkaIP, params.kafkaPort);
+
+        try (final QueryResultStream stream = getQueryResultStream.fromNow(queryId)) {
+            while(!finished.get()) {
+                for(final VisibilityBindingSet visBs : stream.poll(1000)) {
+                    System.out.println(visBs);
+                }
+            }
+        } catch (final Exception e) {
+            System.err.println("Error while reading the results from the stream.");
+            e.printStackTrace();
+            System.exit(1);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/resources/log4j.properties b/extras/rya.streams/client/src/main/resources/log4j.properties
new file mode 100644
index 0000000..b07468c
--- /dev/null
+++ b/extras/rya.streams/client/src/main/resources/log4j.properties
@@ -0,0 +1,27 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+# Root logger option
+log4j.rootLogger=INFO, stdout
+
+# Direct log messages to stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
index 09e874c..ee4378e 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/AddQueryCommandIT.java
@@ -42,7 +42,6 @@ import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
-import org.apache.rya.test.kafka.KafkaITBase;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.After;
 import org.junit.Before;
@@ -52,7 +51,7 @@ import org.junit.Test;
 /**
  * integration Test for adding a new query through a command.
  */
-public class AddQueryCommandIT extends KafkaITBase {
+public class AddQueryCommandIT {
 
     private final String ryaInstance = UUID.randomUUID().toString();
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
index 0079371..c5dad3d 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/DeleteQueryCommandIT.java
@@ -43,7 +43,6 @@ import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
-import org.apache.rya.test.kafka.KafkaITBase;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.After;
 import org.junit.Before;
@@ -53,7 +52,7 @@ import org.junit.Test;
 /**
  * Integration Test for deleting a query from Rya Streams through a command.
  */
-public class DeleteQueryCommandIT extends KafkaITBase {
+public class DeleteQueryCommandIT {
 
     private final String ryaInstance = UUID.randomUUID().toString();
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
index eb759ba..b32967e 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
@@ -38,7 +38,6 @@ import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
-import org.apache.rya.test.kafka.KafkaITBase;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.After;
 import org.junit.Before;
@@ -48,7 +47,7 @@ import org.junit.Test;
 /**
  * integration Test for listing queries through a command.
  */
-public class ListQueryCommandIT extends KafkaITBase {
+public class ListQueryCommandIT {
 
     private final String ryaInstance = UUID.randomUUID().toString();
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLog.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLog.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLog.java
index 19622ae..9403e4b 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLog.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLog.java
@@ -23,11 +23,14 @@ import static java.util.Objects.requireNonNull;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.rya.streams.api.queries.ChangeLogEntry;
 import org.apache.rya.streams.api.queries.QueryChange;
@@ -76,7 +79,17 @@ public class KafkaQueryChangeLog implements QueryChangeLog {
     @Override
     public void write(final QueryChange newChange) throws QueryChangeLogException {
         requireNonNull(newChange);
-        producer.send(new ProducerRecord<>(topic, newChange));
+
+        // Write the change to the log immediately.
+        final Future<RecordMetadata> future = producer.send(new ProducerRecord<>(topic, newChange));
+        producer.flush();
+
+        // Don't return until the write has been completed.
+        try {
+            future.get();
+        } catch (InterruptedException | ExecutionException e) {
+            throw new QueryChangeLogException("Could not record a new query change to the Kafka Query Change Log.", e);
+        }
     }
 
     @Override
@@ -96,6 +109,15 @@ public class KafkaQueryChangeLog implements QueryChangeLog {
     }
 
     /**
+     * Closing this class will also close the {@link Producer} and {@link Consumer} that were passed into it.
+     */
+    @Override
+    public void close() throws Exception {
+        producer.close();
+        consumer.close();
+    }
+
+    /**
      * A {@link CloseableIteration} to iterate over a consumer's results. Since
      * the consumer returns in bulk when poll(), a cache of recent polling is
      * maintained.
@@ -149,10 +171,10 @@ public class KafkaQueryChangeLog implements QueryChangeLog {
             final ConsumerRecords<?, QueryChange> records = consumer.poll(3000L);
             final List<ChangeLogEntry<QueryChange>> changes = new ArrayList<>();
             records.forEach(
-                    record -> 
-                        changes.add(new ChangeLogEntry<QueryChange>(record.offset(), record.value()))
+                    record ->
+                        changes.add(new ChangeLogEntry<>(record.offset(), record.value()))
                     );
             iterCache = changes.iterator();
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f3ac7df1/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogFactory.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogFactory.java
new file mode 100644
index 0000000..5042b30
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogFactory.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.queries;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Properties;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Creates instances of {@link KafkaQueryChangeLog}.
+ */
+@DefaultAnnotation(NonNull.class)
+public class KafkaQueryChangeLogFactory {
+
+    /**
+     * Creates an instance of {@link KafkaQueryChangeLog} using a new {@link Producer} and {@link Consumer}.
+     *
+     * @param bootstrapServers - Indicates which instance of Kafka that will be connected to. (not null)
+     * @param topic - The topic the QueryChangeLog is persisted to. (not null)
+     * @return A new instance of {@link KafkaQueryChangeLog}.
+     */
+    public static KafkaQueryChangeLog make(
+            final String bootstrapServers,
+            final String topic) {
+        requireNonNull(bootstrapServers);
+        requireNonNull(topic);
+
+        final Properties producerProperties = new Properties();
+        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
+        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
+
+        final Properties consumerProperties = new Properties();
+        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
+        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
+
+        final Producer<?, QueryChange> producer = new KafkaProducer<>(producerProperties);
+        final Consumer<?, QueryChange> consumer = new KafkaConsumer<>(consumerProperties);
+        return new KafkaQueryChangeLog(producer, consumer, topic);
+    }
+}
\ No newline at end of file


[16/50] [abbrv] incubator-rya git commit: RYA-377 Add ListQueries command

Posted by ca...@apache.org.
RYA-377 Add ListQueries command

Added command for listing queries
Added Integration Test for the command
Added default implementation for the interacot

Added default usage to command interface


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

Branch: refs/heads/master
Commit: adc44fd369b4f54497fa7a4d1ef099d166effbf3
Parents: a8b511b
Author: Andrew Smith <sm...@gmail.com>
Authored: Fri Oct 27 17:08:47 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../rya/streams/api/interactor/ListQueries.java |   4 +-
 .../interactor/defaults/DefaultListQueries.java |  54 ++++++++
 .../rya/streams/client/RyaStreamsCommand.java   |   9 +-
 .../client/command/ListQueriesCommand.java      | 124 +++++++++++++++++++
 .../client/command/ListQueryCommandIT.java      |  60 +++++++++
 5 files changed, 248 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/adc44fd3/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/ListQueries.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/ListQueries.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/ListQueries.java
index 4cab856..5d03f5c 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/ListQueries.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/ListQueries.java
@@ -18,7 +18,7 @@
  */
 package org.apache.rya.streams.api.interactor;
 
-import java.util.List;
+import java.util.Set;
 
 import org.apache.rya.streams.api.entity.StreamsQuery;
 import org.apache.rya.streams.api.exception.RyaStreamsException;
@@ -38,5 +38,5 @@ public interface ListQueries {
      * @return All of the queries that are managed.
      * @throws RyaStreamsException The queries could not be listed.
      */
-    public List<StreamsQuery> all() throws RyaStreamsException;
+    public Set<StreamsQuery> all() throws RyaStreamsException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/adc44fd3/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultListQueries.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultListQueries.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultListQueries.java
new file mode 100644
index 0000000..82ca691
--- /dev/null
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultListQueries.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.api.interactor.defaults;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Set;
+
+import org.apache.rya.streams.api.entity.StreamsQuery;
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+import org.apache.rya.streams.api.interactor.ListQueries;
+import org.apache.rya.streams.api.queries.QueryRepository;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Lists all queries in Rya Streams.
+ */
+@DefaultAnnotation(NonNull.class)
+public class DefaultListQueries implements ListQueries {
+    private final QueryRepository repository;
+
+    /**
+     * Creates a new {@link DefaultAddQuery}.
+     *
+     * @param repository - The {@link QueryRepository} to add a query to. (not
+     *        null)
+     */
+    public DefaultListQueries(final QueryRepository repository) {
+        this.repository = requireNonNull(repository);
+    }
+
+    @Override
+    public Set<StreamsQuery> all() throws RyaStreamsException {
+        return repository.list();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/adc44fd3/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
index c4c55e8..967b79e 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/RyaStreamsCommand.java
@@ -18,6 +18,7 @@
  */
 package org.apache.rya.streams.client;
 
+import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
 import com.google.common.base.Strings;
 
@@ -80,7 +81,13 @@ public interface RyaStreamsCommand {
     /**
      * @return Describes what arguments may be provided to the command.
      */
-    public String getUsage();
+    default public String getUsage() {
+        final JCommander parser = new JCommander(new Parameters());
+
+        final StringBuilder usage = new StringBuilder();
+        parser.usage(usage);
+        return usage.toString();
+    }
 
     /**
      * Execute the command using the command line arguments.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/adc44fd3/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
new file mode 100644
index 0000000..ec40b50
--- /dev/null
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/ListQueriesCommand.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.client.command;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.rya.streams.api.entity.StreamsQuery;
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+import org.apache.rya.streams.api.interactor.ListQueries;
+import org.apache.rya.streams.api.interactor.defaults.DefaultListQueries;
+import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
+import org.apache.rya.streams.api.queries.QueryChange;
+import org.apache.rya.streams.api.queries.QueryChangeLog;
+import org.apache.rya.streams.api.queries.QueryRepository;
+import org.apache.rya.streams.client.RyaStreamsCommand;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLog;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
+import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.ParameterException;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A command that lists all queries currently in Rya Streams.
+ */
+@DefaultAnnotation(NonNull.class)
+public class ListQueriesCommand implements RyaStreamsCommand {
+    private static final Logger log = LoggerFactory.getLogger(ListQueriesCommand.class);
+
+    @Override
+    public String getCommand() {
+        return "list-queries";
+    }
+
+    @Override
+    public String getDescription() {
+        return "Lists all queries currently in Rya Streams.";
+    }
+
+    @Override
+    public void execute(final String[] args) throws ArgumentsException, ExecutionException {
+        requireNonNull(args);
+
+        // Parse the command line arguments.
+        final Parameters params = new Parameters();
+        try {
+            new JCommander(params, args);
+        } catch (final ParameterException e) {
+            throw new ArgumentsException("Could not list the queries because of invalid command line parameters.", e);
+        }
+        log.trace("Executing the List Query Command.\n" + params.toString());
+
+        // Create properties for interacting with Kafka.
+        final Properties producerProperties = new Properties();
+        producerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
+        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
+        final Properties consumerProperties = new Properties();
+        consumerProperties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, params.kafkaIP + ":" + params.kafkaPort);
+        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
+        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
+        final Producer<?, QueryChange> queryProducer = new KafkaProducer<>(producerProperties);
+        final Consumer<?, QueryChange> queryConsumer = new KafkaConsumer<>(consumerProperties);
+
+        final QueryChangeLog changeLog = new KafkaQueryChangeLog(queryProducer, queryConsumer, params.topicName);
+        final QueryRepository repo = new InMemoryQueryRepository(changeLog);
+        final ListQueries listQueries = new DefaultListQueries(repo);
+        try {
+            final Set<StreamsQuery> queries = listQueries.all();
+            logQueries(queries);
+        } catch (final RyaStreamsException e) {
+            log.error("Unable to retrieve the queries.", e);
+        }
+    }
+
+    private void logQueries(final Set<StreamsQuery> queries) {
+        final StringBuilder sb = new StringBuilder();
+        sb.append("\n");
+        sb.append("Queries in Rya Streams:\n");
+        sb.append("---------------------------------------------------------\n");
+        queries.forEach(query -> {
+            sb.append("ID: ");
+            sb.append(query.getQueryId());
+            sb.append("\t\t");
+            sb.append("Query: ");
+            sb.append(query.getSparql());
+            sb.append("\n");
+        });
+        log.trace(sb.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/adc44fd3/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
new file mode 100644
index 0000000..be90c5f
--- /dev/null
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/ListQueryCommandIT.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.client.command;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Properties;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.rya.test.kafka.KafkaITBase;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * integration Test for listing queries through a command.
+ */
+public class ListQueryCommandIT extends KafkaITBase {
+    private String[] args;
+
+    @Rule
+    public KafkaTestInstanceRule rule = new KafkaTestInstanceRule(true);
+
+    @Before
+    public void setup() {
+        final Properties props = rule.createBootstrapServerConfig();
+        final String location = props.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG);
+        final String[] tokens = location.split(":");
+        args = new String[] {
+                "-t", rule.getKafkaTopicName(),
+                "-p", tokens[1],
+                "-i", tokens[0]
+        };
+    }
+
+    @Test
+    public void happyListQueriesTest() throws Exception {
+        final ListQueriesCommand command = new ListQueriesCommand();
+        command.execute(args);
+        // not sure what to assert here.
+        assertEquals(true, true);
+    }
+}


[07/50] [abbrv] incubator-rya git commit: RYA-377 Interface for topology factory

Posted by ca...@apache.org.
RYA-377 Interface for topology factory


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

Branch: refs/heads/master
Commit: c4966ff6c872d948548a90d66bee86be1727ea11
Parents: 0a2170a
Author: Andrew Smith <sm...@gmail.com>
Authored: Tue Nov 14 15:34:51 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../kafka/topology/TopologyBuilderFactory.java  | 48 ++++++++++++++++++++
 1 file changed, 48 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4966ff6/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyBuilderFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyBuilderFactory.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyBuilderFactory.java
new file mode 100644
index 0000000..9e9dd92
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyBuilderFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.topology;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.algebra.TupleExpr;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Factory for building {@link TopologyBuilder}s from a SPARQL query.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface TopologyBuilderFactory {
+
+    /**
+     * Builds a {@link TopologyBuilder} based on the provided sparql query where
+     * each {@link TupleExpr} in the parsed query is a processor in the
+     * topology.
+     *
+     * @param sparqlQuery - The SPARQL query to build a topology for. (not null)
+     * @param statementTopic - The topic for the source to read from. (not null)
+     * @param statementTopic - The topic for the sink to write to. (not null)
+     * @return - The created {@link TopologyBuilder}.
+     * @throws MalformedQueryException - The provided query is not a valid
+     *         SPARQL query.
+     */
+    public TopologyBuilder build(final String sparqlQuery, final String statementTopic, final String resultTopic)
+            throws Exception;
+}


[27/50] [abbrv] incubator-rya git commit: RYA-377 Fixed a bug where streams jobs would not resume where they had left off after being resumed.

Posted by ca...@apache.org.
RYA-377 Fixed a bug where streams jobs would not resume where they had left off after being resumed.


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

Branch: refs/heads/master
Commit: 3ebf6db80d83757f199ae708d18f5e61f29d45af
Parents: a51765e
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Dec 12 20:12:12 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../rya/streams/kafka/interactor/KafkaRunQuery.java      | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3ebf6db8/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaRunQuery.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaRunQuery.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaRunQuery.java
index e587998..aef7c58 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaRunQuery.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaRunQuery.java
@@ -25,7 +25,7 @@ import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 
-import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.processor.TopologyBuilder;
@@ -103,8 +103,13 @@ public class KafkaRunQuery implements RunQuery {
 
         // Setup the Kafka Stream program.
         final Properties streamsProps = new Properties();
-        streamsProps.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaHostname + ":" + kafkaPort);
-        streamsProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "KafkaRunQuery-" + UUID.randomUUID());
+        streamsProps.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaHostname + ":" + kafkaPort);
+
+        // Use the Query ID as the Application ID to ensure we resume where we left off the last time this command was run.
+        streamsProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "KafkaRunQuery-" + queryId);
+
+        // Always start at the beginning of the input topic.
+        streamsProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
 
         final KafkaStreams streams = new KafkaStreams(topologyBuilder, new StreamsConfig(streamsProps));
 


[37/50] [abbrv] incubator-rya git commit: RYA-377 Update the Projection processor to handle Construct queries.

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
new file mode 100644
index 0000000..99e2451
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.projection;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.ProjectionEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RdfTestUtil;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
+import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
+import org.apache.rya.streams.kafka.processors.projection.ProjectionProcessorSupplier;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Sets;
+
+/**
+ * Integration tests the methods of {@link StatementPatternProcessor}.
+ */
+public class ProjectionProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void showProcessorWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "SELECT (?person AS ?p) ?otherPerson " +
+                "WHERE { " +
+                    "?person <urn:talksTo> ?otherPerson . " +
+                "}";
+        final Projection projection = RdfTestUtil.getProjection(sparql);
+        final StatementPattern sp = RdfTestUtil.getSp(sparql);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
+
+        // Add a processor that handles the projection.
+        builder.addProcessor("P1", new ProjectionProcessorSupplier(
+                ProjectionEvaluator.make(projection),
+                result -> ProcessorResult.make(new UnaryResult(result))), "SP1");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "P1");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Load some data into the input topic.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        final MapBindingSet expectedBs = new MapBindingSet();
+        expectedBs.addBinding("p", vf.createURI("urn:Alice"));
+        expectedBs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add(new VisibilityBindingSet(expectedBs, "a"));
+
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorTest.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorTest.java
new file mode 100644
index 0000000..7ff2c96
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorTest.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.projection;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.function.projection.ProjectionEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.RdfTestUtil;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.projection.ProjectionProcessorSupplier.ProjectionProcessor;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Unit tests the methods of {@link ProjectionProcessor}.
+ */
+public class ProjectionProcessorTest {
+
+    @Test
+    public void showProjectionFunctionIsCalled() throws Exception {
+        // A query whose projection does not change anything.
+        final Projection projection = RdfTestUtil.getProjection(
+                "SELECT (?person AS ?p) (?employee AS ?e) ?business " +
+                "WHERE { " +
+                    "?person <urn:talksTo> ?employee . " +
+                    "?employee <urn:worksAt> ?business . " +
+                "}");
+
+        // Create a Binding Set that contains the result of the WHERE clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet inputBs = new MapBindingSet();
+        inputBs.addBinding("person", vf.createURI("urn:Alice"));
+        inputBs.addBinding("employee", vf.createURI("urn:Bob"));
+        inputBs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        final VisibilityBindingSet inputVisBs = new VisibilityBindingSet(inputBs, "a");
+
+        // The expected binding set changes the "person" binding name to "p" and "employee" to "e".
+        final MapBindingSet expectedBs = new MapBindingSet();
+        expectedBs.addBinding("p", vf.createURI("urn:Alice"));
+        expectedBs.addBinding("e", vf.createURI("urn:Bob"));
+        expectedBs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        final VisibilityBindingSet expectedVisBs = new VisibilityBindingSet(expectedBs, "a");
+
+        // Show it resulted in the correct output BindingSet.
+        final ProcessorContext context = mock(ProcessorContext.class);
+        final ProjectionProcessor processor = new ProjectionProcessor(
+                ProjectionEvaluator.make(projection),
+                result -> ProcessorResult.make(new UnaryResult(result)));
+        processor.init(context);
+
+        processor.process("key", ProcessorResult.make(new UnaryResult(inputVisBs)));
+
+        // Verify the expected binding set was emitted.
+        final ProcessorResult expected = ProcessorResult.make(new UnaryResult(expectedVisBs));
+        verify(context, times(1)).forward(eq("key"), eq(expected));
+    }
+}
\ No newline at end of file


[04/50] [abbrv] incubator-rya git commit: RYA-377 Implement the JoinProcessor for KafkaStreams.

Posted by ca...@apache.org.
RYA-377 Implement the JoinProcessor for KafkaStreams.


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

Branch: refs/heads/master
Commit: 3471cb7eedaadde5352c584cfa4135a4dbd66520
Parents: c4966ff
Author: kchilton2 <ke...@gmail.com>
Authored: Fri Nov 10 13:44:25 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 extras/rya.streams/kafka/pom.xml                |   5 +-
 .../StatementPatternProcessorSupplier.java      |   5 +
 .../processors/join/CloseableIterator.java      |  32 ++
 .../processors/join/JoinProcessorSupplier.java  | 191 +++++++
 .../kafka/processors/join/JoinStateStore.java   |  11 +-
 .../processors/join/KeyValueJoinStateStore.java | 259 ++++++++++
 .../apache/rya/streams/kafka/KafkaTestUtil.java |  82 +++
 .../processors/StatementPatternProcessorIT.java | 240 +++++++--
 .../kafka/processors/join/JoinProcessorIT.java  | 507 +++++++++++++++++++
 .../kafka/src/test/resources/log4j.properties   |  29 ++
 10 files changed, 1296 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/pom.xml b/extras/rya.streams/kafka/pom.xml
index 33cc985..d3f6891 100644
--- a/extras/rya.streams/kafka/pom.xml
+++ b/extras/rya.streams/kafka/pom.xml
@@ -57,11 +57,12 @@ under the License.
             <groupId>org.apache.kafka</groupId>
             <artifactId>kafka-clients</artifactId>
         </dependency>
-       <dependency>
+        <dependency>
             <groupId>org.apache.kafka</groupId>
             <artifactId>kafka-streams</artifactId>
         </dependency>
  
+        <!-- Misc. dependencies -->
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
@@ -79,4 +80,4 @@ under the License.
             <scope>test</scope>
         </dependency>
     </dependencies>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
index 6991783..386fe98 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
@@ -30,6 +30,8 @@ import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.algebra.StatementPattern;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -70,6 +72,8 @@ public class StatementPatternProcessorSupplier implements ProcessorSupplier<Stri
     @DefaultAnnotation(NonNull.class)
     public static final class StatementPatternProcessor implements Processor<String, VisibilityStatement> {
 
+        private static final Logger log = LoggerFactory.getLogger(StatementPatternProcessor.class);
+
         private final StatementPatternMatcher spMatcher;
         private final ProcessorResultFactory resultFactory;
 
@@ -104,6 +108,7 @@ public class StatementPatternProcessorSupplier implements ProcessorSupplier<Stri
 
                 // Wrap the binding set as a result and forward it to the downstream processor.
                 final ProcessorResult resultValue = resultFactory.make(visBs);
+                log.debug("\nOUTPUT:\n{}", visBs);
                 context.forward(key, resultValue);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/CloseableIterator.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/CloseableIterator.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/CloseableIterator.java
new file mode 100644
index 0000000..9ea927d
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/CloseableIterator.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.join;
+
+import java.util.Iterator;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * An {@link Iterator} that is also {@link AutoCloseable}.
+ *
+ * @param <T> - The type of elements that will be iterated over.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface CloseableIterator<T> extends Iterator<T>, AutoCloseable { }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorSupplier.java
new file mode 100644
index 0000000..9ed2363
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorSupplier.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.join;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.rya.api.function.join.IterativeJoin;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
+import org.apache.rya.streams.kafka.processors.ProcessorResultFactory;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessor;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessorSupplier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Supplies {@link JoinProcessor} instances.
+ */
+@DefaultAnnotation(NonNull.class)
+public class JoinProcessorSupplier extends RyaStreamsProcessorSupplier {
+
+    private final String stateStoreName;
+    private final IterativeJoin join;
+    private final List<String> joinVars;
+    private final List<String> allVars;
+
+    /**
+     * Constructs an instance of {@link JoinProcessorSupplier}.
+     *
+     * @param stateStoreName - The name of the state store the processor will use. (not null)
+     * @param join - The join function the supplied processor will use. (not null)
+     * @param joinVars - The variables that the supplied processor will join over. (not null)
+     * @param allVars - An ordered list of all the variables that may appear in resulting Binding Sets.
+     *   This list must lead with the same variables and order as {@code joinVars}. (not null)
+     * @param resultFactory - The factory that the supplied processors will use to create results. (not null)
+     * @throws IllegalArgumentException Thrown if {@code allVars} does not start with {@code joinVars}.
+     */
+    public JoinProcessorSupplier(
+            final String stateStoreName,
+            final IterativeJoin join,
+            final List<String> joinVars,
+            final List<String> allVars,
+            final ProcessorResultFactory resultFactory) throws IllegalArgumentException {
+        super(resultFactory);
+        this.stateStoreName = requireNonNull(stateStoreName);
+        this.join = requireNonNull(join);
+        this.joinVars = requireNonNull(joinVars);
+        this.allVars = requireNonNull(allVars);
+
+        if(!allVars.subList(0, joinVars.size()).equals(joinVars)) {
+            throw new IllegalArgumentException("All vars must be lead by the join vars, but it did not. " +
+                    "Join Vars: " + joinVars + ", All Vars: " + allVars);
+        }
+    }
+
+    @Override
+    public Processor<Object, ProcessorResult> get() {
+        return new JoinProcessor(stateStoreName, join, joinVars, allVars, super.getResultFactory());
+    }
+
+    /**
+     * Joins {@link VisibilityBindingSet}s against all binding sets that were emitted on the other side. This function
+     * does not have an age off policy, so it will match everything that could have ever possibly matched, however this
+     * may become prohibitive for joins that match a large volume of binding sets since this will indefinitely grow
+     * within the state store.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static class JoinProcessor extends RyaStreamsProcessor {
+
+        private static final Logger log = LoggerFactory.getLogger(JoinProcessor.class);
+
+        private final String stateStoreName;
+        private final IterativeJoin join;
+        private final List<String> joinVars;
+        private final List<String> allVars;
+        private final ProcessorResultFactory resultFactory;
+
+        private ProcessorContext context;
+        private JoinStateStore joinStateStore;
+
+        /**
+         * Constructs an instance of {@link JoinProcessor}.
+         *
+         * @param stateStoreName - The name of the state store the processor will use. (not null)
+         * @param join - The join function that the processor will use. (not null)
+         * @param joinVars - The variables that the processor will join over. (not null)
+         * @param allVars - An ordered list of all the variables that may appear in resulting Binding Sets.
+         *   This list must lead with the same variables and order as {@code joinVars}. (not null)
+         * @param resultFactory - The factory that will format this processor's final results
+         *   for the downstream processor. (not null)
+         */
+        public JoinProcessor(
+                final String stateStoreName,
+                final IterativeJoin join,
+                final List<String> joinVars,
+                final List<String> allVars,
+                final ProcessorResultFactory resultFactory) {
+            super(resultFactory);
+            this.stateStoreName = requireNonNull(stateStoreName);
+            this.join = requireNonNull(join);
+            this.joinVars = requireNonNull(joinVars);
+            this.allVars = requireNonNull(allVars);
+            this.resultFactory = requireNonNull(resultFactory);
+
+            if(!allVars.subList(0, joinVars.size()).equals(joinVars)) {
+                throw new IllegalArgumentException("All vars must be lead by the join vars, but it did not. " +
+                        "Join Vars: " + joinVars + ", All Vars: " + allVars);
+            }
+        }
+
+        @Override
+        public void init(final ProcessorContext context) {
+            // Hold onto the context so that we can foward results.
+            this.context = context;
+
+            // Get a reference to the state store that keeps track of what can be joined with.
+            final KeyValueStore<String, VisibilityBindingSet> stateStore =
+                    (KeyValueStore<String, VisibilityBindingSet>) context.getStateStore( stateStoreName );
+            joinStateStore = new KeyValueJoinStateStore( stateStore, joinVars, allVars );
+        }
+
+        @Override
+        public void process(final Object key, final ProcessorResult value) {
+            // Log the key/value that have been encountered.
+            log.debug("\nINPUT:\nSide: {}\nBinding Set: {}", value.getBinary().getSide(), value.getBinary().getResult());
+
+            // Must be a binary result.
+            final BinaryResult binary = value.getBinary();
+
+            // Store the new result in the state store so that future joins may include it.
+            joinStateStore.store(binary);
+
+            // Fetch the binding sets that the emitted value joins with.
+            try(final CloseableIterator<VisibilityBindingSet> otherSide = joinStateStore.getJoinedValues(binary)) {
+                // Create an iterator that performs the join operation.
+                final Iterator<VisibilityBindingSet> joinResults = binary.getSide() == Side.LEFT ?
+                        join.newLeftResult(binary.getResult(), otherSide) :
+                        join.newRightResult(otherSide, binary.getResult());
+
+                // Format each join result and forward it to the downstream processor.
+                while(joinResults.hasNext()) {
+                    final VisibilityBindingSet joinResult = joinResults.next();
+                    final ProcessorResult resultValue = resultFactory.make(joinResult);
+                    log.debug("\nOUTPUT:\n{}", joinResult);
+                    context.forward(key, resultValue);
+                }
+            } catch (final Exception e) {
+                final String msg = "Problem encountered while iterating over the other side's values within the state store.";
+                log.error(msg, e);
+                throw new RuntimeException(msg, e);
+            }
+        }
+
+        @Override
+        public void punctuate(final long timestamp) {
+            // Nothing to do.
+        }
+
+        @Override
+        public void close() {
+            // Nothing to do.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
index e53ec68..17a6ebb 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
@@ -18,9 +18,6 @@
  */
 package org.apache.rya.streams.kafka.processors.join;
 
-import java.util.Iterator;
-import java.util.List;
-
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
 
@@ -37,17 +34,15 @@ public interface JoinStateStore {
     /**
      * Store a {@link VisibilityBindingSet} based on the side it was emitted from.
      *
-     * @param joinVars - An ordered list of the variables that are being joined over. (not null)
      * @param result - The result whose value will be stored. (not null)
      */
-    public void store(List<String> joinVars, BinaryResult result);
+    public void store(BinaryResult result);
 
     /**
      * Get the previously stored {@link VisibilityBindingSet}s that join with the provided result.
      *
-     * @param joinVars - An ordered list of the variables to join over. (not null)
-     * @param result - Defines the values that will be used to join. (not null)
+     * @param result - The value that will be joined with. (not null)
      * @return The {@link VisibilityBinidngSet}s that join with {@code result}.
      */
-    public Iterator<VisibilityBindingSet> getJoinedValues(List<String> joinVars, BinaryResult result);
+    public CloseableIterator<VisibilityBindingSet> getJoinedValues(BinaryResult result);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
new file mode 100644
index 0000000..d73b40e
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.join;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
+import org.openrdf.query.impl.MapBindingSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Joiner;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A {@link KeyValueStore} implementation of {@link JoinStateStore}.
+ * </p>
+ * This is a key/value store, so we need to store the {@link VisibilityBindingSet}s using keys that allow us to fetch
+ * all binding sets that join from a specific side. We use the following pattern to accomplish this:
+ * <pre>
+ * [side],[joinVar1 value], [joinVar2 value], ..., [joinVarN value]
+ * </pre>
+ * This will group all binding sets that have been emitted from a specific side and who have the same join variables
+ * next to each other within the store. This isn't enough information to fetch that group, though. We must provide a
+ * start and end key to bound the range that is fetched back. To accomplish this, we place a start of range marker
+ * as the first key for all unique [side]/[join values] groups, and an end of range marker as the last key for each
+ * of those groups.
+ * </p>
+ * The rows follow this pattern:
+ * <pre>
+ * [side],[joinVar1 value], [joinVar2 value], ..., [joinVarN value]0x00
+ * [side],[joinVar1 value], [joinVar2 value], ..., [joinVarN value],[remainingBindingValues]
+ * [side],[joinVar1 value], [joinVar2 value], ..., [joinVarN value]0xFF
+ * </pre>
+ * </p>
+ * When an iterator over the results is returned, it skips over the start and end of range markers.
+ */
+@DefaultAnnotation(NonNull.class)
+public class KeyValueJoinStateStore implements JoinStateStore {
+
+    private static final Logger log = LoggerFactory.getLogger(KeyValueJoinStateStore.class);
+
+    /**
+     * This is the minimum value in UTF-8 character.
+     */
+    private static final String START_RANGE_SUFFIX = new String(new byte[] { 0x00 } );
+
+    /**
+     * This is the maximum value of a UTF-8 character.
+     */
+    private static final String END_RANGE_SUFFIX = new String(new byte[] { (byte) 0XFF } );
+
+    /**
+     * A default empty value that is stored for a start of range or end of range marker.
+     */
+    private static final VisibilityBindingSet RANGE_MARKER_VALUE = new VisibilityBindingSet(new MapBindingSet(), "");
+
+    private final KeyValueStore<String, VisibilityBindingSet> store;
+    private final List<String> joinVars;
+    private final List<String> allVars;
+
+    /**
+     * Constructs an instance of {@link KeyValueJoinStateStore}.
+     *
+     * @param store - The state store that will be used. (not null)
+     * @param joinVars - The variables that are used to build grouping keys. (not null)
+     * @param allVars - The variables that are used to build full value keys. (not null)
+     * @throws IllegalArgumentException Thrown if {@code allVars} does not start with {@code joinVars}.
+     */
+    public KeyValueJoinStateStore(
+            final KeyValueStore<String, VisibilityBindingSet> store,
+            final List<String> joinVars,
+            final List<String> allVars) throws IllegalArgumentException {
+        this.store = requireNonNull(store);
+        this.joinVars = requireNonNull(joinVars);
+        this.allVars = requireNonNull(allVars);
+
+        for(int i = 0; i < joinVars.size(); i++) {
+            if(!joinVars.get(i).equals(allVars.get(i))) {
+                throw new IllegalArgumentException("All vars must be lead by the join vars, but it did not. " +
+                        "Join Vars: " + joinVars + ", All Vars: " + allVars);
+            }
+        }
+    }
+
+    @Override
+    public void store(final BinaryResult result) {
+        requireNonNull(result);
+
+        // The join key prefix is an ordered list of values from the binding set that match the join variables.
+        // This is a prefix for every row that holds values for a specific set of join variable values.
+        final Side side = result.getSide();
+        final VisibilityBindingSet bs = result.getResult();
+        final String joinKeyPrefix = makeCommaDelimitedValues(side, joinVars, bs);
+
+        final List<KeyValue<String, VisibilityBindingSet>> values = new ArrayList<>();
+
+        // For each join variable set, we need a start key for scanning,
+        final String startKey = joinKeyPrefix + START_RANGE_SUFFIX;
+        values.add( new KeyValue<>(startKey, RANGE_MARKER_VALUE) );
+
+        // The actual value that was emitted as a result.
+        final String valueKey = makeCommaDelimitedValues(side, allVars, bs);
+        values.add( new KeyValue<>(valueKey, bs) );
+
+        // And the end key for scanning.
+        final String endKey = joinKeyPrefix + END_RANGE_SUFFIX;
+        values.add( new KeyValue<>(endKey, RANGE_MARKER_VALUE) );
+
+        // Write the pairs to the store.
+        log.debug("\nStoring the following values: {}\n", values);
+        store.putAll( values );
+    }
+
+    @Override
+    public CloseableIterator<VisibilityBindingSet> getJoinedValues(final BinaryResult result) {
+        requireNonNull(result);
+
+        // Get an iterator over the values that start with the join variables for the other side.
+        final Side otherSide = result.getSide() == Side.LEFT ? Side.RIGHT : Side.LEFT;
+        final VisibilityBindingSet bs = result.getResult();
+        final String joinKeyPrefix = makeCommaDelimitedValues(otherSide, joinVars, bs);
+
+        final String startKey = joinKeyPrefix + START_RANGE_SUFFIX;
+        final String endKey = joinKeyPrefix + END_RANGE_SUFFIX;
+        final KeyValueIterator<String, VisibilityBindingSet> rangeIt = store.range(startKey, endKey);
+
+        // Return a CloseableIterator over the range's value fields, skipping the start and end entry.
+        return new CloseableIterator<VisibilityBindingSet>() {
+
+            private Optional<VisibilityBindingSet> next = null;
+
+            @Override
+            public boolean hasNext() {
+                // If the iterator has not been initialized yet, read a value in.
+                if(next == null) {
+                    next = readNext();
+                }
+
+                // Return true if there is a next value, otherwise false.
+                return next.isPresent();
+            }
+
+            @Override
+            public VisibilityBindingSet next() {
+                // If the iterator has not been initialized yet, read a value in.
+                if(next == null) {
+                    next = readNext();
+                }
+
+                // It's illegal to call next() when there is no next value.
+                if(!next.isPresent()) {
+                    throw new IllegalStateException("May not invoke next() when there is nothing left in the Iterator.");
+                }
+
+                // Update and return the next value.
+                final VisibilityBindingSet ret = next.get();
+                log.debug("\nReturning: {}", ret);
+                next = readNext();
+                return ret;
+            }
+
+            private Optional<VisibilityBindingSet> readNext() {
+                // Check to see if there's anything left in the iterator.
+                if(!rangeIt.hasNext()) {
+                    return Optional.empty();
+                }
+
+                // Read a candidate key/value pair from the iterator.
+                KeyValue<String, VisibilityBindingSet> candidate = rangeIt.next();
+
+                // If we are initializing, then the first thing we must read is a start of range marker.
+                if(next == null) {
+                    if(!candidate.key.endsWith(START_RANGE_SUFFIX)) {
+                        throw new IllegalStateException("The first key encountered must be a start of range key.");
+                    }
+                    log.debug("Read the start of range markers.\n");
+
+                    // Read a new candidate to skip this one.
+                    if(!rangeIt.hasNext()) {
+                        throw new IllegalStateException("There must be another entry after the start of range key.");
+                    }
+                    candidate = rangeIt.next();
+                }
+
+                // If that value is an end of range key, then we are finished. Otherwise, return it.
+                else if(candidate.key.endsWith(END_RANGE_SUFFIX)) {
+                    log.debug("Read the end of range marker.\n");
+
+                    // If there are more messages, that's a problem.
+                    if(rangeIt.hasNext()) {
+                        throw new IllegalStateException("The end of range marker must be the last key in the iterator.");
+                    }
+
+                    return Optional.empty();
+                }
+
+                // Otherwise we found a new value.
+                return Optional.of( candidate.value );
+            }
+
+            @Override
+            public void close() throws Exception {
+                rangeIt.close();
+            }
+        };
+    }
+
+    /**
+     * A utility function that helps construct the keys used by {@link KeyValueJoinStateStore}.
+     *
+     * @param side - The side value for the key. (not null)
+     * @param vars - Which variables within the binding set to use for the key's values. (not null)
+     * @param bindingSet - The binding set the key is being constructed from. (not null)
+     * @return A comma delimited list of the binding values, leading with the side.
+     */
+    private static String makeCommaDelimitedValues(final Side side, final List<String> vars, final VisibilityBindingSet bindingSet) {
+        requireNonNull(side);
+        requireNonNull(vars);
+        requireNonNull(bindingSet);
+
+        // Make a an ordered list of the binding set variables.
+        final List<String> values = new ArrayList<>();
+        values.add(side.toString());
+        for(final String var : vars) {
+            values.add( bindingSet.hasBinding(var) ? bindingSet.getBinding(var).getValue().toString() : "" );
+        }
+
+        // Return a comma delimited list of those values.
+        return Joiner.on(",").join(values);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
index bff4fdb..0a1a8a4 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
@@ -19,10 +19,13 @@
 package org.apache.rya.streams.kafka;
 
 import static java.util.Objects.requireNonNull;
+import static org.junit.Assert.assertEquals;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Properties;
+import java.util.Set;
 import java.util.UUID;
 
 import org.apache.kafka.clients.consumer.Consumer;
@@ -34,8 +37,20 @@ import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 
+import com.google.common.collect.Sets;
+
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
 
@@ -124,4 +139,71 @@ public final class KafkaTestUtil {
 
         return values;
     }
+
+    /**
+     * Runs a Kafka Streams topology, loads statements into the input topic, read the binding sets that come out of
+     * the results topic, and ensures the expected results match the read results.
+     *
+     * @param kafka - The embedded kafka instance that is being tested with. (not null)
+     * @param statementsTopic - The topic statements will be written to. (not null)
+     * @param resultsTopic - The topic results will be read from. (not null)
+     * @param builder - The streams topology that will be executed. (not null)
+     * @param startupMs - How long to wait for the topology to start before writing the statements.
+     * @param statements - The statements that will be loaded into the topic. (not null)
+     * @param expected - The expected results. (not null)
+     * @throws Exception If any exception was thrown while running the test.
+     */
+    public static void runStreamProcessingTest(
+            final KafkaTestInstanceRule kafka,
+            final String statementsTopic,
+            final String resultsTopic,
+            final TopologyBuilder builder,
+            final int startupMs,
+            final List<VisibilityStatement> statements,
+            final Set<VisibilityBindingSet> expected) throws Exception {
+        requireNonNull(kafka);
+        requireNonNull(statementsTopic);
+        requireNonNull(resultsTopic);
+        requireNonNull(builder);
+        requireNonNull(statements);
+        requireNonNull(expected);
+
+        // Explicitly create the topics that are being used.
+        kafka.createTopic(statementsTopic);
+        kafka.createTopic(resultsTopic);
+
+        // Start the streams program.
+        final Properties props = kafka.createBootstrapServerConfig();
+        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "StatementPatternProcessorIT");
+
+        final KafkaStreams streams = new KafkaStreams(builder, new StreamsConfig(props));
+        streams.cleanUp();
+        try {
+            streams.start();
+
+            // Wait for the streams application to start. Streams only see data after their consumers are connected.
+            Thread.sleep(startupMs);
+
+            // Load the statements into the input topic.
+            try(Producer<String, VisibilityStatement> producer = KafkaTestUtil.makeProducer(
+                    kafka, StringSerializer.class, VisibilityStatementSerializer.class)) {
+                new KafkaLoadStatements(statementsTopic, producer).fromCollection(statements);
+            }
+
+            // Wait for the final results to appear in the output topic and verify the expected Binding Sets were found.
+            try(Consumer<String, VisibilityBindingSet> consumer = KafkaTestUtil.fromStartConsumer(
+                    kafka, StringDeserializer.class, VisibilityBindingSetDeserializer.class)) {
+                // Register the topic.
+                consumer.subscribe(Arrays.asList(resultsTopic));
+
+                // Poll for the result.
+                final Set<VisibilityBindingSet> results = Sets.newHashSet( KafkaTestUtil.pollForResults(500, 6, expected.size(), consumer) );
+
+                // Show the correct binding sets results from the job.
+                assertEquals(expected, results);
+            }
+        } finally {
+            streams.close();
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
index 1b58b42..371fd0b 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
@@ -18,34 +18,25 @@
  */
 package org.apache.rya.streams.kafka.processors;
 
-import static org.junit.Assert.assertEquals;
-
 import java.util.ArrayList;
-import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Properties;
+import java.util.Set;
 import java.util.UUID;
 
-import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
-import org.apache.kafka.streams.KafkaStreams;
-import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.RdfTestUtil;
-import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
 import org.apache.rya.streams.kafka.processors.RyaStreamsSinkFormatterSupplier.RyaStreamsSinkFormatter;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
-import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -63,7 +54,50 @@ public class StatementPatternProcessorIT {
     public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
 
     @Test
-    public void statementPatternMatches() throws Exception {
+    public void singlePattern_singleStatement() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPattern object that will be evaluated.
+        final StatementPattern sp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "SP1");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Create a statement that generate an SP result.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        final QueryBindingSet bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+    }
+
+    @Test
+    public void singlePattern_manyStatements() throws Exception {
         // Enumerate some topics that will be re-used
         final String ryaInstance = UUID.randomUUID().toString();
         final UUID queryId = UUID.randomUUID();
@@ -88,48 +122,144 @@ public class StatementPatternProcessorIT {
         // Add a sink that writes the data out to a new Kafka topic.
         builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
 
-        // Start the streams program.
-        final Properties props = kafka.createBootstrapServerConfig();
-        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "StatementPatternProcessorIT");
-
-        final KafkaStreams streams = new KafkaStreams(builder, new StreamsConfig(props));
-        streams.cleanUp();
-        try {
-            streams.start();
-
-            // Wait for the streams application to start. Streams only see data after their consumers are connected.
-            Thread.sleep(2000);
-
-            // Load some data into the input topic.
-            final ValueFactory vf = new ValueFactoryImpl();
-            final List<VisibilityStatement> statements = new ArrayList<>();
-            statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
-
-            try(Producer<String, VisibilityStatement> producer = KafkaTestUtil.makeProducer(
-                    kafka, StringSerializer.class, VisibilityStatementSerializer.class)) {
-                new KafkaLoadStatements(statementsTopic, producer).fromCollection(statements);
-            }
-
-            // Wait for the final results to appear in the output topic and verify the expected Binding Set was found.
-            try(Consumer<String, VisibilityBindingSet> consumer = KafkaTestUtil.fromStartConsumer(
-                    kafka, StringDeserializer.class, VisibilityBindingSetDeserializer.class)) {
-                // Register the topic.
-                consumer.subscribe(Arrays.asList(resultsTopic));
-
-                // Poll for the result.
-                final List<VisibilityBindingSet> results = KafkaTestUtil.pollForResults(500, 6, 1, consumer);
-
-                // Show the correct binding set results from the job.
-                final QueryBindingSet bs = new QueryBindingSet();
-                bs.addBinding("person", vf.createURI("urn:Alice"));
-                bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
-                final VisibilityBindingSet expected = new VisibilityBindingSet(bs, "a");
-
-                final VisibilityBindingSet result = results.iterator().next();
-                assertEquals(expected, result);
-            }
-        } finally {
-            streams.close();
-        }
+        // Create some statements where some generates SP results and others do not.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoJoin")), "b") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:talksTo"), vf.createURI("urn:Alice")), "a|b") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "c") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        QueryBindingSet bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Alice"));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+    }
+
+    @Test
+    public void multiplePatterns_singleStatement() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPattern object that will be evaluated.
+        final StatementPattern sp1 = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
+        final StatementPattern sp2 = RdfTestUtil.getSp("SELECT * WHERE { ?person ?action <urn:Bob> }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp1, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
+
+        // Add a processor that handles the second statement pattern.
+        builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "SP1", "SP2");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Create some statements where some generates SP results and others do not.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        QueryBindingSet bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("action", vf.createURI("urn:talksTo"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+    }
+
+    @Test
+    public void multiplePatterns_multipleStatements() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPattern object that will be evaluated.
+        final StatementPattern sp1 = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
+        final StatementPattern sp2 = RdfTestUtil.getSp("SELECT * WHERE { ?person ?action <urn:Bob> }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp1, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
+
+        // Add a processor that handles the second statement pattern.
+        builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "SP1", "SP2");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Create some statements where some generates SP results and others do not.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Charlie")), "a|b") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:walksWith"), vf.createURI("urn:Bob")), "b") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        QueryBindingSet bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("action", vf.createURI("urn:talksTo"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Charlie"));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Charlie"));
+        bs.addBinding("action", vf.createURI("urn:walksWith"));
+        expected.add( new VisibilityBindingSet(bs, "b") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
new file mode 100644
index 0000000..14a559f
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
@@ -0,0 +1,507 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.join;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.processor.StateStoreSupplier;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.rya.api.function.join.LeftOuterJoin;
+import org.apache.rya.api.function.join.NaturalJoin;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RdfTestUtil;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.RyaStreamsSinkFormatterSupplier.RyaStreamsSinkFormatter;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier.JoinProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerde;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Integration tests the methods of {@link JoinProcessor}.
+ */
+public class JoinProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test(expected = IllegalArgumentException.class)
+    public void badAllVars() throws IllegalArgumentException {
+        new JoinProcessorSupplier(
+                "NATURAL_JOIN",
+                new NaturalJoin(),
+                Lists.newArrayList("employee"),
+                Lists.newArrayList("person", "employee", "business"),
+                result -> ProcessorResult.make( new UnaryResult(result) ));
+    }
+
+    @Test
+    public void newLeftResult() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPatterns that will be evaluated.
+        final StatementPattern leftSp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
+        final StatementPattern rightSp = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("LEFT_SP", new StatementPatternProcessorSupplier(leftSp,
+                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
+
+        // Add a processor that handles the second statement pattern.
+        builder.addProcessor("RIGHT_SP", new StatementPatternProcessorSupplier(rightSp,
+                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
+
+        // Add a processor that handles a natrual join over the SPs.
+        builder.addProcessor("NATURAL_JOIN", new JoinProcessorSupplier(
+                "NATURAL_JOIN",
+                new NaturalJoin(),
+                Lists.newArrayList("employee"),
+                Lists.newArrayList("employee", "person", "business"),
+                result -> ProcessorResult.make( new UnaryResult(result) )), "LEFT_SP", "RIGHT_SP");
+
+        // Add a state store for the join processor.
+        final StateStoreSupplier joinStoreSupplier =
+                Stores.create( "NATURAL_JOIN" )
+                  .withStringKeys()
+                  .withValues(new VisibilityBindingSetSerde())
+                  .inMemory()
+                  .build();
+        builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "NATURAL_JOIN");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Create some statements that generate a bunch of right SP results.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "a&b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), "b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "b|c") );
+
+        // Add a statement that will generate a left result that joins with some of those right results.
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "c") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
+        expected.add( new VisibilityBindingSet(bs, "a&b&c") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+    }
+
+    @Test
+    public void newRightResult() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPatterns that will be evaluated.
+        final StatementPattern leftSp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
+        final StatementPattern rightSp = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("LEFT_SP", new StatementPatternProcessorSupplier(leftSp,
+                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
+
+        // Add a processor that handles the second statement pattern.
+        builder.addProcessor("RIGHT_SP", new StatementPatternProcessorSupplier(rightSp,
+                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
+
+        // Add a processor that handles a natrual join over the SPs.
+        builder.addProcessor("NATURAL_JOIN", new JoinProcessorSupplier(
+                "NATURAL_JOIN",
+                new NaturalJoin(),
+                Lists.newArrayList("employee"),
+                Lists.newArrayList("employee", "person", "business"),
+                result -> ProcessorResult.make( new UnaryResult(result) )), "LEFT_SP", "RIGHT_SP");
+
+        // Add a state store for the join processor.
+        final StateStoreSupplier joinStoreSupplier =
+                Stores.create( "NATURAL_JOIN" )
+                  .withStringKeys()
+                  .withValues(new VisibilityBindingSetSerde())
+                  .inMemory()
+                  .build();
+        builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "NATURAL_JOIN");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Create some statements that generate a bunch of right SP results.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "a&b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), "b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "b|c") );
+
+        // Add a statement that will generate a left result that joins with some of those right results.
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "c") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
+        expected.add( new VisibilityBindingSet(bs, "a&b&c") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+    }
+
+    @Test
+    public void newResultsBothSides() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPatterns that will be evaluated.
+        final StatementPattern leftSp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
+        final StatementPattern rightSp = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("LEFT_SP", new StatementPatternProcessorSupplier(leftSp,
+                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
+
+        // Add a processor that handles the second statement pattern.
+        builder.addProcessor("RIGHT_SP", new StatementPatternProcessorSupplier(rightSp,
+                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
+
+        // Add a processor that handles a natrual join over the SPs.
+        builder.addProcessor("NATURAL_JOIN", new JoinProcessorSupplier(
+                "NATURAL_JOIN",
+                new NaturalJoin(),
+                Lists.newArrayList("employee"),
+                Lists.newArrayList("employee", "person", "business"),
+                result -> ProcessorResult.make( new UnaryResult(result) )), "LEFT_SP", "RIGHT_SP");
+
+        // Add a state store for the join processor.
+        final StateStoreSupplier joinStoreSupplier =
+                Stores.create( "NATURAL_JOIN" )
+                  .withStringKeys()
+                  .withValues(new VisibilityBindingSetSerde())
+                  .inMemory()
+                  .build();
+        builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "NATURAL_JOIN");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Create some statements that generate a bunch of right SP results.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "a&b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "c") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), "b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "b|c") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:talksTo"), vf.createURI("urn:Charlie")), "c") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
+        expected.add( new VisibilityBindingSet(bs, "a&b&c") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("employee", vf.createURI("urn:Charlie"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add( new VisibilityBindingSet(bs, "a&c") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+    }
+
+    @Test
+    public void manyJoins() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPatterns that will be evaluated.
+        final StatementPattern sp1 = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
+        final StatementPattern sp2 = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
+        final StatementPattern sp3 = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:hourlyWage> ?wage }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp1,
+                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
+
+        // Add a processor that handles the second statement pattern.
+        builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2,
+                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
+
+        // Add a processor that handles a natural join over SPs 1 and 2.
+        builder.addProcessor("JOIN1", new JoinProcessorSupplier(
+                "JOIN1",
+                new NaturalJoin(),
+                Lists.newArrayList("employee"),
+                Lists.newArrayList("employee", "person", "business"),
+                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "SP1", "SP2");
+
+        // Add a processor that handles the third statement pattern.
+        builder.addProcessor("SP3", new StatementPatternProcessorSupplier(sp3,
+                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
+
+        // Add a processor that handles a natural join over JOIN1 and SP3.
+        builder.addProcessor("JOIN2", new JoinProcessorSupplier(
+                "JOIN2",
+                new NaturalJoin(),
+                Lists.newArrayList("employee"),
+                Lists.newArrayList("employee", "business", "wage"),
+                result -> ProcessorResult.make( new UnaryResult(result) )), "JOIN1", "SP3");
+
+        // Setup the join state suppliers.
+        final StateStoreSupplier join1StoreSupplier =
+                Stores.create( "JOIN1" )
+                  .withStringKeys()
+                  .withValues(new VisibilityBindingSetSerde())
+                  .inMemory()
+                  .build();
+        builder.addStateStore(join1StoreSupplier, "JOIN1");
+
+        final StateStoreSupplier join2StoreSupplier =
+                Stores.create( "JOIN2" )
+                  .withStringKeys()
+                  .withValues(new VisibilityBindingSetSerde())
+                  .inMemory()
+                  .build();
+        builder.addStateStore(join2StoreSupplier, "JOIN2");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "JOIN2");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Create some statements that generate a bunch of right SP results.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hourlyWage"), vf.createLiteral(7.25)), "a") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        bs.addBinding("wage", vf.createLiteral(7.25));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 3000, statements, expected);
+    }
+
+    @Test
+    public void leftJoin() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPatterns that will be evaluated.
+        final StatementPattern requiredSp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
+        final StatementPattern optionalSp = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("REQUIRED_SP", new StatementPatternProcessorSupplier(requiredSp,
+                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
+
+        // Add a processor that handles the second statement pattern.
+        builder.addProcessor("OPTIONAL_SP", new StatementPatternProcessorSupplier(optionalSp,
+                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
+
+        // Add a processor that handles a natrual join over the SPs.
+        builder.addProcessor("LEFT_JOIN", new JoinProcessorSupplier(
+                "LEFT_JOIN",
+                new LeftOuterJoin(),
+                Lists.newArrayList("employee"),
+                Lists.newArrayList("employee", "person", "business"),
+                result -> ProcessorResult.make( new UnaryResult(result) )), "REQUIRED_SP", "OPTIONAL_SP");
+
+        // Add a state store for the join processor.
+        final StateStoreSupplier joinStoreSupplier =
+                Stores.create( "LEFT_JOIN" )
+                  .withStringKeys()
+                  .withValues(new VisibilityBindingSetSerde())
+                  .inMemory()
+                  .build();
+        builder.addStateStore(joinStoreSupplier, "LEFT_JOIN");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "LEFT_JOIN");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Create some statements that generate a result that includes the optional value as well as one that does not.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:talksTo"), vf.createURI("urn:Charlie")), "c") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "d") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
+        expected.add( new VisibilityBindingSet(bs, "a&b") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("employee", vf.createURI("urn:Charlie"));
+        expected.add( new VisibilityBindingSet(bs, "c") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3471cb7e/extras/rya.streams/kafka/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/resources/log4j.properties b/extras/rya.streams/kafka/src/test/resources/log4j.properties
new file mode 100644
index 0000000..05e7be5
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Root logger option
+log4j.rootLogger=ERROR, stdout
+
+# Direct log messages to stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
+
+log4j.logger.org.apache.rya.streams.kafka.processors=debug


[17/50] [abbrv] incubator-rya git commit: RYA-377 Implemented a common RyaStreams Processor and ProcessorSupplier.

Posted by ca...@apache.org.
RYA-377 Implemented a common RyaStreams Processor and ProcessorSupplier.


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

Branch: refs/heads/master
Commit: 516e8908b5a359ba153ad48beaae5cb20fccfbc5
Parents: 07fcb5f
Author: kchilton2 <ke...@gmail.com>
Authored: Mon Nov 6 15:17:44 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../kafka/processors/ProcessorResult.java       | 205 +++++++++++++++++++
 .../processors/ProcessorResultFactory.java      |  43 ++++
 .../kafka/processors/RyaStreamsProcessor.java   |  56 +++++
 .../processors/RyaStreamsProcessorSupplier.java |  51 +++++
 4 files changed, 355 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/516e8908/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
new file mode 100644
index 0000000..ac3d849
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors;
+
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.rya.api.model.VisibilityBindingSet;
+
+import com.google.common.base.Optional;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Represents a value that is emitted from a Rya Streams {@link Processor}. We can't just emit a
+ * {@link VisibilityBindingSet} because some downstream processors require more information about
+ * which upstream processor is emitting the result in order to do their work.
+ * </p>
+ * Currently there are only two types processors:
+ * <ul>
+ *   <li>Unary Processor - A processor that only has a single upstream node feeding it input.</li>
+ *   <li>Binary Processor - A processor that has two upstream nodes feeding it input.</li>
+ * </ul>
+ * If a processor is emitting to a unary processor, then use {@link #make(UnaryResult)} to create its
+ * result. If it is emitting to a binary processor, then use {@link #make(BinaryResult)}.
+ */
+@DefaultAnnotation(NonNull.class)
+public class ProcessorResult {
+
+    private final ResultType type;
+    private final Optional<UnaryResult> unary;
+    private final Optional<BinaryResult> binary;
+
+    /**
+     * Constructs an instance of {@link ProcessorResult}. Private to force users to use the static factory methods.
+     *
+     * @param type - Indicates the type of result this object holds. (not null)
+     * @param unary - The unary result if that is this object's type. (not null)
+     * @param binary - The binary result if that is this object's type. (not null)
+     */
+    private  ProcessorResult(
+            final ResultType type,
+            final Optional<UnaryResult> unary,
+            final Optional<BinaryResult> binary) {
+        this.type = requireNonNull(type);
+        this.unary= requireNonNull(unary);
+        this.binary= requireNonNull(binary);
+    }
+
+    /**
+     * @return Indicates the type of result this object holds.
+     */
+    public ResultType getType() {
+        return type;
+    }
+
+    /**
+     * @return The unary result if that is this object's type.
+     * @throws IllegalStateException If this object's type is not {@link ResultType#UNARY}.
+     */
+    public UnaryResult getUnary() throws IllegalStateException {
+        checkState(type == ResultType.UNARY, "The ResultType must be " + ResultType.UNARY + " to invoke this method, " +
+                "but it is " + type + ".");
+        return unary.get();
+    }
+
+    /**
+     * @return The binary result if that is this object's type.
+     * @throws IllegalStateException If this object's type is not {@link ResultType#BINARY}.
+     */
+    public BinaryResult getBinary() throws IllegalStateException {
+        checkState(type == ResultType.BINARY, "The ResultType must be " + ResultType.BINARY + " to invoke this method, " +
+                "but it is " + type + ".");
+        return binary.get();
+    }
+
+    /**
+     * Creates a {@link ProcessorResult} using the supplied value.
+     *
+     * @param result - The result that will be held by the created object. (not null)
+     * @return An object holding the provided result.
+     */
+    public static ProcessorResult make(final UnaryResult result) {
+        requireNonNull(result);
+        return new ProcessorResult(ResultType.UNARY, Optional.of(result), Optional.absent());
+    }
+
+    /**
+     * Creates a {@link ProcessorResult} using the supplied value.
+     *
+     * @param result - The result that will be held by the created object. (not null)
+     * @return An object holding the provided result.
+     */
+    public static ProcessorResult make(final BinaryResult result) {
+        requireNonNull(result);
+        return new ProcessorResult(ResultType.BINARY, Optional.absent(), Optional.of(result));
+    }
+
+    /**
+     * Indicates the type of result held by a {@link ProcessorResult}.
+     */
+    public static enum ResultType {
+        /**
+         * The {@link ProcessorResult} holds a {@link UnaryResult}.
+         */
+        UNARY,
+
+        /**
+         * The {@link ProcessorResult} holds a {@link BinaryResult}.
+         */
+        BINARY;
+    }
+
+    /**
+     * The result of a Rya Streams {@link Processor} whose downstream processor is unary.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static final class UnaryResult {
+        private final VisibilityBindingSet result;
+
+        /**
+         * Constructs an instance of {@link UnaryResult}.
+         *
+         * @param result - The binding set that is being emitted to the downstream unary processor. (not null)
+         */
+        public UnaryResult(final VisibilityBindingSet result) {
+            this.result = requireNonNull(result);
+        }
+
+        /**
+         * @return The binding set that is being emitted to the downstream unary processor.
+         */
+        public VisibilityBindingSet getResult() {
+            return result;
+        }
+    }
+
+    /**
+     * The result of a Rya Streams {@link Processor} whose downstream processor is binary.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static final class BinaryResult {
+        private final Side side;
+        private final VisibilityBindingSet result;
+
+        /**
+         * Constructs an instance of {@link BinaryResult}.
+         *
+         * @param side - Which side of the downstream binary processor the result is being emitted to. (not null)
+         * @param result - The binding set that is being emitted to the downstream binary processor. (not null)
+         */
+        public BinaryResult(final Side side, final VisibilityBindingSet result) {
+            this.side = requireNonNull(side);
+            this.result = requireNonNull(result);
+        }
+
+        /**
+         * @return Which side of the downstream binary processor the result is being emitted to.
+         */
+        public Side getSide() {
+            return side;
+        }
+
+        /**
+         * @return The binding set that is being emitted to the downstream binary processor.
+         */
+        public VisibilityBindingSet getResult() {
+            return result;
+        }
+
+        /**
+         * A label that is used to by the downstream binary prcoessor to distinguish which upstream processor
+         * produced the {@link BinaryResult}.
+         */
+        public static enum Side {
+            /**
+             * The result is being emitted from the "left" upstream processor.
+             */
+            LEFT,
+
+            /**
+             * The result is being emitted from the "right" upstream processor.
+             */
+            RIGHT;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/516e8908/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResultFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResultFactory.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResultFactory.java
new file mode 100644
index 0000000..4bc42a9
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResultFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Augments a {@link VisibilityBidingSet] that needs to be output by a {@link RyaStreamsProcessor}
+ * so that it has all of the information the downstream processor needs as well as whatever
+ * key the downstream processor requires.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface ProcessorResultFactory {
+
+    /**
+     * Augments a {@link VisibilityBidingSet] that needs to be output by a {@link RyaStreamsProcessor}
+     * so that it has all of the information the downstream processor needs as well as whatever
+     * key the downstream processor requires.
+     *
+     * @param result - The result that is being emitted. (not null)
+     * @return A {@link ProcessorResult} that is formatted for the downstream processor.
+     */
+    public ProcessorResult make(VisibilityBindingSet result);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/516e8908/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsProcessor.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsProcessor.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsProcessor.java
new file mode 100644
index 0000000..f6262c8
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsProcessor.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * All Rya Streams {@link Processor} implementations emit {@link ProcessorResult} objects. This
+ * abstract class holds onto the {@link ProcessorResultFactory} that is used to augment results
+ * before sending them to the downstream processor via {@link ProcessorContext#forward(Object, Object)}.
+ */
+@DefaultAnnotation(NonNull.class)
+public abstract class RyaStreamsProcessor implements Processor<Object, ProcessorResult> {
+
+    private final ProcessorResultFactory resultFactory;
+
+    /**
+     * Constructs an instance of {@link RyaStreamsProcessor}.
+     *
+     * @param resultFactory - The {@link ProcessorResultFactory} the child class will used to format results
+     *   before sending them to {@link ProcessorContext#forward(Object, Object)}. (not null)
+     */
+    public RyaStreamsProcessor(final ProcessorResultFactory resultFactory) {
+        this.resultFactory = requireNonNull(resultFactory);
+    }
+
+    /**
+     * @return The {@link ProcessorResultFactory} the child class will used to format results
+     *   before sending them to {@link ProcessorContext#forward(Object, Object)}.
+     */
+    public ProcessorResultFactory getResultFactory() {
+        return resultFactory;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/516e8908/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsProcessorSupplier.java
new file mode 100644
index 0000000..fef0bba
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsProcessorSupplier.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.kafka.streams.processor.ProcessorSupplier;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A {@link ProcessorSupplier} that should be implemented for each {@link RyaStreamsProcessor} that is implemented.
+ */
+@DefaultAnnotation(NonNull.class)
+public abstract class RyaStreamsProcessorSupplier implements ProcessorSupplier<Object, ProcessorResult> {
+
+    private final ProcessorResultFactory resultFactory;
+
+    /**
+     * Constructs an instance of {@link RyaStreamsProcessorSupplier}.
+     *
+     * @param resultFactory - The {@link ProcessorResultFactory} that will be used by processors created by this class. (not null)
+     */
+    public RyaStreamsProcessorSupplier(final ProcessorResultFactory resultFactory) {
+        this.resultFactory = requireNonNull(resultFactory);
+    }
+
+    /**
+     * @return The {@link ProcessorResultFactory} that will be used by processors created by this class. (not null)
+     */
+    public ProcessorResultFactory getResultFactory() {
+        return resultFactory;
+    }
+}
\ No newline at end of file


[23/50] [abbrv] incubator-rya git commit: RYA-377 Updated the Rya Streams client to be able to stream VisibilityStatement results to the console.

Posted by ca...@apache.org.
RYA-377 Updated the Rya Streams client to be able to stream VisibilityStatement results to the console.


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

Branch: refs/heads/master
Commit: bd36443de7e824c9fe8f0a97d3ef7a75c223271c
Parents: 95df37a
Author: kchilton2 <ke...@gmail.com>
Authored: Thu Nov 30 17:25:31 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 extras/rya.streams/api/pom.xml                  |  5 ++
 .../streams/api/entity/QueryResultStream.java   | 13 ++--
 .../api/interactor/GetQueryResultStream.java    |  8 ++-
 .../interactor/defaults/DefaultAddQuery.java    | 16 ++++-
 .../defaults/DefaultAddQueryTest.java           | 64 ++++++++++++++++++++
 .../client/command/StreamResultsCommand.java    | 54 +++++++++++++++--
 extras/rya.streams/kafka/pom.xml                |  2 +-
 .../kafka/entity/KafkaQueryResultStream.java    | 11 ++--
 .../interactor/KafkaGetQueryResultStream.java   | 27 ++++++---
 .../interactor/KafkaGetQueryResultStreamIT.java | 59 ++++++++++++++----
 10 files changed, 216 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/api/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/pom.xml b/extras/rya.streams/api/pom.xml
index 2a1f51c..250028f 100644
--- a/extras/rya.streams/api/pom.xml
+++ b/extras/rya.streams/api/pom.xml
@@ -41,6 +41,11 @@ under the License.
         </dependency>
         
         <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-queryparser-sparql</artifactId>
+        </dependency>
+        
+        <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
index aa5dcfd..8f1e589 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
@@ -22,17 +22,18 @@ import static java.util.Objects.requireNonNull;
 
 import java.util.UUID;
 
-import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.api.exception.RyaStreamsException;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
 
 /**
- * An infinite stream of {@link VisibilityBindingSet}s that are the results of a query within Rya Streams.
+ * An infinite stream of values that are the results of a query within Rya Streams.
+ *
+ * @param <V> - The query results' value type.
  */
 @DefaultAnnotation(NonNull.class)
-public abstract class QueryResultStream implements AutoCloseable {
+public abstract class QueryResultStream<V> implements AutoCloseable {
 
     private final UUID queryId;
 
@@ -57,10 +58,10 @@ public abstract class QueryResultStream implements AutoCloseable {
      * Wait at most {@code timeoutMs} milliseconds for the next collection of results.
      *
      * @param timeoutMs - The number of milliseconds to at most wait for the next collection of results. (not null)
-     * @return The next collection of {@link VisibilityBindingSet}s that are the result of the query. Empty if
-     *   there where no new results within the timout period.
+     * @return The next collection of values that are the result of the query. Empty if there where no new results
+     *   within the timeout period.
      * @throws IllegalStateException If the stream has been closed.
      * @throws RyaStreamsException Could not fetch the next set of results.
      */
-    public abstract Iterable<VisibilityBindingSet> poll(long timeoutMs) throws IllegalStateException, RyaStreamsException;
+    public abstract Iterable<V> poll(long timeoutMs) throws IllegalStateException, RyaStreamsException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/GetQueryResultStream.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/GetQueryResultStream.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/GetQueryResultStream.java
index 9ca577c..951d060 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/GetQueryResultStream.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/GetQueryResultStream.java
@@ -28,9 +28,11 @@ import edu.umd.cs.findbugs.annotations.NonNull;
 
 /**
  * Get a {@link QueryResultStream} over the results of a query that is being managed by Rya Streams.
+ *
+ * @param <T> - The type of results that are in the result stream.
  */
 @DefaultAnnotation(NonNull.class)
-public interface GetQueryResultStream {
+public interface GetQueryResultStream<T> {
 
     /**
      * Stream all of the results that have been produced by a query.
@@ -39,7 +41,7 @@ public interface GetQueryResultStream {
      * @return A {@link QueryResultStream} that starts with the first result that was ever produced.
      * @throws RyaStreamsException Could not create the result stream.
      */
-    public QueryResultStream fromStart(UUID queryId) throws RyaStreamsException;
+    public QueryResultStream<T> fromStart(UUID queryId) throws RyaStreamsException;
 
     /**
      * Stream results that have been produced by a query after this method was invoked.
@@ -48,5 +50,5 @@ public interface GetQueryResultStream {
      * @return A {@link QueryResultStream} that only returns results that were produced after this method is invoked.
      * @throws RyaStreamsException Could not create the result stream.
      */
-    public QueryResultStream fromNow(UUID queryId) throws RyaStreamsException;
+    public QueryResultStream<T> fromNow(UUID queryId) throws RyaStreamsException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultAddQuery.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultAddQuery.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultAddQuery.java
index 9704322..f94835c 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultAddQuery.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/defaults/DefaultAddQuery.java
@@ -24,6 +24,8 @@ import org.apache.rya.streams.api.entity.StreamsQuery;
 import org.apache.rya.streams.api.exception.RyaStreamsException;
 import org.apache.rya.streams.api.interactor.AddQuery;
 import org.apache.rya.streams.api.queries.QueryRepository;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.parser.sparql.SPARQLParser;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -35,6 +37,8 @@ import edu.umd.cs.findbugs.annotations.NonNull;
 public class DefaultAddQuery implements AddQuery {
     private final QueryRepository repository;
 
+    private final SPARQLParser parser = new SPARQLParser();
+
     /**
      * Creates a new {@link DefaultAddQuery}.
      *
@@ -46,6 +50,16 @@ public class DefaultAddQuery implements AddQuery {
 
     @Override
     public StreamsQuery addQuery(final String query) throws RyaStreamsException {
+        requireNonNull(query);
+
+        // Make sure the SPARQL is valid.
+        try {
+            parser.parseQuery(query, null);
+        } catch (final MalformedQueryException e) {
+            throw new RyaStreamsException("Could not add the query because the SPARQL is invalid.", e);
+        }
+
+        // If it is, then store it in the repository.
         return repository.add(query);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/api/src/test/java/org/apache/rya/streams/api/interactor/defaults/DefaultAddQueryTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/test/java/org/apache/rya/streams/api/interactor/defaults/DefaultAddQueryTest.java b/extras/rya.streams/api/src/test/java/org/apache/rya/streams/api/interactor/defaults/DefaultAddQueryTest.java
new file mode 100644
index 0000000..88be6e7
--- /dev/null
+++ b/extras/rya.streams/api/src/test/java/org/apache/rya/streams/api/interactor/defaults/DefaultAddQueryTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.api.interactor.defaults;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+import org.apache.rya.streams.api.interactor.AddQuery;
+import org.apache.rya.streams.api.queries.QueryRepository;
+import org.junit.Test;
+
+/**
+ * Unit tests the methods of {@link DefaultAddQuery}.
+ */
+public class DefaultAddQueryTest {
+
+    @Test
+    public void addQuery_validSparql() throws Exception {
+        // Valid SPARQL.
+        final String sparql = "SELECT * WHERE { ?person <urn:worksAt> ?business }";
+
+        // Setup the interactor.
+        final QueryRepository repo = mock(QueryRepository.class);
+        final AddQuery addQuery = new DefaultAddQuery(repo);
+
+        // Add the query.
+        addQuery.addQuery(sparql);
+
+        // Verify the call was forwarded to the repository.
+        verify(repo, times(1)).add(eq(sparql));
+    }
+
+    @Test(expected = RyaStreamsException.class)
+    public void addQuery_invalidSparql() throws Exception {
+        // Inalid SPARQL.
+        final String sparql = "This is not sparql.";
+
+        // Setup the interactor.
+        final QueryRepository repo = mock(QueryRepository.class);
+        final AddQuery addQuery = new DefaultAddQuery(repo);
+
+        // Add the query.
+        addQuery.addQuery(sparql);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
index 64f78a3..7c548f1 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/StreamResultsCommand.java
@@ -20,14 +20,26 @@ package org.apache.rya.streams.client.command;
 
 import static java.util.Objects.requireNonNull;
 
+import java.util.Optional;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.api.entity.QueryResultStream;
+import org.apache.rya.streams.api.entity.StreamsQuery;
 import org.apache.rya.streams.api.interactor.GetQueryResultStream;
+import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
+import org.apache.rya.streams.api.queries.QueryChangeLog;
+import org.apache.rya.streams.api.queries.QueryRepository;
 import org.apache.rya.streams.client.RyaStreamsCommand;
+import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.interactor.KafkaGetQueryResultStream;
+import org.apache.rya.streams.kafka.queries.KafkaQueryChangeLogFactory;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.algebra.Reduced;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.parser.sparql.SPARQLParser;
 
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
@@ -107,6 +119,12 @@ public class StreamResultsCommand implements RyaStreamsCommand {
             throw new ArgumentsException("Could not stream the query's results because of invalid command line parameters.", e);
         }
 
+        // Create the Kafka backed QueryChangeLog.
+        final String bootstrapServers = params.kafkaIP + ":" + params.kafkaPort;
+        final String topic = KafkaTopics.queryChangeLogTopic(params.ryaInstance);
+        final QueryChangeLog queryChangeLog = KafkaQueryChangeLogFactory.make(bootstrapServers, topic);
+
+        // Parse the Query ID from the command line parameters.
         final UUID queryId;
         try {
             queryId = UUID.fromString( params.queryId );
@@ -114,6 +132,19 @@ public class StreamResultsCommand implements RyaStreamsCommand {
             throw new ArgumentsException("Invalid Query ID " + params.queryId);
         }
 
+        // Fetch the SPARQL of the query whose results will be streamed.
+        final String sparql;
+        try(QueryRepository queryRepo = new InMemoryQueryRepository(queryChangeLog)) {
+            final Optional<StreamsQuery> sQuery = queryRepo.get(queryId);
+            if(!sQuery.isPresent()) {
+                throw new ExecutionException("Could not read the results for query with ID " + queryId +
+                        " because no such query exists.");
+            }
+            sparql = sQuery.get().getSparql();
+        } catch (final Exception e) {
+            throw new ExecutionException("Problem encountered while closing the QueryRepository.", e);
+        }
+
         // This command executes until the application is killed, so create a kill boolean.
         final AtomicBoolean finished = new AtomicBoolean(false);
         Runtime.getRuntime().addShutdownHook(new Thread() {
@@ -123,13 +154,24 @@ public class StreamResultsCommand implements RyaStreamsCommand {
             }
         });
 
-        // Execute the command.
-        final GetQueryResultStream getQueryResultStream = new KafkaGetQueryResultStream(params.kafkaIP, params.kafkaPort);
+        // Build the interactor based on the type of result the query produces.
+        final GetQueryResultStream<?> getQueryResultStream;
+        try {
+            final TupleExpr tupleExpr = new SPARQLParser().parseQuery(sparql, null).getTupleExpr();
+            if(tupleExpr instanceof Reduced) {
+                getQueryResultStream = new KafkaGetQueryResultStream<>(params.kafkaIP, params.kafkaPort, VisibilityStatementDeserializer.class);
+            } else {
+                getQueryResultStream = new KafkaGetQueryResultStream<>(params.kafkaIP, params.kafkaPort, VisibilityBindingSetDeserializer.class);
+            }
+        } catch (final MalformedQueryException e) {
+            throw new ExecutionException("Could not parse the SPARQL for the query: " + sparql, e);
+        }
 
-        try (final QueryResultStream stream = getQueryResultStream.fromStart(queryId)) {
+        // Iterate through the results and print them to the console until the program or the stream ends.
+        try (final QueryResultStream<?> stream = getQueryResultStream.fromStart(queryId)) {
             while(!finished.get()) {
-                for(final VisibilityBindingSet visBs : stream.poll(1000)) {
-                    System.out.println(visBs);
+                for(final Object result : stream.poll(1000)) {
+                    System.out.println(result);
                 }
             }
         } catch (final Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/pom.xml b/extras/rya.streams/kafka/pom.xml
index 2d33f32..8926870 100644
--- a/extras/rya.streams/kafka/pom.xml
+++ b/extras/rya.streams/kafka/pom.xml
@@ -41,7 +41,7 @@ under the License.
                     <dependency>
                         <groupId>org.apache.rya</groupId>
                         <artifactId>rya.pcj.functions.geo</artifactId>
-                        <version>3.2.12-incubating-SNAPSHOT</version>
+                        <version>${project.version}</version>
                     </dependency>
                     <dependency>
                         <groupId>org.apache.rya</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/entity/KafkaQueryResultStream.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/entity/KafkaQueryResultStream.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/entity/KafkaQueryResultStream.java
index 360aaa2..02a3812 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/entity/KafkaQueryResultStream.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/entity/KafkaQueryResultStream.java
@@ -26,7 +26,6 @@ import java.util.UUID;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.api.entity.QueryResultStream;
 import org.apache.rya.streams.api.exception.RyaStreamsException;
 
@@ -37,11 +36,13 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * A Kafka implementation of {@link QueryResultStream}. It delegates the {@link #poll(long)} method to
  * a {@link Consumer}. As a result, the starting point of this stream is whatever position the consumer
  * starts at within the Kafka topic.
+ *
+ * @param <V> - The type of the consumed records' value.
  */
 @DefaultAnnotation(NonNull.class)
-public class KafkaQueryResultStream extends QueryResultStream {
+public class KafkaQueryResultStream<V> extends QueryResultStream<V> {
 
-    private final Consumer<?, VisibilityBindingSet> consumer;
+    private final Consumer<?, V> consumer;
 
     /**
      * Constructs an instance of {@link KafkaQueryResultStream}.
@@ -49,13 +50,13 @@ public class KafkaQueryResultStream extends QueryResultStream {
      * @param queryId - The query the results are for. (not null)
      * @param consumer - The consumer that will be polled by this class. (not null)
      */
-    public KafkaQueryResultStream(final UUID queryId, final Consumer<?, VisibilityBindingSet> consumer) {
+    public KafkaQueryResultStream(final UUID queryId, final Consumer<?, V> consumer) {
         super(queryId);
         this.consumer = requireNonNull(consumer);
     }
 
     @Override
-    public Iterable<VisibilityBindingSet> poll(final long timeoutMs) throws RyaStreamsException {
+    public Iterable<V> poll(final long timeoutMs) throws RyaStreamsException {
         return new RecordEntryIterable<>( consumer.poll(timeoutMs) );
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStream.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStream.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStream.java
index b3c3fea..529b493 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStream.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStream.java
@@ -26,40 +26,47 @@ import java.util.UUID;
 
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.api.entity.QueryResultStream;
 import org.apache.rya.streams.api.exception.RyaStreamsException;
 import org.apache.rya.streams.api.interactor.GetQueryResultStream;
 import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.entity.KafkaQueryResultStream;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
 
 /**
  * A Kafka topic implementation of {@link GetQueryResultStream}.
+ *
+ * @param <T> - The type of results that are in the result stream.
  */
 @DefaultAnnotation(NonNull.class)
-public class KafkaGetQueryResultStream implements GetQueryResultStream {
+public class KafkaGetQueryResultStream<T> implements GetQueryResultStream<T> {
 
     private final String bootstrapServers;
+    private final Class<? extends Deserializer<T>> deserializerClass;
 
     /**
      * Constructs an instance of {@link KafkaGetQueryResultStream}.
      *
      * @param kafkaHostname - The hostname of the Kafka Broker to connect to. (not null)
      * @param kafkaPort - The port of the Kafka Broker to connect to. (not null)
+     * @param deserializerClass - The value deserializer to use when reading from the Kafka topic. (not null)
      */
-    public KafkaGetQueryResultStream(final String kafkaHostname, final String kafkaPort) {
+    public KafkaGetQueryResultStream(
+            final String kafkaHostname,
+            final String kafkaPort,
+            final Class<? extends Deserializer<T>> deserializerClass) {
         requireNonNull(kafkaHostname);
         requireNonNull(kafkaPort);
         bootstrapServers = kafkaHostname + ":" + kafkaPort;
+        this.deserializerClass = requireNonNull(deserializerClass);
     }
 
     @Override
-    public QueryResultStream fromStart(final UUID queryId) throws RyaStreamsException {
+    public QueryResultStream<T> fromStart(final UUID queryId) throws RyaStreamsException {
         requireNonNull(queryId);
 
         // Always start at the earliest point within the topic.
@@ -67,21 +74,21 @@ public class KafkaGetQueryResultStream implements GetQueryResultStream {
     }
 
     @Override
-    public QueryResultStream fromNow(final UUID queryId) throws RyaStreamsException {
+    public QueryResultStream<T> fromNow(final UUID queryId) throws RyaStreamsException {
         requireNonNull(queryId);
 
         // Always start at the latest point within the topic.
         return makeStream(queryId, "latest");
     }
 
-    private QueryResultStream makeStream(final UUID queryId, final String autoOffsetResetConfig) {
+    private QueryResultStream<T> makeStream(final UUID queryId, final String autoOffsetResetConfig) {
         // Configure which instance of Kafka to connect to.
         final Properties props = new Properties();
         props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
 
         // Nothing meaningful is in the key and the value is a VisibilityBindingSet.
         props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
-        props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VisibilityBindingSetDeserializer.class);
+        props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserializerClass);
 
         // Use a UUID for the Group Id so that we never register as part of the same group as another consumer.
         props.put(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
@@ -96,13 +103,13 @@ public class KafkaGetQueryResultStream implements GetQueryResultStream {
 
         // We are not closing the consumer here because the returned QueryResultStream is responsible for closing the
         // underlying resources required to process it.
-        final KafkaConsumer<Object, VisibilityBindingSet> consumer = new KafkaConsumer<>(props);
+        final KafkaConsumer<String, T> consumer = new KafkaConsumer<>(props);
 
         // Register the consumer for the query's results.
         final String resultTopic = KafkaTopics.queryResultsTopic(queryId);
         consumer.subscribe(Arrays.asList(resultTopic));
 
         // Return the result stream.
-        return new KafkaQueryResultStream(queryId, consumer);
+        return new KafkaQueryResultStream<>(queryId, consumer);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bd36443d/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
index c740ba2..8882753 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
@@ -29,10 +29,14 @@ import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.api.entity.QueryResultStream;
 import org.apache.rya.streams.api.interactor.GetQueryResultStream;
 import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.Rule;
@@ -54,23 +58,23 @@ public class KafkaGetQueryResultStreamIT {
      * the target number of results, or it hits the target number of results.
      *
      * @param pollMs - How long each poll could take.
-     * @param pollIterations - The maximum nubmer of polls that will be attempted.
+     * @param pollIterations - The maximum number of polls that will be attempted.
      * @param targetSize - The number of results to read before stopping.
      * @param stream - The stream that will be polled.
      * @return The results that were read from the stream.
      * @throws Exception If the poll failed.
      */
-    private List<VisibilityBindingSet> pollForResults(
+    private <T> List<T> pollForResults(
             final int pollMs,
             final int pollIterations,
             final int targetSize,
-            final QueryResultStream stream)  throws Exception{
-        final List<VisibilityBindingSet> read = new ArrayList<>();
+            final QueryResultStream<T> stream)  throws Exception{
+        final List<T> read = new ArrayList<>();
 
         int i = 0;
         while(read.size() < targetSize && i < pollIterations) {
-            for(final VisibilityBindingSet visBs : stream.poll(pollMs)) {
-                read.add( visBs );
+            for(final T result : stream.poll(pollMs)) {
+                read.add( result );
             }
             i++;
         }
@@ -109,7 +113,8 @@ public class KafkaGetQueryResultStreamIT {
         }
 
         // Use the interactor that is being tested to read all of the visibility binding sets.
-        final GetQueryResultStream interactor = new KafkaGetQueryResultStream(kafka.getKafkaHostname(), kafka.getKafkaPort());
+        final GetQueryResultStream<VisibilityBindingSet> interactor =
+                new KafkaGetQueryResultStream<>(kafka.getKafkaHostname(), kafka.getKafkaPort(), VisibilityBindingSetDeserializer.class);
         final List<VisibilityBindingSet> read = pollForResults(500, 3, 3, interactor.fromStart(queryId));
 
         // Show the fetched binding sets match the original, as well as their order.
@@ -133,8 +138,9 @@ public class KafkaGetQueryResultStreamIT {
             producer.flush();
 
             // Use the interactor that is being tested to read all of the visibility binding sets that appear after this point.
-            final GetQueryResultStream interactor = new KafkaGetQueryResultStream(kafka.getKafkaHostname(), kafka.getKafkaPort());
-            try(QueryResultStream results = interactor.fromNow(queryId)) {
+            final GetQueryResultStream<VisibilityBindingSet> interactor =
+                    new KafkaGetQueryResultStream<>(kafka.getKafkaHostname(), kafka.getKafkaPort(), VisibilityBindingSetDeserializer.class);
+            try(QueryResultStream<VisibilityBindingSet> results = interactor.fromNow(queryId)) {
                 // Read results from the stream.
                 List<VisibilityBindingSet> read = new ArrayList<>();
                 for(final VisibilityBindingSet visBs : results.poll(500)) {
@@ -175,11 +181,42 @@ public class KafkaGetQueryResultStreamIT {
         final UUID queryId = UUID.randomUUID();
 
         // Use the interactor that is being tested to create a result stream and immediately close it.
-        final GetQueryResultStream interactor = new KafkaGetQueryResultStream(kafka.getKafkaHostname(), kafka.getKafkaPort());
-        final QueryResultStream results = interactor.fromStart(queryId);
+        final GetQueryResultStream<VisibilityBindingSet> interactor =
+                new KafkaGetQueryResultStream<>(kafka.getKafkaHostname(), kafka.getKafkaPort(), VisibilityBindingSetDeserializer.class);
+        final QueryResultStream<VisibilityBindingSet> results = interactor.fromStart(queryId);
         results.close();
 
         // Try to poll the closed stream.
         results.poll(1);
     }
+
+    @Test
+    public void fromStart_visibilityStatements() throws Exception {
+        // Create an ID for the query.
+        final UUID queryId = UUID.randomUUID();
+
+        // Create some statements that will be written to the result topic.
+        final List<VisibilityStatement> original = new ArrayList<>();
+        final ValueFactory vf = new ValueFactoryImpl();
+        original.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+        original.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(63)), "b") );
+        original.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral("urn:34")), "") );
+
+        // Write the entries to the query result topic in Kafka.
+        try(final Producer<?, VisibilityStatement> producer =
+                KafkaTestUtil.makeProducer(kafka, StringSerializer.class, VisibilityStatementSerializer.class)) {
+            final String resultTopic = KafkaTopics.queryResultsTopic(queryId);
+            for(final VisibilityStatement visStmt : original) {
+                producer.send(new ProducerRecord<>(resultTopic, visStmt));
+            }
+        }
+
+        // Use the interactor that is being tested to read all of the visibility binding sets.
+        final GetQueryResultStream<VisibilityStatement> interactor =
+                new KafkaGetQueryResultStream<>(kafka.getKafkaHostname(), kafka.getKafkaPort(), VisibilityStatementDeserializer.class);
+        final List<VisibilityStatement> read = pollForResults(500, 3, 3, interactor.fromStart(queryId));
+
+        // Show the fetched binding sets match the original, as well as their order.
+        assertEquals(original, read);
+    }
 }
\ No newline at end of file


[45/50] [abbrv] incubator-rya git commit: RYA-377 add temporal equals function

Posted by ca...@apache.org.
RYA-377 add temporal equals function

Also added core functionality for adding other temporal
functions.

Updated geo tests to remove temporal stuff.


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

Branch: refs/heads/master
Commit: bc92591185ee8cc4598daf5f30bf4cf5235c23bb
Parents: bd36443
Author: Andrew Smith <sm...@gmail.com>
Authored: Thu Nov 30 13:11:49 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../api/function/temporal/EqualsTemporal.java   |  46 +++++++
 .../temporal/TemporalRelationFunction.java      |  65 ++++++++++
 ...f.query.algebra.evaluation.function.Function |  17 +++
 .../temporal/TemporalFunctionsTest.java         |  75 +++++++++++
 .../kafka/processors/filter/GeoFilterIT.java    |   6 +-
 .../processors/filter/TemporalFilterIT.java     | 127 +++++++++++++++++++
 6 files changed, 332 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bc925911/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
new file mode 100644
index 0000000..c8a6041
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rya.api.function.temporal;
+
+import java.time.ZonedDateTime;
+import java.util.Objects;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Filter function in a SPARQL query used to filter equality over time.
+ */
+@DefaultAnnotation(NonNull.class)
+public class EqualsTemporal extends TemporalRelationFunction {
+    private static final String URI = BASE_URI + "equals";
+
+    @Override
+    public String getURI() {
+        return URI;
+    }
+
+    @Override
+    protected boolean relation(final ZonedDateTime d1, final ZonedDateTime d2) {
+        Objects.requireNonNull(d1);
+        Objects.requireNonNull(d2);
+        return d1.isEqual(d2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bc925911/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalRelationFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalRelationFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalRelationFunction.java
new file mode 100644
index 0000000..02710d9
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalRelationFunction.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.temporal;
+
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeParseException;
+
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.query.algebra.evaluation.ValueExprEvaluationException;
+import org.openrdf.query.algebra.evaluation.function.Function;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Function for comparing 2 {@link ZonedDateTime} objects in a SPARQL filter.
+ */
+@DefaultAnnotation(NonNull.class)
+abstract class TemporalRelationFunction implements Function {
+    public static final String BASE_URI = "http://rya.apache.org/ns/temporal/";
+
+    @Override
+    public Value evaluate(final ValueFactory valueFactory, final Value... args) throws ValueExprEvaluationException {
+        if (args.length != 2) {
+            throw new ValueExprEvaluationException(getURI() + " requires exactly 2 arguments, got " + args.length);
+        }
+
+        try {
+            final ZonedDateTime date1 = ZonedDateTime.parse(args[0].stringValue());
+            final ZonedDateTime date2 = ZonedDateTime.parse(args[1].stringValue());
+            final boolean result = relation(date1, date2);
+
+            return valueFactory.createLiteral(result);
+        } catch (final DateTimeParseException e) {
+            throw new ValueExprEvaluationException("Date/Times provided must be of the ISO-8601 format. Example: 2007-04-05T14:30Z");
+        }
+    }
+
+    /**
+     * The comparison function to perform between 2 {@link ZonedDateTime}
+     * objects.
+     *
+     * @param d1 first {@link ZonedDateTime} to compare. (not null)
+     * @param d2 second {@link ZonedDateTime} to compare. (not null)
+     * @return The result of the comparison between {@link ZonedDateTime}s.
+     */
+    protected abstract boolean relation(ZonedDateTime d1, ZonedDateTime d2);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bc925911/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function b/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
new file mode 100644
index 0000000..475b9dd
--- /dev/null
+++ b/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
@@ -0,0 +1,17 @@
+# 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.
+org.apache.rya.api.function.temporal.EqualsTemporal
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bc925911/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/TemporalFunctionsTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/TemporalFunctionsTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/TemporalFunctionsTest.java
new file mode 100644
index 0000000..e0dabe1
--- /dev/null
+++ b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/TemporalFunctionsTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.temporal;
+
+import static org.junit.Assert.assertEquals;
+
+import java.time.ZonedDateTime;
+
+import org.junit.Test;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+public class TemporalFunctionsTest {
+    private static final ZonedDateTime TIME = ZonedDateTime.parse("2015-12-30T12:00:00Z");
+    private static final ZonedDateTime TIME_10 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
+    private static final ZonedDateTime TIME_20 = ZonedDateTime.parse("2015-12-30T12:00:20Z");
+
+    final ValueFactory VF = ValueFactoryImpl.getInstance();
+
+    @Test
+    public void testEquals_equal() throws Exception {
+        final EqualsTemporal function = new EqualsTemporal();
+
+        // 2 times equal
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(true), rez);
+    }
+
+    @Test
+    public void testEquals_before() throws Exception {
+        final EqualsTemporal function = new EqualsTemporal();
+
+        // first time is before
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME_10.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+
+    @Test
+    public void testEquals_after() throws Exception {
+        final EqualsTemporal function = new EqualsTemporal();
+
+        // first time is after
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME_20.toString());
+        args[1] = VF.createLiteral(TIME_10.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bc925911/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
index b9d81ef..642ecbc 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
@@ -57,7 +57,7 @@ import com.vividsolutions.jts.geom.GeometryFactory;
 import com.vividsolutions.jts.io.WKTWriter;
 
 /**
- * Integration tests the methods of {@link FilterProcessor}.
+ * Integration tests the geo methods of {@link FilterProcessor}.
  */
 public class GeoFilterIT {
     private static final String GEO = "http://www.opengis.net/def/function/geosparql/";
@@ -92,9 +92,7 @@ public class GeoFilterIT {
 
         // Get the RDF model objects that will be used to build the query.
         final String sparql =
-                "PREFIX time: <http://www.w3.org/2006/time#> \n"
-                        + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
-                        + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
                         + "PREFIX geof: <" + GEO + ">\n"
                         + "SELECT * \n"
                         + "WHERE { \n"

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/bc925911/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
new file mode 100644
index 0000000..2bc98ca
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.filter;
+
+import static org.junit.Assert.assertEquals;
+
+import java.time.ZonedDateTime;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.evaluation.function.Function;
+import org.openrdf.query.algebra.evaluation.function.FunctionRegistry;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Integration tests the temporal methods of {@link FilterProcessor}.
+ */
+public class TemporalFilterIT {
+    private static final ValueFactory vf = new ValueFactoryImpl();
+    private static final String TEMPORAL = "http://rya.apache.org/ns/temporal";
+    private static final ZonedDateTime time1 = ZonedDateTime.parse("2015-12-30T12:00:00Z");
+    private static final ZonedDateTime time2 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(false);
+
+    @Test
+    public void temporalFunctionsRegistered() {
+        int count = 0;
+        final Collection<Function> funcs = FunctionRegistry.getInstance().getAll();
+        for (final Function fun : funcs) {
+            if (fun.getURI().startsWith(TEMPORAL)) {
+                count++;
+            }
+        }
+
+        // There are 1 temporal functions registered, ensure that there are 1.
+        assertEquals(1, count);
+    }
+
+    @Test
+    public void showProcessorWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX time: <http://www.w3.org/2006/time/> \n"
+                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:time> time:atTime ?date .\n"
+                        + " FILTER(tempf:equals(?date, \"" + time1.toString() + "\")) "
+                        + "}";
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("date", vf.createLiteral(time1.toString()));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    private List<VisibilityStatement> getStatements() throws Exception {
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(statement(time1), "a"));
+        statements.add(new VisibilityStatement(statement(time2), "a"));
+        return statements;
+    }
+
+    private static Statement statement(final ZonedDateTime time) {
+        final Resource subject = vf.createURI("urn:time");
+        final URI predicate = vf.createURI("http://www.w3.org/2006/time/atTime");
+        final Value object = vf.createLiteral(time.toString());
+        return new StatementImpl(subject, predicate, object);
+    }
+}
\ No newline at end of file


[21/50] [abbrv] incubator-rya git commit: RYA-377 Added Kafka Streams to parent pom and rya.streams kafka pom

Posted by ca...@apache.org.
RYA-377 Added Kafka Streams to parent pom and rya.streams kafka pom


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

Branch: refs/heads/master
Commit: 07fcb5f3dcc064b345925c14aabf83bdbb7a4d41
Parents: f3ac7df
Author: Andrew Smith <sm...@gmail.com>
Authored: Fri Nov 3 15:28:28 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 extras/rya.streams/kafka/pom.xml | 6 +++++-
 pom.xml                          | 5 +++++
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/07fcb5f3/extras/rya.streams/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/pom.xml b/extras/rya.streams/kafka/pom.xml
index 2d70e2c..d5fffe0 100644
--- a/extras/rya.streams/kafka/pom.xml
+++ b/extras/rya.streams/kafka/pom.xml
@@ -53,7 +53,11 @@ under the License.
             <groupId>org.apache.kafka</groupId>
             <artifactId>kafka-clients</artifactId>
         </dependency>
-        
+       <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka-streams</artifactId>
+        </dependency>
+ 
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/07fcb5f3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 19f2d7a..e79ef75 100644
--- a/pom.xml
+++ b/pom.xml
@@ -868,6 +868,11 @@ under the License.
             </dependency>
             <dependency>
                 <groupId>org.apache.kafka</groupId>
+                <artifactId>kafka-streams</artifactId>
+                <version>${kafka.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.kafka</groupId>
                 <artifactId>kafka_2.11</artifactId>
                 <version>${kafka.version}</version>
                 <exclusions>


[14/50] [abbrv] incubator-rya git commit: RYA-377 Abstract Join into api.function

Posted by ca...@apache.org.
RYA-377 Abstract Join into api.function

Moved Join interfaces out of fluo
Refactored fluo to use the new interfaces
Copied visibility flattening code from accumulo


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

Branch: refs/heads/master
Commit: 0ad2c511b107cbdba6445693e72fb6f7c700b431
Parents: 516e890
Author: Andrew Smith <sm...@gmail.com>
Authored: Tue Nov 7 12:56:39 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 common/pom.xml                                  |   1 +
 common/rya.api.function/pom.xml                 |  56 ++
 .../rya/api/function/join/IterativeJoin.java    |  49 ++
 .../api/function/join/LazyJoiningIterator.java  | 110 ++++
 .../rya/api/function/join/LeftOuterJoin.java    |  71 +++
 .../rya/api/function/join/NaturalJoin.java      |  62 +++
 .../api/function/join/IterativeJoinTest.java    |  85 +++
 .../api/function/join/LeftOuterJoinTest.java    | 179 ++++++
 .../rya/api/function/join/NaturalJoinTest.java  | 169 ++++++
 common/rya.api.model/pom.xml                    |   5 +-
 .../api/model/visibility/ArrayByteSequence.java | 143 +++++
 .../api/model/visibility/Authorizations.java    |  77 +++
 .../model/visibility/BadArgumentException.java  |  42 ++
 .../rya/api/model/visibility/ByteSequence.java  | 114 ++++
 .../api/model/visibility/ColumnVisibility.java  | 551 +++++++++++++++++++
 .../model/visibility/FastByteComparison.java    | 240 ++++++++
 .../model/visibility/VisibilitySimplifier.java  |  89 +++
 .../model/visibility/WritableComparator.java    |  53 ++
 .../visibility/VisibilitySimplifierTest.java    |  91 +++
 extras/rya.pcj.fluo/pcj.fluo.app/pom.xml        |   5 +
 .../pcj/fluo/app/JoinResultUpdater.java         | 244 ++------
 .../app/batch/JoinBatchBindingSetUpdater.java   |  44 +-
 .../fluo/app/batch/JoinBatchInformation.java    |  34 +-
 .../JoinBatchInformationTypeAdapter.java        |  42 +-
 .../pcj/fluo/app/export/ExporterManager.java    |  66 +--
 .../pcj/fluo/app/IterativeJoinTest.java         |  88 ---
 .../pcj/fluo/app/LeftOuterJoinTest.java         | 181 ------
 .../indexing/pcj/fluo/app/NaturalJoinTest.java  | 171 ------
 .../BatchInformationSerializerTest.java         |   2 +-
 .../indexing/pcj/fluo/integration/BatchIT.java  |   2 +-
 pom.xml                                         |   5 +
 31 files changed, 2327 insertions(+), 744 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 8106cf5..3767caa 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -35,6 +35,7 @@ under the License.
     <modules>
         <module>rya.api</module>
         <module>rya.api.model</module>
+        <module>rya.api.function</module>
         <module>rya.provenance</module>
     </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.function/pom.xml
----------------------------------------------------------------------
diff --git a/common/rya.api.function/pom.xml b/common/rya.api.function/pom.xml
new file mode 100644
index 0000000..f05dd6f
--- /dev/null
+++ b/common/rya.api.function/pom.xml
@@ -0,0 +1,56 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+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>
+    <parent>
+        <groupId>org.apache.rya</groupId>
+        <artifactId>rya.common</artifactId>
+        <version>3.2.12-incubating-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>rya.api.function</artifactId>
+    <name>Apache Rya Common API - Functions</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.api.model</artifactId>
+        </dependency>
+
+        <!-- Third Party Dependencies -->
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>com.github.stephenc.findbugs</groupId>
+            <artifactId>findbugs-annotations</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/IterativeJoin.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/IterativeJoin.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/IterativeJoin.java
new file mode 100644
index 0000000..d667fc9
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/IterativeJoin.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.join;
+
+import java.util.Iterator;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+
+/**
+ * Defines each of the cases that may generate new join results when iteratively computing a query's join node.
+ */
+public interface IterativeJoin {
+
+    /**
+     * Invoked when a new {@link VisibilityBindingSet} is emitted from the left child
+     * node of the join.
+     *
+     * @param newLeftResult - A new VisibilityBindingSet that has been emitted from the left child node.
+     * @param rightResults - The right child node's binding sets that will be joined with the new left result. (not null)
+     * @return The new BindingSet results for the join.
+     */
+    public Iterator<VisibilityBindingSet> newLeftResult(VisibilityBindingSet newLeftResult, Iterator<VisibilityBindingSet> rightResults);
+
+    /**
+     * Invoked when a new {@link VisibilityBindingSet} is emitted from the right child
+     * node of the join.
+     *
+     * @param leftResults - The left child node's binding sets that will be joined with the new right result.
+     * @param newRightResult - A new BindingSet that has been emitted from the right child node.
+     * @return The new BindingSet results for the join.
+     */
+    public Iterator<VisibilityBindingSet> newRightResult(Iterator<VisibilityBindingSet> leftResults, VisibilityBindingSet newRightResult);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/LazyJoiningIterator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/LazyJoiningIterator.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/LazyJoiningIterator.java
new file mode 100644
index 0000000..b504a7e
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/LazyJoiningIterator.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.join;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Iterator;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.visibility.VisibilitySimplifier;
+import org.openrdf.query.Binding;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.impl.MapBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Joins a {@link BindingSet} (which is new to the left or right side of a join)
+ * to all binding sets on the other side that join with it.
+ * <p>
+ * This is done lazily so that you don't have to load all of the BindingSets
+ * into memory at once.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class LazyJoiningIterator implements Iterator<VisibilityBindingSet> {
+    private final Side newResultSide;
+    private final VisibilityBindingSet newResult;
+    private final Iterator<VisibilityBindingSet> joinedResults;
+
+    /**
+     * Constructs an instance of {@link LazyJoiningIterator}.
+     *
+     * @param newResultSide - Indicates which side of the join the
+     *        {@code newResult} arrived on. (not null)
+     * @param newResult - A binding set that will be joined with some other
+     *        binding sets. (not null)
+     * @param joinedResults - The binding sets that will be joined with
+     *        {@code newResult}. (not null)
+     */
+    public LazyJoiningIterator(final Side newResultSide, final VisibilityBindingSet newResult,
+            final Iterator<VisibilityBindingSet> joinedResults) {
+        this.newResultSide = requireNonNull(newResultSide);
+        this.newResult = requireNonNull(newResult);
+        this.joinedResults = requireNonNull(joinedResults);
+    }
+
+    @Override
+    public boolean hasNext() {
+        return joinedResults.hasNext();
+    }
+
+    @Override
+    public VisibilityBindingSet next() {
+        final MapBindingSet bs = new MapBindingSet();
+
+        for (final Binding binding : newResult) {
+            bs.addBinding(binding);
+        }
+
+        final VisibilityBindingSet joinResult = joinedResults.next();
+        for (final Binding binding : joinResult) {
+            bs.addBinding(binding);
+        }
+
+        // We want to make sure the visibilities are always written the same way,
+        // so figure out which are on the left side and which are on the right side.
+        final String leftVisi;
+        final String rightVisi;
+        if (newResultSide == Side.LEFT) {
+            leftVisi = newResult.getVisibility();
+            rightVisi = joinResult.getVisibility();
+        } else {
+            leftVisi = joinResult.getVisibility();
+            rightVisi = newResult.getVisibility();
+        }
+
+        final String visibility = VisibilitySimplifier.unionAndSimplify(leftVisi, rightVisi);
+
+        return new VisibilityBindingSet(bs, visibility);
+    }
+
+    @Override
+    public void remove() {
+        throw new UnsupportedOperationException("remove() is unsupported.");
+    }
+
+    /**
+     * The different sides a new binding set may appear on.
+     */
+    public static enum Side {
+        LEFT, RIGHT;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/LeftOuterJoin.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/LeftOuterJoin.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/LeftOuterJoin.java
new file mode 100644
index 0000000..79af26c
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/LeftOuterJoin.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.join;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.rya.api.function.join.LazyJoiningIterator.Side;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.query.BindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Implements an {@link IterativeJoin} that uses the Left Outer Join
+ * algorithm defined by Relational Algebra.
+ * <p>
+ * This is how you add optional information to a {@link BindingSet}. Left
+ * binding sets are emitted even if they do not join with anything on the right.
+ * However, right binding sets must be joined with a left binding set.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class LeftOuterJoin implements IterativeJoin {
+    @Override
+    public Iterator<VisibilityBindingSet> newLeftResult(final VisibilityBindingSet newLeftResult, final Iterator<VisibilityBindingSet> rightResults) {
+        requireNonNull(newLeftResult);
+        requireNonNull(rightResults);
+
+        // If the required portion does not join with any optional portions,
+        // then emit a BindingSet that matches the new left result.
+        if(!rightResults.hasNext()) {
+            final List<VisibilityBindingSet> leftResultList = new ArrayList<>();
+            leftResultList.add(newLeftResult);
+            return leftResultList.iterator();
+        }
+
+        // Otherwise, return an iterator that holds the new required result
+        // joined with the right results.
+        return new LazyJoiningIterator(Side.LEFT, newLeftResult, rightResults);
+    }
+
+    @Override
+    public Iterator<VisibilityBindingSet> newRightResult(final Iterator<VisibilityBindingSet> leftResults, final VisibilityBindingSet newRightResult) {
+        requireNonNull(leftResults);
+        requireNonNull(newRightResult);
+
+        // The right result is optional, so if it does not join with anything
+        // on the left, then do not emit anything.
+        return new LazyJoiningIterator(Side.RIGHT, newRightResult, leftResults);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/NaturalJoin.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/NaturalJoin.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/NaturalJoin.java
new file mode 100644
index 0000000..2db6ba4
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/join/NaturalJoin.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.join;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Iterator;
+
+import org.apache.rya.api.function.join.LazyJoiningIterator.Side;
+import org.apache.rya.api.model.VisibilityBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Implements an {@link IterativeJoin} that uses the Natural Join algorithm
+ * defined by Relational Algebra.
+ * <p>
+ * This is how you combine {@code BindnigSet}s that may have common Binding
+ * names. When two Binding Sets are joined, any bindings that appear in both
+ * binding sets are only included once.
+ */
+@DefaultAnnotation(NonNull.class)
+public class NaturalJoin implements IterativeJoin {
+    @Override
+    public Iterator<VisibilityBindingSet> newLeftResult(final VisibilityBindingSet newLeftResult,
+            final Iterator<VisibilityBindingSet> rightResults) {
+        requireNonNull(newLeftResult);
+        requireNonNull(rightResults);
+
+        // Both sides are required, so if there are no right results, then do
+        // not emit anything.
+        return new LazyJoiningIterator(Side.LEFT, newLeftResult, rightResults);
+    }
+
+    @Override
+    public Iterator<VisibilityBindingSet> newRightResult(final Iterator<VisibilityBindingSet> leftResults,
+            final VisibilityBindingSet newRightResult) {
+        requireNonNull(leftResults);
+        requireNonNull(newRightResult);
+
+        // Both sides are required, so if there are no left reuslts, then do not
+        // emit anything.
+        return new LazyJoiningIterator(Side.RIGHT, newRightResult, leftResults);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/IterativeJoinTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/IterativeJoinTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/IterativeJoinTest.java
new file mode 100644
index 0000000..5d357c3
--- /dev/null
+++ b/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/IterativeJoinTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.join;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Tests the methods of {@link IterativeJoin}.
+ */
+@RunWith(Parameterized.class)
+public class IterativeJoinTest {
+
+    @Parameters
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][] {
+            {new NaturalJoin()},
+            {new LeftOuterJoin()}
+           });
+    }
+
+    @Parameter
+    public IterativeJoin join;
+
+    /**
+     * This test ensures the same binding sets are created as the result of a
+     * {@link IterativeJoin} regardless of which side the notification is triggered on.
+     */
+    @Test
+    public void naturalJoin_sideDoesNotMatter() {
+        // Create the binding sets that will be joined.
+        final ValueFactory vf = new ValueFactoryImpl();
+
+        final MapBindingSet bs1 = new MapBindingSet();
+        bs1.addBinding("id", vf.createLiteral("some_uid"));
+        bs1.addBinding("name", vf.createLiteral("Alice"));
+        final VisibilityBindingSet vbs1 = new VisibilityBindingSet(bs1, "a");
+
+        final MapBindingSet bs2 = new MapBindingSet();
+        bs2.addBinding("id", vf.createLiteral("some_uid"));
+        bs2.addBinding("hair", vf.createLiteral("brown"));
+        final VisibilityBindingSet vbs2 = new VisibilityBindingSet(bs2, "b");
+
+        // new vbs1 shows up on the left, matches vbs2 on the right
+        final Iterator<VisibilityBindingSet> newLeftIt = join.newLeftResult(vbs1, Collections.singleton(vbs2).iterator());
+        final VisibilityBindingSet newLeftResult = newLeftIt.next();
+
+        // new vbs2 shows up on the right, matches vbs1 on the left
+        final Iterator<VisibilityBindingSet> newRightIt = join.newRightResult(Collections.singleton(vbs1).iterator(), vbs2);
+        final VisibilityBindingSet newRightResult = newRightIt.next();
+
+        // Ensure those two results are the same.
+        assertEquals(newLeftResult, newRightResult);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/LeftOuterJoinTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/LeftOuterJoinTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/LeftOuterJoinTest.java
new file mode 100644
index 0000000..7d17e22
--- /dev/null
+++ b/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/LeftOuterJoinTest.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.join;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+/**
+ * Tests the methods of {@link LeftOuterJoin}.
+ */
+public class LeftOuterJoinTest {
+
+    private final ValueFactory vf = new ValueFactoryImpl();
+
+    @Test
+    public void newLeftResult_noRightMatches() {
+        final IterativeJoin leftOuterJoin = new LeftOuterJoin();
+
+        // There is a new left result.
+        final MapBindingSet mapLeftResult = new MapBindingSet();
+        mapLeftResult.addBinding("name", vf.createLiteral("Bob"));
+        final VisibilityBindingSet newLeftResult = new VisibilityBindingSet(mapLeftResult);
+
+        // There are no right results that join with the left result.
+        final Iterator<VisibilityBindingSet> rightResults= new ArrayList<VisibilityBindingSet>().iterator();
+
+        // Therefore, the left result is a new join result.
+        final Iterator<VisibilityBindingSet> newJoinResultsIt = leftOuterJoin.newLeftResult(newLeftResult, rightResults);
+
+        final Set<BindingSet> newJoinResults = new HashSet<>();
+        while(newJoinResultsIt.hasNext()) {
+            newJoinResults.add( newJoinResultsIt.next() );
+        }
+
+        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet( newLeftResult );
+
+        assertEquals(expected, newJoinResults);
+    }
+
+    @Test
+    public void newLeftResult_joinsWithRightResults() {
+        final IterativeJoin leftOuterJoin = new LeftOuterJoin();
+
+        // There is a new left result.
+        final MapBindingSet mapLeftResult = new MapBindingSet();
+        mapLeftResult.addBinding("name", vf.createLiteral("Bob"));
+        mapLeftResult.addBinding("height", vf.createLiteral("5'9\""));
+        final VisibilityBindingSet newLeftResult = new VisibilityBindingSet(mapLeftResult);
+
+        // There are a few right results that join with the left result.
+        final MapBindingSet nameAge = new MapBindingSet();
+        nameAge.addBinding("name", vf.createLiteral("Bob"));
+        nameAge.addBinding("age", vf.createLiteral(56));
+        final VisibilityBindingSet visiAge = new VisibilityBindingSet(nameAge);
+
+        final MapBindingSet nameHair = new MapBindingSet();
+        nameHair.addBinding("name", vf.createLiteral("Bob"));
+        nameHair.addBinding("hairColor", vf.createLiteral("Brown"));
+        final VisibilityBindingSet visiHair = new VisibilityBindingSet(nameHair);
+
+        final Iterator<VisibilityBindingSet> rightResults = Lists.<VisibilityBindingSet>newArrayList(visiAge, visiHair).iterator();
+
+        // Therefore, there are a few new join results that mix the two together.
+        final Iterator<VisibilityBindingSet> newJoinResultsIt = leftOuterJoin.newLeftResult(newLeftResult, rightResults);
+
+        final Set<BindingSet> newJoinResults = new HashSet<>();
+        while(newJoinResultsIt.hasNext()) {
+            newJoinResults.add( newJoinResultsIt.next() );
+        }
+
+        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet();
+        final MapBindingSet nameHeightAge = new MapBindingSet();
+        nameHeightAge.addBinding("name", vf.createLiteral("Bob"));
+        nameHeightAge.addBinding("height", vf.createLiteral("5'9\""));
+        nameHeightAge.addBinding("age", vf.createLiteral(56));
+        expected.add(new VisibilityBindingSet(nameHeightAge));
+
+        final MapBindingSet nameHeightHair = new MapBindingSet();
+        nameHeightHair.addBinding("name", vf.createLiteral("Bob"));
+        nameHeightHair.addBinding("height", vf.createLiteral("5'9\""));
+        nameHeightHair.addBinding("hairColor", vf.createLiteral("Brown"));
+        expected.add(new VisibilityBindingSet(nameHeightHair));
+
+        assertEquals(expected, newJoinResults);
+    }
+
+    @Test
+    public void newRightResult_noLeftMatches() {
+        final IterativeJoin leftOuterJoin = new LeftOuterJoin();
+
+        // There are no left results.
+        final Iterator<VisibilityBindingSet> leftResults= new ArrayList<VisibilityBindingSet>().iterator();
+
+        // There is a new right result.
+        final MapBindingSet newRightResult = new MapBindingSet();
+        newRightResult.addBinding("name", vf.createLiteral("Bob"));
+
+        // Therefore, there are no new join results.
+        final Iterator<VisibilityBindingSet> newJoinResultsIt = leftOuterJoin.newRightResult(leftResults, new VisibilityBindingSet(newRightResult));
+        assertFalse( newJoinResultsIt.hasNext() );
+    }
+
+    @Test
+    public void newRightResult_joinsWithLeftResults() {
+        final IterativeJoin leftOuterJoin = new LeftOuterJoin();
+
+        // There are a few left results that join with the new right result.
+        final MapBindingSet nameAge = new MapBindingSet();
+        nameAge.addBinding("name", vf.createLiteral("Bob"));
+        nameAge.addBinding("age", vf.createLiteral(56));
+
+        final MapBindingSet nameHair = new MapBindingSet();
+        nameHair.addBinding("name", vf.createLiteral("Bob"));
+        nameHair.addBinding("hairColor", vf.createLiteral("Brown"));
+
+        final Iterator<VisibilityBindingSet> leftResults = Lists.<VisibilityBindingSet>newArrayList(
+                new VisibilityBindingSet(nameAge),
+                new VisibilityBindingSet(nameHair)).iterator();
+
+        // There is a new right result.
+        final MapBindingSet newRightResult = new MapBindingSet();
+        newRightResult.addBinding("name", vf.createLiteral("Bob"));
+        newRightResult.addBinding("height", vf.createLiteral("5'9\""));
+
+        // Therefore, there are a few new join results that mix the two together.
+        final Iterator<VisibilityBindingSet> newJoinResultsIt = leftOuterJoin.newRightResult(leftResults, new VisibilityBindingSet(newRightResult));
+
+        final Set<BindingSet> newJoinResults = new HashSet<>();
+        while(newJoinResultsIt.hasNext()) {
+            newJoinResults.add( newJoinResultsIt.next() );
+        }
+
+        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet();
+        final MapBindingSet nameHeightAge = new MapBindingSet();
+        nameHeightAge.addBinding("name", vf.createLiteral("Bob"));
+        nameHeightAge.addBinding("height", vf.createLiteral("5'9\""));
+        nameHeightAge.addBinding("age", vf.createLiteral(56));
+        expected.add(new VisibilityBindingSet(nameHeightAge));
+
+        final MapBindingSet nameHeightHair = new MapBindingSet();
+        nameHeightHair.addBinding("name", vf.createLiteral("Bob"));
+        nameHeightHair.addBinding("height", vf.createLiteral("5'9\""));
+        nameHeightHair.addBinding("hairColor", vf.createLiteral("Brown"));
+        expected.add(new VisibilityBindingSet(nameHeightHair));
+
+        assertEquals(expected, newJoinResults);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/NaturalJoinTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/NaturalJoinTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/NaturalJoinTest.java
new file mode 100644
index 0000000..dd3b7e6
--- /dev/null
+++ b/common/rya.api.function/src/main/test/org/apache/rya/api/function/join/NaturalJoinTest.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.join;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+/**
+ * Tests the methods of {@link NaturalJoin}.
+ */
+public class NaturalJoinTest {
+
+    private final ValueFactory vf = new ValueFactoryImpl();
+
+    @Test
+    public void newLeftResult_noRightMatches() {
+        final IterativeJoin naturalJoin = new NaturalJoin();
+
+        // There is a new left result.
+        final MapBindingSet newLeftResult = new MapBindingSet();
+        newLeftResult.addBinding("name", vf.createLiteral("Bob"));
+
+        // There are no right results that join with the left result.
+        final Iterator<VisibilityBindingSet> rightResults= new ArrayList<VisibilityBindingSet>().iterator();
+
+        // Therefore, the left result is a new join result.
+        final Iterator<VisibilityBindingSet> newJoinResultsIt = naturalJoin.newLeftResult(new VisibilityBindingSet(newLeftResult), rightResults);
+        assertFalse( newJoinResultsIt.hasNext() );
+    }
+
+    @Test
+    public void newLeftResult_joinsWithRightResults() {
+        final IterativeJoin naturalJoin = new NaturalJoin();
+
+        // There is a new left result.
+        final MapBindingSet newLeftResult = new MapBindingSet();
+        newLeftResult.addBinding("name", vf.createLiteral("Bob"));
+        newLeftResult.addBinding("height", vf.createLiteral("5'9\""));
+
+        // There are a few right results that join with the left result.
+        final MapBindingSet nameAge = new MapBindingSet();
+        nameAge.addBinding("name", vf.createLiteral("Bob"));
+        nameAge.addBinding("age", vf.createLiteral(56));
+
+        final MapBindingSet nameHair = new MapBindingSet();
+        nameHair.addBinding("name", vf.createLiteral("Bob"));
+        nameHair.addBinding("hairColor", vf.createLiteral("Brown"));
+
+        final Iterator<VisibilityBindingSet> rightResults = Lists.<VisibilityBindingSet>newArrayList(
+                new VisibilityBindingSet(nameAge),
+                new VisibilityBindingSet(nameHair)).iterator();
+
+        // Therefore, there are a few new join results that mix the two together.
+        final Iterator<VisibilityBindingSet> newJoinResultsIt = naturalJoin.newLeftResult(new VisibilityBindingSet(newLeftResult), rightResults);
+
+        final Set<BindingSet> newJoinResults = new HashSet<>();
+        while(newJoinResultsIt.hasNext()) {
+            newJoinResults.add( newJoinResultsIt.next() );
+        }
+
+        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet();
+        final MapBindingSet nameHeightAge = new MapBindingSet();
+        nameHeightAge.addBinding("name", vf.createLiteral("Bob"));
+        nameHeightAge.addBinding("height", vf.createLiteral("5'9\""));
+        nameHeightAge.addBinding("age", vf.createLiteral(56));
+        expected.add(new VisibilityBindingSet(nameHeightAge));
+
+        final MapBindingSet nameHeightHair = new MapBindingSet();
+        nameHeightHair.addBinding("name", vf.createLiteral("Bob"));
+        nameHeightHair.addBinding("height", vf.createLiteral("5'9\""));
+        nameHeightHair.addBinding("hairColor", vf.createLiteral("Brown"));
+        expected.add(new VisibilityBindingSet(nameHeightHair));
+
+        assertEquals(expected, newJoinResults);
+    }
+
+    @Test
+    public void newRightResult_noLeftMatches() {
+        final IterativeJoin naturalJoin = new NaturalJoin();
+
+        // There are no left results.
+        final Iterator<VisibilityBindingSet> leftResults= new ArrayList<VisibilityBindingSet>().iterator();
+
+        // There is a new right result.
+        final MapBindingSet newRightResult = new MapBindingSet();
+        newRightResult.addBinding("name", vf.createLiteral("Bob"));
+
+        // Therefore, there are no new join results.
+        final Iterator<VisibilityBindingSet> newJoinResultsIt = naturalJoin.newRightResult(leftResults, new VisibilityBindingSet(newRightResult));
+        assertFalse( newJoinResultsIt.hasNext() );
+    }
+
+    @Test
+    public void newRightResult_joinsWithLeftResults() {
+        final IterativeJoin naturalJoin = new NaturalJoin();
+
+        // There are a few left results that join with the new right result.
+        final MapBindingSet nameAge = new MapBindingSet();
+        nameAge.addBinding("name", vf.createLiteral("Bob"));
+        nameAge.addBinding("age", vf.createLiteral(56));
+
+        final MapBindingSet nameHair = new MapBindingSet();
+        nameHair.addBinding("name", vf.createLiteral("Bob"));
+        nameHair.addBinding("hairColor", vf.createLiteral("Brown"));
+
+        final Iterator<VisibilityBindingSet> leftResults = Lists.<VisibilityBindingSet>newArrayList(
+                new VisibilityBindingSet(nameAge),
+                new VisibilityBindingSet(nameHair)).iterator();
+
+        // There is a new right result.
+        final MapBindingSet newRightResult = new MapBindingSet();
+        newRightResult.addBinding("name", vf.createLiteral("Bob"));
+        newRightResult.addBinding("height", vf.createLiteral("5'9\""));
+
+        // Therefore, there are a few new join results that mix the two together.
+        final Iterator<VisibilityBindingSet> newJoinResultsIt = naturalJoin.newRightResult(leftResults, new VisibilityBindingSet(newRightResult));
+
+        final Set<BindingSet> newJoinResults = new HashSet<>();
+        while(newJoinResultsIt.hasNext()) {
+            newJoinResults.add( newJoinResultsIt.next() );
+        }
+
+        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet();
+        final MapBindingSet nameHeightAge = new MapBindingSet();
+        nameHeightAge.addBinding("name", vf.createLiteral("Bob"));
+        nameHeightAge.addBinding("height", vf.createLiteral("5'9\""));
+        nameHeightAge.addBinding("age", vf.createLiteral(56));
+        expected.add(new VisibilityBindingSet(nameHeightAge));
+
+        final MapBindingSet nameHeightHair = new MapBindingSet();
+        nameHeightHair.addBinding("name", vf.createLiteral("Bob"));
+        nameHeightHair.addBinding("height", vf.createLiteral("5'9\""));
+        nameHeightHair.addBinding("hairColor", vf.createLiteral("Brown"));
+        expected.add(new VisibilityBindingSet(nameHeightHair));
+
+        assertEquals(expected, newJoinResults);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/pom.xml
----------------------------------------------------------------------
diff --git a/common/rya.api.model/pom.xml b/common/rya.api.model/pom.xml
index ce77134..13893c6 100644
--- a/common/rya.api.model/pom.xml
+++ b/common/rya.api.model/pom.xml
@@ -39,7 +39,10 @@ under the License.
             <groupId>org.openrdf.sesame</groupId>
             <artifactId>sesame-query</artifactId>
         </dependency>
-
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
 
         <dependency>
             <groupId>com.github.stephenc.findbugs</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ArrayByteSequence.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ArrayByteSequence.java b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ArrayByteSequence.java
new file mode 100644
index 0000000..f0f5eb6
--- /dev/null
+++ b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ArrayByteSequence.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rya.api.model.visibility;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+/**
+ * XXX
+ * This class has been copied over because Rya has decided to use the Accumulo
+ * implementation of visibilities to control who is able to access what data
+ * within a Rya instance. Until we implement an Accumulo agnostic method for
+ * handling those visibility expressions, we have chosen to pull the Accumulo
+ * code into our API.
+ *
+ * Copied from accumulo's  org.apache.accumulo.core.data.ArrayByteSequence
+ *   <dependancy>
+ *     <groupId>org.apache.accumulo</groupId>
+ *     <artifactId>accumulo-core</artifactId>
+ *     <version>1.6.4</version>
+ *   </dependancy>
+ */
+public class ArrayByteSequence extends ByteSequence implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    protected byte data[];
+    protected int offset;
+    protected int length;
+
+    public ArrayByteSequence(final byte data[]) {
+        this.data = data;
+        offset = 0;
+        length = data.length;
+    }
+
+    public ArrayByteSequence(final byte data[], final int offset, final int length) {
+
+        if (offset < 0 || offset > data.length || length < 0 || offset + length > data.length) {
+            throw new IllegalArgumentException(" Bad offset and/or length data.length = " + data.length + " offset = "
+                    + offset + " length = " + length);
+        }
+
+        this.data = data;
+        this.offset = offset;
+        this.length = length;
+
+    }
+
+    public ArrayByteSequence(final String s) {
+        this(s.getBytes(UTF_8));
+    }
+
+    public ArrayByteSequence(final ByteBuffer buffer) {
+        length = buffer.remaining();
+
+        if (buffer.hasArray()) {
+            data = buffer.array();
+            offset = buffer.position();
+        } else {
+            data = new byte[length];
+            offset = 0;
+            buffer.get(data);
+        }
+    }
+
+    @Override
+    public byte byteAt(final int i) {
+
+        if (i < 0) {
+            throw new IllegalArgumentException("i < 0, " + i);
+        }
+
+        if (i >= length) {
+            throw new IllegalArgumentException("i >= length, " + i + " >= " + length);
+        }
+
+        return data[offset + i];
+    }
+
+    @Override
+    public byte[] getBackingArray() {
+        return data;
+    }
+
+    @Override
+    public boolean isBackedByArray() {
+        return true;
+    }
+
+    @Override
+    public int length() {
+        return length;
+    }
+
+    @Override
+    public int offset() {
+        return offset;
+    }
+
+    @Override
+    public ByteSequence subSequence(final int start, final int end) {
+
+        if (start > end || start < 0 || end > length) {
+            throw new IllegalArgumentException(
+                    "Bad start and/end start = " + start + " end=" + end + " offset=" + offset + " length=" + length);
+        }
+
+        return new ArrayByteSequence(data, offset + start, end - start);
+    }
+
+    @Override
+    public byte[] toArray() {
+        if (offset == 0 && length == data.length) {
+            return data;
+        }
+
+        final byte[] copy = new byte[length];
+        System.arraycopy(data, offset, copy, 0, length);
+        return copy;
+    }
+
+    @Override
+    public String toString() {
+        return new String(data, offset, length, UTF_8);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/Authorizations.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/Authorizations.java b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/Authorizations.java
new file mode 100644
index 0000000..70d4601
--- /dev/null
+++ b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/Authorizations.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rya.api.model.visibility;
+
+import java.io.Serializable;
+
+/**
+ * A collection of authorization strings.
+ * 
+ * XXX
+ * This class has been copied over because Rya has decided to use the Accumulo
+ * implementation of visibilities to control who is able to access what data
+ * within a Rya instance. Until we implement an Accumulo agnostic method for
+ * handling those visibility expressions, we have chosen to pull the Accumulo
+ * code into our API.
+ *
+ * Copied from accumulo's  org.apache.accumulo.core.security.ByteSequence
+ *   <dependancy>
+ *     <groupId>org.apache.accumulo</groupId>
+ *     <artifactId>accumulo-core</artifactId>
+ *     <version>1.6.4</version>
+ *   </dependancy>
+ */
+public class Authorizations implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private static final boolean[] validAuthChars = new boolean[256];
+
+    /**
+     * A special header string used when serializing instances of this class.
+     *
+     * @see #serialize()
+     */
+    public static final String HEADER = "!AUTH1:";
+
+    static {
+        for (int i = 0; i < 256; i++) {
+            validAuthChars[i] = false;
+        }
+
+        for (int i = 'a'; i <= 'z'; i++) {
+            validAuthChars[i] = true;
+        }
+
+        for (int i = 'A'; i <= 'Z'; i++) {
+            validAuthChars[i] = true;
+        }
+
+        for (int i = '0'; i <= '9'; i++) {
+            validAuthChars[i] = true;
+        }
+
+        validAuthChars['_'] = true;
+        validAuthChars['-'] = true;
+        validAuthChars[':'] = true;
+        validAuthChars['.'] = true;
+        validAuthChars['/'] = true;
+    }
+
+    static final boolean isValidAuthChar(final byte b) {
+        return validAuthChars[0xff & b];
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/BadArgumentException.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/BadArgumentException.java b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/BadArgumentException.java
new file mode 100644
index 0000000..de40b5e
--- /dev/null
+++ b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/BadArgumentException.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rya.api.model.visibility;
+
+import java.util.regex.PatternSyntaxException;
+
+/**
+ * XXX
+ * This class has been copied over because Rya has decided to use the Accumulo
+ * implementation of visibilities to control who is able to access what data
+ * within a Rya instance. Until we implement an Accumulo agnostic method for
+ * handling those visibility expressions, we have chosen to pull the Accumulo
+ * code into our API.
+ *
+ * Copied from accumulo's org.apache.accumulo.core.util.BadArgumentException
+ *   <dependancy>
+ *     <groupId>org.apache.accumulo</groupId>
+ *     <artifactId>accumulo-core</artifactId>
+ *     <version>1.6.4</version>
+ *   </dependancy>
+ */
+public final class BadArgumentException extends PatternSyntaxException {
+    private static final long serialVersionUID = 1L;
+
+    public BadArgumentException(final String desc, final String badarg, final int index) {
+        super(desc, badarg, index);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ByteSequence.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ByteSequence.java b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ByteSequence.java
new file mode 100644
index 0000000..ad1aa54
--- /dev/null
+++ b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ByteSequence.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rya.api.model.visibility;
+
+/**
+ * XXX
+ * This class has been copied over because Rya has decided to use the Accumulo
+ * implementation of visibilities to control who is able to access what data
+ * within a Rya instance. Until we implement an Accumulo agnostic method for
+ * handling those visibility expressions, we have chosen to pull the Accumulo
+ * code into our API.
+ *
+ * Copied from accumulo's org.apache.accumulo.core.data.ByteSequence
+ *   <dependancy>
+ *     <groupId>org.apache.accumulo</groupId>
+ *     <artifactId>accumulo-core</artifactId>
+ *     <version>1.6.4</version>
+ *   </dependancy>
+ */
+public abstract class ByteSequence implements Comparable<ByteSequence> {
+
+    public abstract byte byteAt(int i);
+
+    public abstract int length();
+
+    public abstract ByteSequence subSequence(int start, int end);
+
+    // may copy data
+    public abstract byte[] toArray();
+
+    public abstract boolean isBackedByArray();
+
+    public abstract byte[] getBackingArray();
+
+    public abstract int offset();
+
+    public static int compareBytes(final ByteSequence bs1, final ByteSequence bs2) {
+
+        final int minLen = Math.min(bs1.length(), bs2.length());
+
+        for (int i = 0; i < minLen; i++) {
+            final int a = bs1.byteAt(i) & 0xff;
+            final int b = bs2.byteAt(i) & 0xff;
+
+            if (a != b) {
+                return a - b;
+            }
+        }
+
+        return bs1.length() - bs2.length();
+    }
+
+    @Override
+    public int compareTo(final ByteSequence obs) {
+        if (isBackedByArray() && obs.isBackedByArray()) {
+            return WritableComparator.compareBytes(getBackingArray(), offset(), length(), obs.getBackingArray(),
+                    obs.offset(), obs.length());
+        }
+
+        return compareBytes(this, obs);
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (o instanceof ByteSequence) {
+            final ByteSequence obs = (ByteSequence) o;
+
+            if (this == o) {
+                return true;
+            }
+
+            if (length() != obs.length()) {
+                return false;
+            }
+
+            return compareTo(obs) == 0;
+        }
+
+        return false;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int hash = 1;
+        if (isBackedByArray()) {
+            final byte[] data = getBackingArray();
+            final int end = offset() + length();
+            for (int i = offset(); i < end; i++) {
+                hash = 31 * hash + data[i];
+            }
+        } else {
+            for (int i = 0; i < length(); i++) {
+                hash = 31 * hash + byteAt(i);
+            }
+        }
+        return hash;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ColumnVisibility.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ColumnVisibility.java b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ColumnVisibility.java
new file mode 100644
index 0000000..052de45
--- /dev/null
+++ b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/ColumnVisibility.java
@@ -0,0 +1,551 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rya.api.model.visibility;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.TreeSet;
+
+/**
+ * Validate the column visibility is a valid expression and set the visibility
+ * for a Mutation. See {@link ColumnVisibility#ColumnVisibility(byte[])} for the
+ * definition of an expression.
+ *
+ * <P>
+ * The expression is a sequence of characters from the set [A-Za-z0-9_-.] along
+ * with the binary operators "&amp;" and "|" indicating that both operands are
+ * necessary, or the either is necessary. The following are valid expressions
+ * for visibility:
+ *
+ * <pre>
+ * A
+ * A|B
+ * (A|B)&amp;(C|D)
+ * orange|(red&amp;yellow)
+ * </pre>
+ *
+ * <P>
+ * The following are not valid expressions for visibility:
+ *
+ * <pre>
+ * A|B&amp;C
+ * A=B
+ * A|B|
+ * A&amp;|B
+ * ()
+ * )
+ * dog|!cat
+ * </pre>
+ *
+ * <P>
+ * In addition to the base set of visibilities, any character can be used in the
+ * expression if it is quoted. If the quoted term contains '&quot;' or '\', then
+ * escape the character with '\'. The {@link #quote(String)} method can be used
+ * to properly quote and escape terms automatically. The following is an example
+ * of a quoted term:
+ *
+ * <pre>
+ * &quot;A#C&quot;<span />&amp;<span />B
+ * </pre>
+ * 
+ * XXX      
+ * This class has been copied over because Rya has decided to use the Accumulo
+ * implementation of visibilities to control who is able to access what data
+ * within a Rya instance. Until we implement an Accumulo agnostic method for
+ * handling those visibility expressions, we have chosen to pull the Accumulo
+ * code into our API.
+ *
+ * Copied from accumulo's org.apache.accumulo.core.security.ColumnVisibility
+ *   <dependancy>
+ *     <groupId>org.apache.accumulo</groupId>
+ *     <artifactId>accumulo-core</artifactId>
+ *     <version>1.6.4</version>
+ *   </dependancy>
+ */
+public class ColumnVisibility {
+
+    Node node = null;
+    private byte[] expression;
+
+    /**
+     * Accessor for the underlying byte string.
+     *
+     * @return byte array representation of a visibility expression
+     */
+    public byte[] getExpression() {
+        return expression;
+    }
+
+    /**
+     * The node types in a parse tree for a visibility expression.
+     */
+    public static enum NodeType {
+        EMPTY, TERM, OR, AND,
+    }
+
+    /**
+     * All empty nodes are equal and represent the same value.
+     */
+    private static final Node EMPTY_NODE = new Node(NodeType.EMPTY, 0);
+
+    /**
+     * A node in the parse tree for a visibility expression.
+     */
+    public static class Node {
+        /**
+         * An empty list of nodes.
+         */
+        public final static List<Node> EMPTY = Collections.emptyList();
+        NodeType type;
+        int start;
+        int end;
+        List<Node> children = EMPTY;
+
+        public Node(final NodeType type, final int start) {
+            this.type = type;
+            this.start = start;
+            end = start + 1;
+        }
+
+        public Node(final int start, final int end) {
+            type = NodeType.TERM;
+            this.start = start;
+            this.end = end;
+        }
+
+        public void add(final Node child) {
+            if (children == EMPTY) {
+                children = new ArrayList<Node>();
+            }
+
+            children.add(child);
+        }
+
+        public NodeType getType() {
+            return type;
+        }
+
+        public List<Node> getChildren() {
+            return children;
+        }
+
+        public int getTermStart() {
+            return start;
+        }
+
+        public int getTermEnd() {
+            return end;
+        }
+
+        public ByteSequence getTerm(final byte expression[]) {
+            if (type != NodeType.TERM) {
+                throw new RuntimeException();
+            }
+
+            if (expression[start] == '"') {
+                // its a quoted term
+                final int qStart = start + 1;
+                final int qEnd = end - 1;
+
+                return new ArrayByteSequence(expression, qStart, qEnd - qStart);
+            }
+            return new ArrayByteSequence(expression, start, end - start);
+        }
+    }
+
+    /**
+     * A node comparator. Nodes sort according to node type, terms sort
+     * lexicographically. AND and OR nodes sort by number of children, or if the
+     * same by corresponding children.
+     */
+    public static class NodeComparator implements Comparator<Node>, Serializable {
+
+        private static final long serialVersionUID = 1L;
+        byte[] text;
+
+        /**
+         * Creates a new comparator.
+         *
+         * @param text expression string, encoded in UTF-8
+         */
+        public NodeComparator(final byte[] text) {
+            this.text = text;
+        }
+
+        @Override
+        public int compare(final Node a, final Node b) {
+            int diff = a.type.ordinal() - b.type.ordinal();
+            if (diff != 0) {
+                return diff;
+            }
+            switch (a.type) {
+                case EMPTY:
+                    return 0; // All empty nodes are the same
+                case TERM:
+                    return WritableComparator.compareBytes(text, a.start, a.end - a.start, text, b.start,
+                            b.end - b.start);
+                case OR:
+                case AND:
+                    diff = a.children.size() - b.children.size();
+                    if (diff != 0) {
+                        return diff;
+                    }
+                    for (int i = 0; i < a.children.size(); i++) {
+                        diff = compare(a.children.get(i), b.children.get(i));
+                        if (diff != 0) {
+                            return diff;
+                        }
+                    }
+            }
+            return 0;
+        }
+    }
+
+    /*
+     * Convience method that delegates to normalize with a new NodeComparator
+     * constructed using the supplied expression.
+     */
+    public static Node normalize(final Node root, final byte[] expression) {
+        return normalize(root, expression, new NodeComparator(expression));
+    }
+
+    // @formatter:off
+    /*
+     * Walks an expression's AST in order to: 1) roll up expressions with the
+     * same operant (`a&(b&c) becomes a&b&c`) 2) sorts labels lexicographically
+     * (permutations of `a&b&c` are re-ordered to appear as `a&b&c`) 3) dedupes
+     * labels (`a&b&a` becomes `a&b`)
+     */
+    // @formatter:on
+    public static Node normalize(final Node root, final byte[] expression, final NodeComparator comparator) {
+        if (root.type != NodeType.TERM) {
+            final TreeSet<Node> rolledUp = new TreeSet<Node>(comparator);
+            final java.util.Iterator<Node> itr = root.children.iterator();
+            while (itr.hasNext()) {
+                final Node c = normalize(itr.next(), expression, comparator);
+                if (c.type == root.type) {
+                    rolledUp.addAll(c.children);
+                    itr.remove();
+                }
+            }
+            rolledUp.addAll(root.children);
+            root.children.clear();
+            root.children.addAll(rolledUp);
+
+            // need to promote a child if it's an only child
+            if (root.children.size() == 1) {
+                return root.children.get(0);
+            }
+        }
+
+        return root;
+    }
+
+    /*
+     * Walks an expression's AST and appends a string representation to a
+     * supplied StringBuilder. This method adds parens where necessary.
+     */
+    public static void stringify(final Node root, final byte[] expression, final StringBuilder out) {
+        if (root.type == NodeType.TERM) {
+            out.append(new String(expression, root.start, root.end - root.start, UTF_8));
+        } else {
+            String sep = "";
+            for (final Node c : root.children) {
+                out.append(sep);
+                final boolean parens = c.type != NodeType.TERM && root.type != c.type;
+                if (parens) {
+                    out.append("(");
+                }
+                stringify(c, expression, out);
+                if (parens) {
+                    out.append(")");
+                }
+                sep = root.type == NodeType.AND ? "&" : "|";
+            }
+        }
+    }
+
+    /**
+     * Generates a byte[] that represents a normalized, but logically
+     * equivalent, form of this evaluator's expression.
+     *
+     * @return normalized expression in byte[] form
+     */
+    public byte[] flatten() {
+        final Node normRoot = normalize(node, expression);
+        final StringBuilder builder = new StringBuilder(expression.length);
+        stringify(normRoot, expression, builder);
+        return builder.toString().getBytes(UTF_8);
+    }
+
+    private static class ColumnVisibilityParser {
+        private int index = 0;
+        private int parens = 0;
+
+        public ColumnVisibilityParser() {
+        }
+
+        Node parse(final byte[] expression) {
+            if (expression.length > 0) {
+                final Node node = parse_(expression);
+                if (node == null) {
+                    throw new BadArgumentException("operator or missing parens", new String(expression, UTF_8),
+                            index - 1);
+                }
+                if (parens != 0) {
+                    throw new BadArgumentException("parenthesis mis-match", new String(expression, UTF_8), index - 1);
+                }
+                return node;
+            }
+            return null;
+        }
+
+        Node processTerm(final int start, final int end, final Node expr, final byte[] expression) {
+            if (start != end) {
+                if (expr != null) {
+                    throw new BadArgumentException("expression needs | or &", new String(expression, UTF_8), start);
+                }
+                return new Node(start, end);
+            }
+            if (expr == null) {
+                throw new BadArgumentException("empty term", new String(expression, UTF_8), start);
+            }
+            return expr;
+        }
+
+        Node parse_(final byte[] expression) {
+            Node result = null;
+            Node expr = null;
+            final int wholeTermStart = index;
+            int subtermStart = index;
+            boolean subtermComplete = false;
+
+            while (index < expression.length) {
+                switch (expression[index++]) {
+                    case '&': {
+                        expr = processTerm(subtermStart, index - 1, expr, expression);
+                        if (result != null) {
+                            if (!result.type.equals(NodeType.AND)) {
+                                throw new BadArgumentException("cannot mix & and |", new String(expression, UTF_8),
+                                        index - 1);
+                            }
+                        } else {
+                            result = new Node(NodeType.AND, wholeTermStart);
+                        }
+                        result.add(expr);
+                        expr = null;
+                        subtermStart = index;
+                        subtermComplete = false;
+                        break;
+                    }
+                    case '|': {
+                        expr = processTerm(subtermStart, index - 1, expr, expression);
+                        if (result != null) {
+                            if (!result.type.equals(NodeType.OR)) {
+                                throw new BadArgumentException("cannot mix | and &", new String(expression, UTF_8), index - 1);
+                            }
+                        } else {
+                            result = new Node(NodeType.OR, wholeTermStart);
+                        }
+                        result.add(expr);
+                        expr = null;
+                        subtermStart = index;
+                        subtermComplete = false;
+                        break;
+                    }
+                    case '(': {
+                        parens++;
+                        if (subtermStart != index - 1 || expr != null) {
+                            throw new BadArgumentException("expression needs & or |", new String(expression, UTF_8),
+                                    index - 1);
+                        }
+                        expr = parse_(expression);
+                        subtermStart = index;
+                        subtermComplete = false;
+                        break;
+                    }
+                    case ')': {
+                        parens--;
+                        final Node child = processTerm(subtermStart, index - 1, expr, expression);
+                        if (child == null && result == null) {
+                            throw new BadArgumentException("empty expression not allowed",
+                                    new String(expression, UTF_8), index);
+                        }
+                        if (result == null) {
+                            return child;
+                        }
+                        if (result.type == child.type) {
+                            for (final Node c : child.children) {
+                                result.add(c);
+                            }
+                        } else {
+                            result.add(child);
+                        }
+                        result.end = index - 1;
+                        return result;
+                    }
+                    case '"': {
+                        if (subtermStart != index - 1) {
+                            throw new BadArgumentException("expression needs & or |", new String(expression, UTF_8),
+                                    index - 1);
+                        }
+
+                        while (index < expression.length && expression[index] != '"') {
+                            if (expression[index] == '\\') {
+                                index++;
+                                if (expression[index] != '\\' && expression[index] != '"') {
+                                    throw new BadArgumentException("invalid escaping within quotes",
+                                            new String(expression, UTF_8), index - 1);
+                                }
+                            }
+                            index++;
+                        }
+
+                        if (index == expression.length) {
+                            throw new BadArgumentException("unclosed quote", new String(expression, UTF_8),
+                                    subtermStart);
+                        }
+
+                        if (subtermStart + 1 == index) {
+                            throw new BadArgumentException("empty term", new String(expression, UTF_8), subtermStart);
+                        }
+
+                        index++;
+
+                        subtermComplete = true;
+
+                        break;
+                    }
+                    default: {
+                        if (subtermComplete) {
+                            throw new BadArgumentException("expression needs & or |", new String(expression, UTF_8),
+                                    index - 1);
+                        }
+
+                        final byte c = expression[index - 1];
+                        if (!Authorizations.isValidAuthChar(c)) {
+                            throw new BadArgumentException("bad character (" + c + ")", new String(expression, UTF_8),
+                                    index - 1);
+                        }
+                    }
+                }
+            }
+            final Node child = processTerm(subtermStart, index, expr, expression);
+            if (result != null) {
+                result.add(child);
+                result.end = index;
+            } else {
+                result = child;
+            }
+            if (result.type != NodeType.TERM) {
+                if (result.children.size() < 2) {
+                    throw new BadArgumentException("missing term", new String(expression, UTF_8), index);
+                }
+            }
+            return result;
+        }
+    }
+
+    private void validate(final byte[] expression) {
+        if (expression != null && expression.length > 0) {
+            final ColumnVisibilityParser p = new ColumnVisibilityParser();
+            node = p.parse(expression);
+        } else {
+            node = EMPTY_NODE;
+        }
+        this.expression = expression;
+    }
+
+    /**
+     * Creates an empty visibility. Normally, elements with empty visibility can
+     * be seen by everyone. Though, one could change this behavior with filters.
+     *
+     * @see #ColumnVisibility(String)
+     */
+    public ColumnVisibility() {
+        this(new byte[] {});
+    }
+
+    /**
+     * Creates a column visibility for a Mutation.
+     *
+     * @param expression An expression of the rights needed to see this
+     *        mutation. The expression syntax is defined at the class-level
+     *        documentation
+     */
+    public ColumnVisibility(final String expression) {
+        this(expression.getBytes(UTF_8));
+    }
+
+    /**
+     * Creates a column visibility for a Mutation from a string already encoded
+     * in UTF-8 bytes.
+     *
+     * @param expression visibility expression, encoded as UTF-8 bytes
+     * @see #ColumnVisibility(String)
+     */
+    public ColumnVisibility(final byte[] expression) {
+        validate(expression);
+    }
+
+    @Override
+    public String toString() {
+        return "[" + new String(expression, UTF_8) + "]";
+    }
+
+    /**
+     * See {@link #equals(ColumnVisibility)}
+     */
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj instanceof ColumnVisibility) {
+            return equals((ColumnVisibility) obj);
+        }
+        return false;
+    }
+
+    /**
+     * Compares two ColumnVisibilities for string equivalence, not as a
+     * meaningful comparison of terms and conditions.
+     *
+     * @param otherLe other column visibility
+     * @return true if this visibility equals the other via string comparison
+     */
+    public boolean equals(final ColumnVisibility otherLe) {
+        return Arrays.equals(expression, otherLe.expression);
+    }
+
+    @Override
+    public int hashCode() {
+        return Arrays.hashCode(expression);
+    }
+
+    /**
+     * Gets the parse tree for this column visibility.
+     *
+     * @return parse tree node
+     */
+    public Node getParseTree() {
+        return node;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0ad2c511/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/FastByteComparison.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/FastByteComparison.java b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/FastByteComparison.java
new file mode 100644
index 0000000..220661e
--- /dev/null
+++ b/common/rya.api.model/src/main/java/org/apache/rya/api/model/visibility/FastByteComparison.java
@@ -0,0 +1,240 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.rya.api.model.visibility;
+
+import java.lang.reflect.Field;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+
+import com.google.common.primitives.Longs;
+import com.google.common.primitives.UnsignedBytes;
+
+import sun.misc.Unsafe;
+
+/**
+ * Utility code to do optimized byte-array comparison. This is borrowed and
+ * slightly modified from Guava's {@link UnsignedBytes} class to be able to
+ * compare arrays that start at non-zero offsets.
+ *
+ * XXX
+ * This class has been copied over because Rya has decided to use the Accumulo
+ * implementation of visibilities to control who is able to access what data
+ * within a Rya instance. Until we implement an Accumulo agnostic method for
+ * handling those visibility expressions, we have chosen to pull the Accumulo
+ * code into our API.
+ *
+ * Copied from accumulo's org.apache.hadoop.io.FastByteComparisons
+ *   <dependancy>
+ *     <groupId>org.apache.hadoop</groupId>
+ *     <artifactId>hadoop-commons</artifactId>
+ *     <version>2.5</version>
+ *   </dependancy>
+ */
+abstract class FastByteComparisons {
+
+    /**
+     * Lexicographically compare two byte arrays.
+     */
+    public static int compareTo(final byte[] b1, final int s1, final int l1, final byte[] b2, final int s2,
+            final int l2) {
+        return LexicographicalComparerHolder.BEST_COMPARER.compareTo(b1, s1, l1, b2, s2, l2);
+    }
+
+    private interface Comparer<T> {
+        abstract public int compareTo(T buffer1, int offset1, int length1, T buffer2, int offset2, int length2);
+    }
+
+    private static Comparer<byte[]> lexicographicalComparerJavaImpl() {
+        return LexicographicalComparerHolder.PureJavaComparer.INSTANCE;
+    }
+
+    /**
+     * Provides a lexicographical comparer implementation; either a Java
+     * implementation or a faster implementation based on {@link Unsafe}.
+     *
+     * <p>
+     * Uses reflection to gracefully fall back to the Java implementation if
+     * {@code Unsafe} isn't available.
+     */
+    private static class LexicographicalComparerHolder {
+        static final String UNSAFE_COMPARER_NAME = LexicographicalComparerHolder.class.getName() + "$UnsafeComparer";
+
+        static final Comparer<byte[]> BEST_COMPARER = getBestComparer();
+
+        /**
+         * Returns the Unsafe-using Comparer, or falls back to the pure-Java
+         * implementation if unable to do so.
+         */
+        static Comparer<byte[]> getBestComparer() {
+            try {
+                final Class<?> theClass = Class.forName(UNSAFE_COMPARER_NAME);
+
+                // yes, UnsafeComparer does implement Comparer<byte[]>
+                @SuppressWarnings("unchecked")
+                final Comparer<byte[]> comparer = (Comparer<byte[]>) theClass.getEnumConstants()[0];
+                return comparer;
+            } catch (final Throwable t) { // ensure we really catch *everything*
+                return lexicographicalComparerJavaImpl();
+            }
+        }
+
+        private enum PureJavaComparer implements Comparer<byte[]> {
+            INSTANCE;
+
+            @Override
+            public int compareTo(final byte[] buffer1, final int offset1, final int length1, final byte[] buffer2,
+                    final int offset2, final int length2) {
+                // Short circuit equal case
+                if (buffer1 == buffer2 && offset1 == offset2 && length1 == length2) {
+                    return 0;
+                }
+                // Bring WritableComparator code local
+                final int end1 = offset1 + length1;
+                final int end2 = offset2 + length2;
+                for (int i = offset1, j = offset2; i < end1 && j < end2; i++, j++) {
+                    final int a = buffer1[i] & 0xff;
+                    final int b = buffer2[j] & 0xff;
+                    if (a != b) {
+                        return a - b;
+                    }
+                }
+                return length1 - length2;
+            }
+        }
+
+        @SuppressWarnings("unused") // used via reflection
+        private enum UnsafeComparer implements Comparer<byte[]> {
+            INSTANCE;
+
+            static final Unsafe theUnsafe;
+
+            /** The offset to the first element in a byte array. */
+            static final int BYTE_ARRAY_BASE_OFFSET;
+
+            static {
+                theUnsafe = (Unsafe) AccessController.doPrivileged(new PrivilegedAction<Object>() {
+                    @Override
+                    public Object run() {
+                        try {
+                            final Field f = Unsafe.class.getDeclaredField("theUnsafe");
+                            f.setAccessible(true);
+                            return f.get(null);
+                        } catch (final NoSuchFieldException e) {
+                            // It doesn't matter what we throw;
+                            // it's swallowed in getBestComparer().
+                            throw new Error();
+                        } catch (final IllegalAccessException e) {
+                            throw new Error();
+                        }
+                    }
+                });
+
+                BYTE_ARRAY_BASE_OFFSET = theUnsafe.arrayBaseOffset(byte[].class);
+
+                // sanity check - this should never fail
+                if (theUnsafe.arrayIndexScale(byte[].class) != 1) {
+                    throw new AssertionError();
+                }
+            }
+
+            static final boolean littleEndian = ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN);
+
+            /**
+             * Returns true if x1 is less than x2, when both values are treated
+             * as unsigned.
+             */
+            static boolean lessThanUnsigned(final long x1, final long x2) {
+                return x1 + Long.MIN_VALUE < x2 + Long.MIN_VALUE;
+            }
+
+            /**
+             * Lexicographically compare two arrays.
+             *
+             * @param buffer1 left operand
+             * @param buffer2 right operand
+             * @param offset1 Where to start comparing in the left buffer
+             * @param offset2 Where to start comparing in the right buffer
+             * @param length1 How much to compare from the left buffer
+             * @param length2 How much to compare from the right buffer
+             * @return 0 if equal, < 0 if left is less than right, etc.
+             */
+            @Override
+            public int compareTo(final byte[] buffer1, final int offset1, final int length1, final byte[] buffer2,
+                    final int offset2, final int length2) {
+                // Short circuit equal case
+                if (buffer1 == buffer2 && offset1 == offset2 && length1 == length2) {
+                    return 0;
+                }
+                final int minLength = Math.min(length1, length2);
+                final int minWords = minLength / Longs.BYTES;
+                final int offset1Adj = offset1 + BYTE_ARRAY_BASE_OFFSET;
+                final int offset2Adj = offset2 + BYTE_ARRAY_BASE_OFFSET;
+
+                /*
+                 * Compare 8 bytes at a time. Benchmarking shows comparing 8
+                 * bytes at a time is no slower than comparing 4 bytes at a time
+                 * even on 32-bit. On the other hand, it is substantially faster
+                 * on 64-bit.
+                 */
+                for (int i = 0; i < minWords * Longs.BYTES; i += Longs.BYTES) {
+                    final long lw = theUnsafe.getLong(buffer1, offset1Adj + (long) i);
+                    final long rw = theUnsafe.getLong(buffer2, offset2Adj + (long) i);
+                    final long diff = lw ^ rw;
+
+                    if (diff != 0) {
+                        if (!littleEndian) {
+                            return lessThanUnsigned(lw, rw) ? -1 : 1;
+                        }
+
+                        // Use binary search
+                        int n = 0;
+                        int y;
+                        int x = (int) diff;
+                        if (x == 0) {
+                            x = (int) (diff >>> 32);
+                            n = 32;
+                        }
+
+                        y = x << 16;
+                        if (y == 0) {
+                            n += 16;
+                        } else {
+                            x = y;
+                        }
+
+                        y = x << 8;
+                        if (y == 0) {
+                            n += 8;
+                        }
+                        return (int) ((lw >>> n & 0xFFL) - (rw >>> n & 0xFFL));
+                    }
+                }
+
+                // The epilogue to cover the last (minLength % 8) elements.
+                for (int i = minWords * Longs.BYTES; i < minLength; i++) {
+                    final int result = UnsignedBytes.compare(buffer1[offset1 + i], buffer2[offset2 + i]);
+                    if (result != 0) {
+                        return result;
+                    }
+                }
+                return length1 - length2;
+            }
+        }
+    }
+}
\ No newline at end of file


[08/50] [abbrv] incubator-rya git commit: RYA-377 Fixing documentation.

Posted by ca...@apache.org.
RYA-377 Fixing documentation.


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

Branch: refs/heads/master
Commit: 4acbe5a7bc15e5315c2a638866bf8ab7ee755ca9
Parents: 3471cb7
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Nov 14 16:33:16 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../kafka/processors/StatementPatternProcessorSupplier.java    | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4acbe5a7/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
index 386fe98..bc99a7b 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
@@ -49,7 +49,6 @@ public class StatementPatternProcessorSupplier implements ProcessorSupplier<Stri
      * Constructs an instance of {@link StatementPatternProcessorSupplier}.
      *
      * @param sp - The statement pattern that the supplied processors will match against. (not null)
-     * @param keyFactory - The key factory that the supplied processors will use to create result keys. (not null)
      * @param resultFactory - The factory that the supplied processors will use to create results. (not null)
      */
     public StatementPatternProcessorSupplier(
@@ -66,8 +65,7 @@ public class StatementPatternProcessorSupplier implements ProcessorSupplier<Stri
 
     /**
      * Evaluates {@link VisibilityStatement}s against a {@link StatementPattern}. Any who match the pattern
-     * will forward a {@link VisibilityBindingSet} as well as store that new binding set in the local state store
-     * so that downstream join processors may access everything that has ever been emitted for the pattern.
+     * will forward a {@link VisibilityBindingSet} to the downstream processor.
      */
     @DefaultAnnotation(NonNull.class)
     public static final class StatementPatternProcessor implements Processor<String, VisibilityStatement> {
@@ -123,4 +121,4 @@ public class StatementPatternProcessorSupplier implements ProcessorSupplier<Stri
             // Nothing to do.
         }
     }
-}
\ No newline at end of file
+}


[11/50] [abbrv] incubator-rya git commit: RYA-377 interactor for stopping the processing of a query.

Posted by ca...@apache.org.
RYA-377 interactor for stopping the processing of a query.


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

Branch: refs/heads/master
Commit: 27255cc15acb557e8c394075dc78e5adc0b61d44
Parents: 5cb9a27
Author: Andrew Smith <sm...@gmail.com>
Authored: Mon Oct 30 18:16:16 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../rya/streams/api/interactor/StopQuery.java   | 41 ++++++++++++++++++++
 1 file changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/27255cc1/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/StopQuery.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/StopQuery.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/StopQuery.java
new file mode 100644
index 0000000..9fe0b83
--- /dev/null
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/StopQuery.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.api.interactor;
+
+import java.util.UUID;
+
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Stops processing a query in Rya Streams.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface StopQuery {
+
+    /**
+     * Stops processing a query in Rya Streams.
+     *
+     * @param queryID - The UUID for the SPARQL query that will be stopped. (not null)
+     * @throws RyaStreamsException The query could not be processed by Rya Streams.
+     */
+    public void stop(final UUID queryID) throws RyaStreamsException;
+}
\ No newline at end of file


[06/50] [abbrv] incubator-rya git commit: RYA-377 Implement the StatementPatternProcessor for the Rya Streams project.

Posted by ca...@apache.org.
RYA-377 Implement the StatementPatternProcessor for the Rya Streams project.


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

Branch: refs/heads/master
Commit: 653e4b83ac2fecac0c1b2107fbcdffe7bc357371
Parents: 0ad2c51
Author: kchilton2 <ke...@gmail.com>
Authored: Mon Nov 6 16:39:35 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 common/rya.api.function/pom.xml                 |  11 +-
 .../function/sp/StatementPatternMatcher.java    | 142 ++++++++++
 .../sp/StatementPatternMatcherTest.java         | 272 +++++++++++++++++++
 .../streams/api/interactor/LoadStatements.java  |  11 +-
 .../client/command/LoadStatementsCommand.java   |   2 +-
 extras/rya.streams/kafka/pom.xml                |   4 +
 .../kafka/interactor/KafkaLoadStatements.java   |  13 +-
 .../RyaStreamsSinkFormatterSupplier.java        |  83 ++++++
 .../StatementPatternProcessorSupplier.java      | 121 +++++++++
 .../apache/rya/streams/kafka/KafkaTestUtil.java | 127 +++++++++
 .../apache/rya/streams/kafka/RdfTestUtil.java   |  62 +++++
 .../interactor/KafkaGetQueryResultStreamIT.java |  20 +-
 .../kafka/interactor/KafkaLoadStatementsIT.java |  61 ++---
 .../processors/StatementPatternProcessorIT.java | 135 +++++++++
 .../kafka/queries/KafkaQueryChangeLogIT.java    |  26 +-
 .../VisibilityBindingSetKafkaIT.java            |  44 ++-
 .../VisibilityStatementKafkaIT.java             |  44 ++-
 17 files changed, 1039 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/common/rya.api.function/pom.xml
----------------------------------------------------------------------
diff --git a/common/rya.api.function/pom.xml b/common/rya.api.function/pom.xml
index f05dd6f..ce88e36 100644
--- a/common/rya.api.function/pom.xml
+++ b/common/rya.api.function/pom.xml
@@ -31,6 +31,7 @@ under the License.
     <name>Apache Rya Common API - Functions</name>
 
     <dependencies>
+        <!-- Rya dependencies. -->        
         <dependency>
             <groupId>org.apache.rya</groupId>
             <artifactId>rya.api.model</artifactId>
@@ -38,15 +39,23 @@ under the License.
 
         <!-- Third Party Dependencies -->
         <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-queryalgebra-model</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-queryalgebra-evaluation</artifactId>
+        </dependency>
+        <dependency>
             <groupId>com.google.guava</groupId>
             <artifactId>guava</artifactId>
         </dependency>
-
         <dependency>
             <groupId>com.github.stephenc.findbugs</groupId>
             <artifactId>findbugs-annotations</artifactId>
         </dependency>
 
+        <!-- Testing dependencies. -->
         <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/common/rya.api.function/src/main/java/org/apache/rya/api/function/sp/StatementPatternMatcher.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/sp/StatementPatternMatcher.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/sp/StatementPatternMatcher.java
new file mode 100644
index 0000000..208f8d1
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/sp/StatementPatternMatcher.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.sp;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Optional;
+
+import org.openrdf.model.Statement;
+import org.openrdf.model.Value;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.Var;
+import org.openrdf.query.algebra.evaluation.QueryBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+
+/**
+ * Matches {@link Statement}s against a {@link StatementPattern} and returns {@link BindingSet}s
+ * when the statement matched the pattern.
+ */
+@DefaultAnnotation(NonNull.class)
+public class StatementPatternMatcher {
+
+    private final StatementPattern pattern;
+
+    /**
+     * Constructs an instance of {@link StatementPatternMatcher}.
+     *
+     * @param pattern - The pattern that will be matched against. (not null)
+     */
+    public StatementPatternMatcher(final StatementPattern pattern) {
+        this.pattern = requireNonNull(pattern);
+    }
+
+    /**
+     * Matches a {@link Statement} against the provided {@link StatementPattern} and returns a {@link BindingSet}
+     * if the statement matched the pattern.
+     *
+     * @param statement - The statement that will be matched against the pattern. (not null)
+     * @return A {@link BinidngSet} containing the statement's values filled in for the pattern's variables if
+     *   the statement's values match the pattern's constants; otherwise empty.
+     */
+    public Optional<BindingSet> match(final Statement statement) {
+        requireNonNull(statement);
+
+        // Setup the resulting binding set that could be built from this Statement.
+        final QueryBindingSet bs = new QueryBindingSet();
+
+        if(matchesValue(pattern.getSubjectVar(), statement.getSubject(), bs) &&
+                matchesValue(pattern.getPredicateVar(), statement.getPredicate(), bs) &&
+                matchesValue(pattern.getObjectVar(), statement.getObject(), bs) &&
+                matchesContext(pattern.getContextVar(), statement.getContext(), bs)) {
+            return Optional.of(bs);
+        } else {
+            return Optional.empty();
+        }
+    }
+
+    /**
+     * The following table describes how a Subject, Predicate, and Object Var may be handled for a Statement and a
+     * Statement Pattern:
+     * <table border=1>
+     *     <tr> <th>Pattern's var is constant</th> <th>Effect on resulting BS</th> </tr>
+     *     <try> <td>yes</td> <td>Emit a BS if they match, no Context binding</td> </tr>
+     *     <try> <td>no</td>  <td>Emit a BS with a binding for the variable</td> </tr>
+     * </table>
+     *
+     * @param var - The statement pattern variable that is being matched. (not null)
+     * @param stmtValue - The statement's value for the variable. (not null)
+     * @param bs - The binding set that may be updated to include a binding for the variable. (not null)
+     * @return {@code true} if he variable and the statement value match, otherwise {@code false},
+     */
+    private boolean matchesValue(final Var var, final Value stmtValue, final QueryBindingSet bs) {
+        requireNonNull(var);
+        requireNonNull(stmtValue);
+        requireNonNull(bs);
+
+        // If the var is a constant, statement's value must match the var's value.
+        if(var.isConstant()) {
+            if(!stmtValue.equals(var.getValue())) {
+                return false;
+            }
+        } else {
+            // Otherwise it is a variable to be filled in.
+            bs.addBinding(var.getName(), stmtValue);
+        }
+
+        // Either the value matched the constant or the binding set was updated.
+        return true;
+    }
+
+    /**
+     * The following table describes how Context may be handled for a Statement and a Statement Pattern:
+     * <table border=1>
+     *   <tr> <th>Pattern's context state</th> <th>Statement has a context value</th> <th>Effect on resulting BS</th></tr>
+     *   <tr> <td>not mentioned</td>  <td>yes</td> <td>Emit BS without a Context binding</td> </tr>
+     *   <tr> <td>not mentioned</td>  <td>no</td>  <td>Emit BS without a Context binding</td> </tr>
+     *   <tr> <td>has a constant</td> <td>yes</td> <td>Emit BS if they match, no Context binding</td> </tr>
+     *   <tr> <td>has a constant</td> <td>no</td>  <td>Do not emit a BS</td> </tr>
+     *   <tr> <td>has a variable</td> <td>yes</td> <td>Emit BS with Context binding</td> </tr>
+     *   <tr> <td>has a variable</td> <td>no</td>  <td>Do not emit a BS</td> </tr>
+     * </table>
+     *
+     * @param cntxVar - The statement pattern's context variable. This may be {@code null} when there is no context
+     *   specified for the pattern.
+     * @param stmtCntx - The statement's context value. This may be {@code null} when there was no context
+     *   specified within the statement.
+     * @param bs - The binding set that may be updated to include a context binding. (not null)
+     * @return {@code true} if the the pattern's context variable and statement's context matched, otherwise {@code false}.
+     */
+    private boolean matchesContext(@Nullable final Var cntxVar, @Nullable final Value stmtCntx, final QueryBindingSet bs) {
+        if(cntxVar == null) {
+            // If there is no context, automatically matches.
+            return true;
+        } else if(stmtCntx == null) {
+            // If no value was provided within the statement, then it does not match.
+            return false;
+        } else {
+            // Otherwise handle it like a normal variable.
+            return matchesValue(cntxVar, stmtCntx, bs);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/common/rya.api.function/src/test/java/org/apache/rya/api/function/sp/StatementPatternMatcherTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/test/java/org/apache/rya/api/function/sp/StatementPatternMatcherTest.java b/common/rya.api.function/src/test/java/org/apache/rya/api/function/sp/StatementPatternMatcherTest.java
new file mode 100644
index 0000000..78a5418
--- /dev/null
+++ b/common/rya.api.function/src/test/java/org/apache/rya/api/function/sp/StatementPatternMatcherTest.java
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.sp;
+
+import static java.util.Objects.requireNonNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.junit.Test;
+import org.openrdf.model.Statement;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.evaluation.QueryBindingSet;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.parser.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
+
+/**
+ * Unit tests the methods of {@link StatementPatternMatcher}.
+ */
+public class StatementPatternMatcherTest {
+
+    @Test
+    public void matchesSubject() throws Exception {
+        // Create the matcher against a pattern that matches a specific subject.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "<urn:Alice> ?p ?o ." +
+                "}"));
+
+        // Create a statement that matches the pattern.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob"), vf.createURI("urn:testGraph"));
+
+        // Create the expected resulting Binding Set.
+        final QueryBindingSet expected = new QueryBindingSet();
+        expected.addBinding("p", vf.createURI("urn:talksTo"));
+        expected.addBinding("o", vf.createURI("urn:Bob"));
+
+        // Show the expected Binding Set matches the resulting Binding Set.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertEquals(expected, bs.get());
+    }
+
+    @Test
+    public void doesNotMatchSubject() throws Exception {
+        // Create the matcher against a pattern that matches a specific subject.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "<urn:Alice> ?p ?o ." +
+                "}"));
+
+        // Create a statement that does not match the pattern.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob"), vf.createURI("urn:testGraph"));
+
+        // Show the statement did not match.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertFalse(bs.isPresent());
+    }
+
+    @Test
+    public void matchesPredicate() throws Exception {
+        // Create the matcher against a pattern that matches a specific predicate.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "?s <urn:talksTo> ?o ." +
+                "}"));
+
+        // Create a statement that matches the pattern.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob"), vf.createURI("urn:testGraph"));
+
+        // Create the expected resulting Binding Set.
+        final QueryBindingSet expected = new QueryBindingSet();
+        expected.addBinding("s", vf.createURI("urn:Alice"));
+        expected.addBinding("o", vf.createURI("urn:Bob"));
+
+        // Show the expected Binding Set matches the resulting Binding Set.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertEquals(expected, bs.get());
+    }
+
+    @Test
+    public void doesNotMatchPredicate() throws Exception {
+        // Create the matcher against a pattern that matches a specific predicate.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "?s <urn:talksTo> ?o ." +
+                "}"));
+
+        // Create a statement that does not match the pattern.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:knows"), vf.createURI("urn:Bob"), vf.createURI("urn:testGraph"));
+
+        // Show the statement did not match.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertFalse(bs.isPresent());
+    }
+
+    @Test
+    public void matchesObject() throws Exception {
+        // Create the matcher against a pattern that matches a specific object.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "?s ?p <urn:Bob> ." +
+                "}"));
+
+        // Create a statement that matches the pattern.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob"), vf.createURI("urn:testGraph"));
+
+        // Create the expected resulting Binding Set.
+        final QueryBindingSet expected = new QueryBindingSet();
+        expected.addBinding("s", vf.createURI("urn:Alice"));
+        expected.addBinding("p", vf.createURI("urn:talksTo"));
+
+        // Show the expected Binding Set matches the resulting Binding Set.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertEquals(expected, bs.get());
+    }
+
+    @Test
+    public void doesNotMatchObject() throws Exception {
+        // Create the matcher against a pattern that matches a specific object.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "?s ?p <urn:Bob> ." +
+                "}"));
+
+        // Create a statement that does not match the pattern.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:knows"), vf.createURI("urn:Alice"), vf.createURI("urn:testGraph"));
+
+        // Show the statement did not match.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertFalse(bs.isPresent());
+    }
+
+    @Test
+    public void matchesContext() throws Exception {
+        // Create a matcher against a pattern that matches a specific context.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "GRAPH <urn:testGraph> {" +
+                        "?s ?p ?o ." +
+                    "}" +
+                "}"));
+
+        // Create a statement that matches the pattern.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob"), vf.createURI("urn:testGraph"));
+
+        // Create the expected resulting Binding Set.
+        final QueryBindingSet expected = new QueryBindingSet();
+        expected.addBinding("s", vf.createURI("urn:Alice"));
+        expected.addBinding("p", vf.createURI("urn:talksTo"));
+        expected.addBinding("o", vf.createURI("urn:Bob"));
+
+        // Show the expected Binding Set matches the resulting Binding Set.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertEquals(expected, bs.get());
+    }
+
+    @Test
+    public void doesNotMatchContext() throws Exception {
+        // Create a matcher against a pattern that matches a specific context.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "GRAPH <urn:testGraph> {" +
+                        "?s ?p ?o ." +
+                    "}" +
+                "}"));
+
+        // Create a statement that does not match the pattern.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob"), vf.createURI("urn:wrong"));
+
+        // Show the statement did not match.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertFalse(bs.isPresent());
+    }
+
+    @Test
+    public void variableContext() throws Exception {
+        // Create a matcher against a pattern that matches a variable context.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "GRAPH ?c {" +
+                        "?s ?p ?o ." +
+                    "}" +
+                "}"));
+
+        // Create a statement that matches the pattern.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob"), vf.createURI("urn:testGraph"));
+
+        // Create the expected resulting Binding Set.
+        final QueryBindingSet expected = new QueryBindingSet();
+        expected.addBinding("s", vf.createURI("urn:Alice"));
+        expected.addBinding("p", vf.createURI("urn:talksTo"));
+        expected.addBinding("o", vf.createURI("urn:Bob"));
+        expected.addBinding("c", vf.createURI("urn:testGraph"));
+
+        // Show the expected Binding Set matches the resulting Binding Set.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertEquals(expected, bs.get());
+    }
+
+    @Test
+    public void variableContext_contextFreeStatement() throws Exception {
+        // Create a matcher against a pattern that matches a variable context.
+        final StatementPatternMatcher matcher = new StatementPatternMatcher(getSp(
+                "SELECT * WHERE {" +
+                    "GRAPH ?c {" +
+                        "?s ?p ?o ." +
+                    "}" +
+                "}"));
+
+        // Create a statement that does not have a context value.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Statement statement = vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob"));
+
+        // Show the statement did not match.
+        final Optional<BindingSet> bs = matcher.match(statement);
+        assertFalse(bs.isPresent());
+    }
+
+    /**
+     * Fetch the {@link StatementPattern} from a SPARQL string.
+     *
+     * @param sparql - A SPARQL query that contains only a single Statement Patern. (not nul)
+     * @return The {@link StatementPattern} that was in the query, if it could be found. Otherwise {@code null}
+     * @throws Exception The statement pattern could not be found in the parsed SPARQL query.
+     */
+    public static @Nullable StatementPattern getSp(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<StatementPattern> statementPattern = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visitChildren(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final StatementPattern node) throws Exception {
+                statementPattern.set(node);
+            }
+        });
+        return statementPattern.get();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/LoadStatements.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/LoadStatements.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/LoadStatements.java
index babe914..c64a08d 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/LoadStatements.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/LoadStatements.java
@@ -19,6 +19,7 @@
 package org.apache.rya.streams.api.interactor;
 
 import java.nio.file.Path;
+import java.util.Collection;
 
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.api.exception.RyaStreamsException;
@@ -45,5 +46,13 @@ public interface LoadStatements {
      * @throws RyaStreamsException Thrown when the format of the file provided is unknown,
      *         or not a valid RDF format.
      */
-    public void load(final Path statementsPath, final String visibilities) throws RyaStreamsException;
+    public void fromFile(final Path statementsPath, final String visibilities) throws RyaStreamsException;
+
+    /**
+     * Loads a series of {@link VisibilityStatement}s from a collection into the RyaStreams system.
+     *
+     * @param statements - The statements that will be loaded. (not null)
+     * @throws RyaStreamsException The statements could not be loaded.
+     */
+    public void fromCollection(Collection<VisibilityStatement> statements) throws RyaStreamsException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
index 6ae63da..9414b28 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
@@ -127,7 +127,7 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
         final Properties producerProps = buildProperties(params);
         try (final Producer<Object, VisibilityStatement> producer = new KafkaProducer<>(producerProps)) {
             final LoadStatements statements = new KafkaLoadStatements(KafkaTopics.statementsTopic(params.ryaInstance), producer);
-            statements.load(statementsPath, params.visibilities);
+            statements.fromFile(statementsPath, params.visibilities);
         } catch (final Exception e) {
             System.err.println("Unable to parse statements file: " + statementsPath.toString());
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/pom.xml b/extras/rya.streams/kafka/pom.xml
index d5fffe0..33cc985 100644
--- a/extras/rya.streams/kafka/pom.xml
+++ b/extras/rya.streams/kafka/pom.xml
@@ -41,6 +41,10 @@ under the License.
         </dependency>
         <dependency>
             <groupId>org.apache.rya</groupId>
+            <artifactId>rya.api.function</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
             <artifactId>rya.streams.api</artifactId>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
index 4cf8f9b..8ab3ab6 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
@@ -23,6 +23,7 @@ import static java.util.Objects.requireNonNull;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.util.Collection;
 
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
@@ -65,7 +66,7 @@ public class KafkaLoadStatements implements LoadStatements {
 
 
     @Override
-    public void load(final Path statementsPath, final String visibilities) throws RyaStreamsException {
+    public void fromFile(final Path statementsPath, final String visibilities) throws RyaStreamsException {
         requireNonNull(statementsPath);
         requireNonNull(visibilities);
 
@@ -99,4 +100,14 @@ public class KafkaLoadStatements implements LoadStatements {
             throw new RyaStreamsException("Could not load the RDF file's Statements into Rya Streams.", e);
         }
     }
+
+    @Override
+    public void fromCollection(final Collection<VisibilityStatement> statements) throws RyaStreamsException {
+        requireNonNull(statements);
+
+        for(final VisibilityStatement statement : statements) {
+            producer.send(new ProducerRecord<>(topic, statement));
+        }
+        producer.flush();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsSinkFormatterSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsSinkFormatterSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsSinkFormatterSupplier.java
new file mode 100644
index 0000000..d6a8d2d
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/RyaStreamsSinkFormatterSupplier.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.ProcessorSupplier;
+import org.apache.rya.api.model.VisibilityBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Supplies {@link RyaStreamsSinkFormatter} instances.
+ */
+@DefaultAnnotation(NonNull.class)
+public class RyaStreamsSinkFormatterSupplier implements ProcessorSupplier<Object, ProcessorResult> {
+
+    @Override
+    public Processor<Object, ProcessorResult> get() {
+        return new RyaStreamsSinkFormatter();
+    }
+
+    /**
+     * Accepts {@link ProcessorResult}s and forwards just their {@link VisibilityBindingSet} so that it may be
+     * written to a sink.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static final class RyaStreamsSinkFormatter implements Processor<Object, ProcessorResult> {
+
+        private ProcessorContext processorContext;
+
+        @Override
+        public void init(final ProcessorContext context) {
+            processorContext = context;
+        }
+
+        @Override
+        public void process(final Object key, final ProcessorResult value) {
+
+            VisibilityBindingSet result = null;
+            switch(value.getType()) {
+                case UNARY:
+                    result = value.getUnary().getResult();
+                    break;
+
+                case BINARY:
+                    result = value.getBinary().getResult();
+                    break;
+            }
+
+            if(result != null) {
+                processorContext.forward(key, result);
+            }
+        }
+
+        @Override
+        public void punctuate(final long timestamp) {
+            // Does nothing.
+        }
+
+        @Override
+        public void close() {
+            // Does nothing.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
new file mode 100644
index 0000000..6991783
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Optional;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.ProcessorSupplier;
+import org.apache.rya.api.function.sp.StatementPatternMatcher;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.algebra.StatementPattern;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Supplies {@link StatementPatternProcessor} instances.
+ */
+@DefaultAnnotation(NonNull.class)
+public class StatementPatternProcessorSupplier implements ProcessorSupplier<String, VisibilityStatement> {
+
+    private final StatementPattern sp;
+    private final ProcessorResultFactory resultFactory;
+
+    /**
+     * Constructs an instance of {@link StatementPatternProcessorSupplier}.
+     *
+     * @param sp - The statement pattern that the supplied processors will match against. (not null)
+     * @param keyFactory - The key factory that the supplied processors will use to create result keys. (not null)
+     * @param resultFactory - The factory that the supplied processors will use to create results. (not null)
+     */
+    public StatementPatternProcessorSupplier(
+            final StatementPattern sp,
+            final ProcessorResultFactory resultFactory) {
+        this.sp = requireNonNull(sp);
+        this.resultFactory = requireNonNull(resultFactory);
+    }
+
+    @Override
+    public Processor<String, VisibilityStatement> get() {
+        return new StatementPatternProcessor(sp, resultFactory);
+    }
+
+    /**
+     * Evaluates {@link VisibilityStatement}s against a {@link StatementPattern}. Any who match the pattern
+     * will forward a {@link VisibilityBindingSet} as well as store that new binding set in the local state store
+     * so that downstream join processors may access everything that has ever been emitted for the pattern.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static final class StatementPatternProcessor implements Processor<String, VisibilityStatement> {
+
+        private final StatementPatternMatcher spMatcher;
+        private final ProcessorResultFactory resultFactory;
+
+        private ProcessorContext context;
+
+        /**
+         * Constructs an instance of {@link StatementPatternProcessor}.
+         *
+         * @param sp - The statement pattern that the processor will match statements against. (not null)
+         * @param resultFactory - The factory that the processor will use to create results. (not null)
+         */
+        public StatementPatternProcessor(
+                final StatementPattern sp,
+                final ProcessorResultFactory resultFactory) {
+            this.spMatcher = new StatementPatternMatcher( requireNonNull(sp) );
+            this.resultFactory = requireNonNull(resultFactory);
+        }
+
+        @Override
+        public void init(final ProcessorContext context) {
+            this.context = context;
+        }
+
+        @Override
+        public void process(final String key, final VisibilityStatement statement) {
+            // Check to see if the Statement matches the Statement Pattern.
+            final Optional<BindingSet> bs = spMatcher.match(statement);
+
+            if(bs.isPresent()) {
+                // If it does, wrap the Binding Set with the Statement's visibility expression and write it to the state store.
+                final VisibilityBindingSet visBs = new VisibilityBindingSet(bs.get(), statement.getVisibility());
+
+                // Wrap the binding set as a result and forward it to the downstream processor.
+                final ProcessorResult resultValue = resultFactory.make(visBs);
+                context.forward(key, resultValue);
+            }
+        }
+
+        @Override
+        public void punctuate(final long timestamp) {
+            // Nothing to do.
+        }
+
+        @Override
+        public void close() {
+            // Nothing to do.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
new file mode 100644
index 0000000..bff4fdb
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A set of utility functions that are useful when writing tests against a Kafka instance.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class KafkaTestUtil {
+
+    private KafkaTestUtil() { }
+
+    /**
+     * Create a {@link Producer} that is able to write to a topic that is hosted within an embedded instance of Kafka.
+     *
+     * @param kafka - The Kafka rule used to connect to the embedded Kafkfa instance. (not null)
+     * @param keySerializerClass - Serializes the keys. (not null)
+     * @param valueSerializerClass - Serializes the values. (not null)
+     * @return A {@link Producer} that can be used to write records to a topic.
+     */
+    public static <K, V> Producer<K, V> makeProducer(
+            final KafkaTestInstanceRule kafka,
+            final Class<? extends Serializer<K>> keySerializerClass,
+            final Class<? extends Serializer<V>> valueSerializerClass) {
+        requireNonNull(kafka);
+        requireNonNull(keySerializerClass);
+        requireNonNull(valueSerializerClass);
+
+        final Properties props = kafka.createBootstrapServerConfig();
+        props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass.getName());
+        props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName());
+        return new KafkaProducer<>(props);
+    }
+
+    /**
+     * Create a {@link Consumer} that has a unique group ID and reads everything from a topic that is hosted within an
+     * embedded instance of Kafka starting at the earliest point by default.
+     *
+     * @param kafka - The Kafka rule used to connect to the embedded Kafkfa instance. (not null)
+     * @param keyDeserializerClass - Deserializes the keys. (not null)
+     * @param valueDeserializerClass - Deserializes the values. (not null)
+     * @return A {@link Consumer} that can be used to read records from a topic.
+     */
+    public static <K, V> Consumer<K, V> fromStartConsumer(
+            final KafkaTestInstanceRule kafka,
+            final Class<? extends Deserializer<K>> keyDeserializerClass,
+            final Class<? extends Deserializer<V>> valueDeserializerClass) {
+        requireNonNull(kafka);
+        requireNonNull(keyDeserializerClass);
+        requireNonNull(valueDeserializerClass);
+
+        final Properties props = kafka.createBootstrapServerConfig();
+        props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
+        props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass.getName());
+        props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass.getName());
+        return new KafkaConsumer<>(props);
+    }
+
+    /**
+     * Polls a {@link Consumer> until it has either polled too many times without hitting the target number
+     * of results, or it hits the target number of results.
+     *
+     * @param pollMs - How long each poll could take.
+     * @param pollIterations - The maximum number of polls that will be attempted.
+     * @param targetSize - The number of results to read before stopping.
+     * @param consumer - The consumer that will be polled.
+     * @return The results that were read frmo the consumer.
+     * @throws Exception If the poll failed.
+     */
+    public static <K, V> List<V> pollForResults(
+            final int pollMs,
+            final int pollIterations,
+            final int targetSize,
+            final Consumer<K, V> consumer) throws Exception {
+        requireNonNull(consumer);
+
+        final List<V> values = new ArrayList<>();
+
+        int i = 0;
+        while(values.size() < targetSize && i < pollIterations) {
+            for(final ConsumerRecord<K, V> record : consumer.poll(pollMs)) {
+                values.add( record.value() );
+            }
+            i++;
+        }
+
+        return values;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
new file mode 100644
index 0000000..109e40d
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.parser.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+
+/**
+ * A set of utility functions that are useful when writing tests RDF functions.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class RdfTestUtil {
+
+    private RdfTestUtil() { }
+
+    /**
+     * Fetch the {@link StatementPattern} from a SPARQL string.
+     *
+     * @param sparql - A SPARQL query that contains only a single Statement Patern. (not nul)
+     * @return The {@link StatementPattern} that was in the query, if it could be found. Otherwise {@code null}
+     * @throws Exception The statement pattern could not be found in the parsed SPARQL query.
+     */
+    public static @Nullable StatementPattern getSp(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<StatementPattern> statementPattern = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visitChildren(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final StatementPattern node) throws Exception {
+                statementPattern.set(node);
+            }
+        });
+        return statementPattern.get();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
index 3343f76..67889e9 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
@@ -23,17 +23,15 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Properties;
 import java.util.UUID;
 
-import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.api.entity.QueryResultStream;
 import org.apache.rya.streams.api.interactor.GetQueryResultStream;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
@@ -52,16 +50,6 @@ public class KafkaGetQueryResultStreamIT {
     public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
 
     /**
-     * @return A {@link Producer} that is able to write {@link VisibilityBindingSet}s.
-     */
-    private Producer<?, VisibilityBindingSet> makeProducer() {
-        final Properties producerProps = kafka.createBootstrapServerConfig();
-        producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VisibilityBindingSetSerializer.class.getName());
-        return new KafkaProducer<>(producerProps);
-    }
-
-    /**
      * Polls a {@link QueryResultStream} until it has either polled too many times without hitting
      * the target number of results, or it hits the target number of results.
      *
@@ -112,7 +100,8 @@ public class KafkaGetQueryResultStreamIT {
         original.add(new VisibilityBindingSet(bs, "b|c"));
 
         // Write some entries to the query result topic in Kafka.
-        try(final Producer<?, VisibilityBindingSet> producer = makeProducer()) {
+        try(final Producer<?, VisibilityBindingSet> producer =
+                KafkaTestUtil.makeProducer(kafka, StringSerializer.class, VisibilityBindingSetSerializer.class)) {
             final String resultTopic = KafkaTopics.queryResultsTopic(queryId);
             for(final VisibilityBindingSet visBs : original) {
                 producer.send(new ProducerRecord<>(resultTopic, visBs));
@@ -132,7 +121,8 @@ public class KafkaGetQueryResultStreamIT {
         // Create an ID for the query.
         final UUID queryId = UUID.randomUUID();
 
-        try(final Producer<?, VisibilityBindingSet> producer = makeProducer()) {
+        try(final Producer<?, VisibilityBindingSet> producer =
+                KafkaTestUtil.makeProducer(kafka, StringSerializer.class, VisibilityBindingSetSerializer.class)) {
             final String resultTopic = KafkaTopics.queryResultsTopic(queryId);
 
             // Write a single visibility binding set to the query's result topic. This will not appear in the expected results.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
index 5a81d23..b48addd 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
@@ -24,21 +24,14 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Iterator;
 import java.util.List;
-import java.util.Properties;
-import java.util.UUID;
 
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
 import org.apache.rya.test.kafka.KafkaITBase;
@@ -62,48 +55,30 @@ public class KafkaLoadStatementsIT extends KafkaITBase {
 
     @Test(expected = UnsupportedRDFormatException.class)
     public void test_invalidFile() throws Exception {
-        final String topic = rule.getKafkaTopicName();
-        final String visibilities = "a|b|c";
-        final Properties props = rule.createBootstrapServerConfig();
-        props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VisibilityStatementSerializer.class.getName());
-        try (final Producer<Object, VisibilityStatement> producer = new KafkaProducer<>(props)) {
-            final KafkaLoadStatements command = new KafkaLoadStatements(topic, producer);
-            command.load(INVALID, visibilities);
+        try(final Producer<?, VisibilityStatement> producer =
+                KafkaTestUtil.makeProducer(rule, StringSerializer.class, VisibilityStatementSerializer.class)) {
+            final KafkaLoadStatements command = new KafkaLoadStatements(rule.getKafkaTopicName(), producer);
+            command.fromFile(INVALID, "a|b|c");
         }
     }
 
     @Test
     public void testTurtle() throws Exception {
-        final String topic = rule.getKafkaTopicName();
         final String visibilities = "a|b|c";
-        final Properties props = rule.createBootstrapServerConfig();
-        props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VisibilityStatementSerializer.class.getName());
-        try (final Producer<Object, VisibilityStatement> producer = new KafkaProducer<>(props)) {
-            final KafkaLoadStatements command = new KafkaLoadStatements(topic, producer);
-            command.load(TURTLE_FILE, visibilities);
-        }
-
-        // Read a VisibilityBindingSet from the test topic.
-        final List<VisibilityStatement> read = new ArrayList<>();
 
-        final Properties consumerProps = rule.createBootstrapServerConfig();
-        consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
-        consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
-        consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VisibilityStatementDeserializer.class.getName());
+        // Load the statements into the kafka topic.
+        try(final Producer<?, VisibilityStatement> producer =
+                KafkaTestUtil.makeProducer(rule, StringSerializer.class, VisibilityStatementSerializer.class)) {
+            final KafkaLoadStatements command = new KafkaLoadStatements(rule.getKafkaTopicName(), producer);
+            command.fromFile(TURTLE_FILE, visibilities);
+        }
 
-        try (final KafkaConsumer<String, VisibilityStatement> consumer = new KafkaConsumer<>(consumerProps)) {
+        // Read a VisibilityBindingSets from the test topic.
+        final List<VisibilityStatement> read;// = new ArrayList<>();
+        try(Consumer<String, VisibilityStatement> consumer =
+                KafkaTestUtil.fromStartConsumer(rule, StringDeserializer.class, VisibilityStatementDeserializer.class)) {
             consumer.subscribe(Arrays.asList(rule.getKafkaTopicName()));
-            final ConsumerRecords<String, VisibilityStatement> records = consumer.poll(2000);
-
-            assertEquals(3, records.count());
-            final Iterator<ConsumerRecord<String, VisibilityStatement>> iter = records.iterator();
-            while(iter.hasNext()) {
-                final VisibilityStatement visiSet = iter.next().value();
-                read.add(visiSet);
-            }
+            read = KafkaTestUtil.pollForResults(500, 6, 3, consumer);
         }
 
         final List<VisibilityStatement> original = new ArrayList<>();
@@ -121,4 +96,4 @@ public class KafkaLoadStatementsIT extends KafkaITBase {
         // Show the written statement matches the read one.
         assertEquals(original, read);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
new file mode 100644
index 0000000..1b58b42
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RdfTestUtil;
+import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.RyaStreamsSinkFormatterSupplier.RyaStreamsSinkFormatter;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.evaluation.QueryBindingSet;
+
+/**
+ * Integration tests the methods of {@link StatementPatternProcessor}.
+ */
+public class StatementPatternProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void statementPatternMatches() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPattern object that will be evaluated.
+        final StatementPattern sp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "SP1");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Start the streams program.
+        final Properties props = kafka.createBootstrapServerConfig();
+        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "StatementPatternProcessorIT");
+
+        final KafkaStreams streams = new KafkaStreams(builder, new StreamsConfig(props));
+        streams.cleanUp();
+        try {
+            streams.start();
+
+            // Wait for the streams application to start. Streams only see data after their consumers are connected.
+            Thread.sleep(2000);
+
+            // Load some data into the input topic.
+            final ValueFactory vf = new ValueFactoryImpl();
+            final List<VisibilityStatement> statements = new ArrayList<>();
+            statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+
+            try(Producer<String, VisibilityStatement> producer = KafkaTestUtil.makeProducer(
+                    kafka, StringSerializer.class, VisibilityStatementSerializer.class)) {
+                new KafkaLoadStatements(statementsTopic, producer).fromCollection(statements);
+            }
+
+            // Wait for the final results to appear in the output topic and verify the expected Binding Set was found.
+            try(Consumer<String, VisibilityBindingSet> consumer = KafkaTestUtil.fromStartConsumer(
+                    kafka, StringDeserializer.class, VisibilityBindingSetDeserializer.class)) {
+                // Register the topic.
+                consumer.subscribe(Arrays.asList(resultsTopic));
+
+                // Poll for the result.
+                final List<VisibilityBindingSet> results = KafkaTestUtil.pollForResults(500, 6, 1, consumer);
+
+                // Show the correct binding set results from the job.
+                final QueryBindingSet bs = new QueryBindingSet();
+                bs.addBinding("person", vf.createURI("urn:Alice"));
+                bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+                final VisibilityBindingSet expected = new VisibilityBindingSet(bs, "a");
+
+                final VisibilityBindingSet result = results.iterator().next();
+                assertEquals(expected, result);
+            }
+        } finally {
+            streams.close();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
index 9e89ca7..ff2b59b 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
@@ -22,22 +22,18 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Properties;
 import java.util.UUID;
 
 import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.streams.api.queries.ChangeLogEntry;
 import org.apache.rya.streams.api.queries.QueryChange;
 import org.apache.rya.streams.api.queries.QueryChangeLog.QueryChangeLogException;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
 import org.apache.rya.test.kafka.KafkaITBase;
@@ -55,11 +51,10 @@ import info.aduna.iteration.CloseableIteration;
  * Integration tests the {@link KafkaQueryChangeLog}.
  */
 public class KafkaQueryChangeLogIT extends KafkaITBase {
-    KafkaQueryChangeLog changeLog;
 
+    private KafkaQueryChangeLog changeLog;
     private Producer<?, QueryChange> producer;
     private Consumer<?, QueryChange> consumer;
-
     private String topic;
 
     @Rule
@@ -68,25 +63,14 @@ public class KafkaQueryChangeLogIT extends KafkaITBase {
     @Before
     public void setup() {
         topic = rule.getKafkaTopicName();
-        final Properties producerProperties = rule.createBootstrapServerConfig();
-        producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, QueryChangeSerializer.class.getName());
-
-        final Properties consumerProperties = rule.createBootstrapServerConfig();
-        consumerProperties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
-        consumerProperties.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
-        consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, QueryChangeDeserializer.class.getName());
-        producer = new KafkaProducer<>(producerProperties);
-        consumer = new KafkaConsumer<>(consumerProperties);
+        producer = KafkaTestUtil.makeProducer(rule, StringSerializer.class, QueryChangeSerializer.class);
+        consumer = KafkaTestUtil.fromStartConsumer(rule, StringDeserializer.class, QueryChangeDeserializer.class);
         changeLog = new KafkaQueryChangeLog(producer, consumer, topic);
     }
 
     @After
     public void cleanup() {
-        producer.flush();
         producer.close();
-
         consumer.close();
     }
 
@@ -202,4 +186,4 @@ public class KafkaQueryChangeLogIT extends KafkaITBase {
         }
         return changes;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java
index 6104578..f9129ff 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java
@@ -21,18 +21,15 @@ package org.apache.rya.streams.kafka.serialization;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Arrays;
-import java.util.Properties;
-import java.util.UUID;
+import java.util.List;
 
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -49,7 +46,7 @@ public class VisibilityBindingSetKafkaIT {
     public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
 
     @Test
-    public void readAndWrite() {
+    public void readAndWrite() throws Exception {
         // Create the object that will be written to the topic.
         final ValueFactory vf = new ValueFactoryImpl();
 
@@ -59,32 +56,23 @@ public class VisibilityBindingSetKafkaIT {
         final VisibilityBindingSet original = new VisibilityBindingSet(bs, "a|b|c");
 
         // Write a VisibilityBindingSet to the test topic.
-        final Properties producerProps = kafka.createBootstrapServerConfig();
-        producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VisibilityBindingSetSerializer.class.getName());
-
-        try(final KafkaProducer<String, VisibilityBindingSet> producer = new KafkaProducer<>(producerProps)) {
+        try(Producer<String, VisibilityBindingSet> producer = KafkaTestUtil.makeProducer(
+                kafka, StringSerializer.class, VisibilityBindingSetSerializer.class)) {
             producer.send( new ProducerRecord<String, VisibilityBindingSet>(kafka.getKafkaTopicName(), original) );
         }
 
         // Read a VisibilityBindingSet from the test topic.
-        VisibilityBindingSet read;
-
-        final Properties consumerProps = kafka.createBootstrapServerConfig();
-        consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
-        consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
-        consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VisibilityBindingSetDeserializer.class.getName());
-
-        try(final KafkaConsumer<String, VisibilityBindingSet> consumer = new KafkaConsumer<>(consumerProps)) {
+        try(Consumer<String, VisibilityBindingSet> consumer = KafkaTestUtil.fromStartConsumer(
+                kafka, StringDeserializer.class, VisibilityBindingSetDeserializer.class)) {
+            // Register the topic.
             consumer.subscribe(Arrays.asList(kafka.getKafkaTopicName()));
-            final ConsumerRecords<String, VisibilityBindingSet> records = consumer.poll(1000);
 
-            assertEquals(1, records.count());
-            read = records.iterator().next().value();
-        }
+            // Poll for the result.
+            final List<VisibilityBindingSet> results = KafkaTestUtil.pollForResults(500, 6, 1, consumer);
 
-        // Show the written statement matches the read one.
-        assertEquals(original, read);
+            // Show the written statement matches the read one.
+            final VisibilityBindingSet read = results.iterator().next();
+            assertEquals(original, read);
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/653e4b83/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java
index 62122bd..b85eb0c 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java
@@ -21,18 +21,15 @@ package org.apache.rya.streams.kafka.serialization;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Arrays;
-import java.util.Properties;
-import java.util.UUID;
+import java.util.List;
 
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -48,7 +45,7 @@ public class VisibilityStatementKafkaIT {
     public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
 
     @Test
-    public void readAndWrite() {
+    public void readAndWrite() throws Exception {
         // Create the object that will be written to the topic.
         final ValueFactory vf = new ValueFactoryImpl();
         final VisibilityStatement original = new VisibilityStatement(
@@ -60,32 +57,23 @@ public class VisibilityStatementKafkaIT {
                 "a|b|c");
 
         // Write a VisibilityStatement to the test topic.
-        final Properties producerProps = kafka.createBootstrapServerConfig();
-        producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
-        producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VisibilityStatementSerializer.class.getName());
-
-        try(final KafkaProducer<String, VisibilityStatement> producer = new KafkaProducer<>(producerProps)) {
+        try(Producer<String, VisibilityStatement> producer = KafkaTestUtil.makeProducer(
+                kafka, StringSerializer.class, VisibilityStatementSerializer.class)) {
             producer.send( new ProducerRecord<String, VisibilityStatement>(kafka.getKafkaTopicName(), original) );
         }
 
         // Read a VisibilityStatement from the test topic.
-        VisibilityStatement read;
-
-        final Properties consumerProps = kafka.createBootstrapServerConfig();
-        consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
-        consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
-        consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName());
-        consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VisibilityStatementDeserializer.class.getName());
-
-        try(final KafkaConsumer<String, VisibilityStatement> consumer = new KafkaConsumer<>(consumerProps)) {
+        try(Consumer<String, VisibilityStatement> consumer = KafkaTestUtil.fromStartConsumer(
+                kafka, StringDeserializer.class, VisibilityStatementDeserializer.class)) {
+            // Register the topic.
             consumer.subscribe(Arrays.asList(kafka.getKafkaTopicName()));
-            final ConsumerRecords<String, VisibilityStatement> records = consumer.poll(1000);
 
-            assertEquals(1, records.count());
-            read = records.iterator().next().value();
-        }
+            // Poll for the result.
+            final List<VisibilityStatement> results = KafkaTestUtil.pollForResults(500, 6, 1, consumer);
 
-        // Show the written statement matches the read one.
-        assertEquals(original, read);
+            // Show the written statement matches the read one.
+            final VisibilityStatement read = results.iterator().next();
+            assertEquals(original, read);
+        }
     }
 }
\ No newline at end of file


[05/50] [abbrv] incubator-rya git commit: RYA-377 Define the JoinStateStore interface.

Posted by ca...@apache.org.
RYA-377 Define the JoinStateStore interface.


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

Branch: refs/heads/master
Commit: 0a2170a1143266442ab883e86aab9fcd6b365b27
Parents: 653e4b8
Author: kchilton2 <ke...@gmail.com>
Authored: Thu Nov 9 14:41:08 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../kafka/processors/join/JoinStateStore.java   | 53 ++++++++++++++++++++
 1 file changed, 53 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/0a2170a1/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
new file mode 100644
index 0000000..e53ec68
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.join;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Provides a mechanism for storing {@link VisibilityBindingSet}s that have been emitted from either side of
+ * a Join and a way to fetch all {@link VisibilityBindingSet}s that join with it from the other side.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface JoinStateStore {
+
+    /**
+     * Store a {@link VisibilityBindingSet} based on the side it was emitted from.
+     *
+     * @param joinVars - An ordered list of the variables that are being joined over. (not null)
+     * @param result - The result whose value will be stored. (not null)
+     */
+    public void store(List<String> joinVars, BinaryResult result);
+
+    /**
+     * Get the previously stored {@link VisibilityBindingSet}s that join with the provided result.
+     *
+     * @param joinVars - An ordered list of the variables to join over. (not null)
+     * @param result - Defines the values that will be used to join. (not null)
+     * @return The {@link VisibilityBinidngSet}s that join with {@code result}.
+     */
+    public Iterator<VisibilityBindingSet> getJoinedValues(List<String> joinVars, BinaryResult result);
+}
\ No newline at end of file


[34/50] [abbrv] incubator-rya git commit: RYA-377 Fixing broken build.

Posted by ca...@apache.org.
RYA-377 Fixing broken build.


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

Branch: refs/heads/master
Commit: 923448f158a6de8cd0dcc0ce0a60f3bef693a3fc
Parents: 3ebf6db
Author: kchilton2 <ke...@gmail.com>
Authored: Wed Dec 13 15:05:45 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../api/function/Filter/FilterEvaluator.java    | 117 -------------------
 .../api/function/filter/FilterEvaluator.java    | 117 +++++++++++++++++++
 .../rya/api/function/temporal/TemporalURIs.java |  19 +++
 .../function/filter/FilterEvaluatorTest.java    |   2 +-
 .../filter/FilterProcessorSupplier.java         |   2 +-
 .../streams/kafka/topology/TopologyFactory.java |   2 +-
 .../processors/filter/FilterProcessorTest.java  |   2 +-
 7 files changed, 140 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/923448f1/common/rya.api.function/src/main/java/org/apache/rya/api/function/Filter/FilterEvaluator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/Filter/FilterEvaluator.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/Filter/FilterEvaluator.java
deleted file mode 100644
index d1e1776..0000000
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/Filter/FilterEvaluator.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.api.function.Filter;
-
-import static java.util.Objects.requireNonNull;
-
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.QueryEvaluationException;
-import org.openrdf.query.algebra.Filter;
-import org.openrdf.query.algebra.ValueExpr;
-import org.openrdf.query.algebra.evaluation.TripleSource;
-import org.openrdf.query.algebra.evaluation.impl.EvaluationStrategyImpl;
-import org.openrdf.query.algebra.evaluation.util.QueryEvaluationUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-import info.aduna.iteration.CloseableIteration;
-
-/**
- * Processes a {@link Filter} node from a SPARQL query.
- */
-@DefaultAnnotation(NonNull.class)
-public class FilterEvaluator {
-    private static final Logger log = LoggerFactory.getLogger(FilterEvaluator.class);
-
-    /**
-     * Is used to evaluate the conditions of a {@link Filter}.
-     */
-    private static final EvaluationStrategyImpl EVALUATOR = new EvaluationStrategyImpl(
-            new TripleSource() {
-                private final ValueFactory valueFactory = new ValueFactoryImpl();
-
-                @Override
-                public ValueFactory getValueFactory() {
-                    return valueFactory;
-                }
-
-                @Override
-                public CloseableIteration<? extends Statement, QueryEvaluationException> getStatements(
-                        final Resource arg0,
-                        final URI arg1,
-                        final Value arg2,
-                        final Resource... arg3) throws QueryEvaluationException {
-                    throw new UnsupportedOperationException();
-                }
-            });
-
-    private final ValueExpr condition;
-
-    /**
-     * Constructs an instance of {@link FilterEvaluator}.
-     *
-     * @param condition - The condition that defines what passes the filter function. (not null)
-     */
-    public FilterEvaluator(final ValueExpr condition) {
-        this.condition = requireNonNull(condition);
-    }
-
-    /**
-     * Make a {@link FilterEvaluator} that processes the logic of a {@link Filter}.
-     *
-     * @param filter - Defines the Filter that will be processed. (not null)
-     * @return The {@link FilterEvaluator} for the provided {@link Filter}.
-     */
-    public static FilterEvaluator make(final Filter filter) {
-        requireNonNull(filter);
-        final ValueExpr condition = filter.getCondition();
-        return new FilterEvaluator(condition);
-    }
-
-    /**
-     * Checks to see if a {@link VisibilityBindingSet} should be included in the results or not.
-     *
-     * @param bs - The value that will be evaluated against the filter. (not null)
-     * @return {@code true} if the binding set matches the filter and it should be included in the node's results,
-     *   otherwise {@code false} and it should be excluded.
-     */
-    public boolean filter(final VisibilityBindingSet bs) {
-        requireNonNull(bs);
-
-        try {
-            final Value result = EVALUATOR.evaluate(condition, bs);
-            return QueryEvaluationUtil.getEffectiveBooleanValue(result);
-        } catch (final QueryEvaluationException e) {
-            //False returned because for whatever reason, the ValueExpr could not be evaluated.
-            //In the event that the ValueExpr is a FunctionCall, this Exception will be generated if
-            //the Function URI is a valid URI that was found in the FunctionRegistry, but the arguments
-            //for that Function could not be parsed.
-            log.error("Could not evaluate a Filter.", e);
-            return false;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/923448f1/common/rya.api.function/src/main/java/org/apache/rya/api/function/filter/FilterEvaluator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/filter/FilterEvaluator.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/filter/FilterEvaluator.java
new file mode 100644
index 0000000..3ec97cb
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/filter/FilterEvaluator.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.filter;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.algebra.Filter;
+import org.openrdf.query.algebra.ValueExpr;
+import org.openrdf.query.algebra.evaluation.TripleSource;
+import org.openrdf.query.algebra.evaluation.impl.EvaluationStrategyImpl;
+import org.openrdf.query.algebra.evaluation.util.QueryEvaluationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * Processes a {@link Filter} node from a SPARQL query.
+ */
+@DefaultAnnotation(NonNull.class)
+public class FilterEvaluator {
+    private static final Logger log = LoggerFactory.getLogger(FilterEvaluator.class);
+
+    /**
+     * Is used to evaluate the conditions of a {@link Filter}.
+     */
+    private static final EvaluationStrategyImpl EVALUATOR = new EvaluationStrategyImpl(
+            new TripleSource() {
+                private final ValueFactory valueFactory = new ValueFactoryImpl();
+
+                @Override
+                public ValueFactory getValueFactory() {
+                    return valueFactory;
+                }
+
+                @Override
+                public CloseableIteration<? extends Statement, QueryEvaluationException> getStatements(
+                        final Resource arg0,
+                        final URI arg1,
+                        final Value arg2,
+                        final Resource... arg3) throws QueryEvaluationException {
+                    throw new UnsupportedOperationException();
+                }
+            });
+
+    private final ValueExpr condition;
+
+    /**
+     * Constructs an instance of {@link FilterEvaluator}.
+     *
+     * @param condition - The condition that defines what passes the filter function. (not null)
+     */
+    public FilterEvaluator(final ValueExpr condition) {
+        this.condition = requireNonNull(condition);
+    }
+
+    /**
+     * Make a {@link FilterEvaluator} that processes the logic of a {@link Filter}.
+     *
+     * @param filter - Defines the Filter that will be processed. (not null)
+     * @return The {@link FilterEvaluator} for the provided {@link Filter}.
+     */
+    public static FilterEvaluator make(final Filter filter) {
+        requireNonNull(filter);
+        final ValueExpr condition = filter.getCondition();
+        return new FilterEvaluator(condition);
+    }
+
+    /**
+     * Checks to see if a {@link VisibilityBindingSet} should be included in the results or not.
+     *
+     * @param bs - The value that will be evaluated against the filter. (not null)
+     * @return {@code true} if the binding set matches the filter and it should be included in the node's results,
+     *   otherwise {@code false} and it should be excluded.
+     */
+    public boolean filter(final VisibilityBindingSet bs) {
+        requireNonNull(bs);
+
+        try {
+            final Value result = EVALUATOR.evaluate(condition, bs);
+            return QueryEvaluationUtil.getEffectiveBooleanValue(result);
+        } catch (final QueryEvaluationException e) {
+            //False returned because for whatever reason, the ValueExpr could not be evaluated.
+            //In the event that the ValueExpr is a FunctionCall, this Exception will be generated if
+            //the Function URI is a valid URI that was found in the FunctionRegistry, but the arguments
+            //for that Function could not be parsed.
+            log.error("Could not evaluate a Filter.", e);
+            return false;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/923448f1/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalURIs.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalURIs.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalURIs.java
index 7902a5f..3fd016a 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalURIs.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalURIs.java
@@ -1,3 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
 package org.apache.rya.api.function.temporal;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/923448f1/common/rya.api.function/src/test/java/org/apache/rya/api/function/filter/FilterEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/test/java/org/apache/rya/api/function/filter/FilterEvaluatorTest.java b/common/rya.api.function/src/test/java/org/apache/rya/api/function/filter/FilterEvaluatorTest.java
index 8bbf005..5e5cbe6 100644
--- a/common/rya.api.function/src/test/java/org/apache/rya/api/function/filter/FilterEvaluatorTest.java
+++ b/common/rya.api.function/src/test/java/org/apache/rya/api/function/filter/FilterEvaluatorTest.java
@@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.rya.api.function.Filter.FilterEvaluator;
+import org.apache.rya.api.function.filter.FilterEvaluator;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/923448f1/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorSupplier.java
index 5997237..2f83aa9 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorSupplier.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorSupplier.java
@@ -22,7 +22,7 @@ import static java.util.Objects.requireNonNull;
 
 import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
-import org.apache.rya.api.function.Filter.FilterEvaluator;
+import org.apache.rya.api.function.filter.FilterEvaluator;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.kafka.processors.ProcessorResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.ResultType;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/923448f1/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
index 4046e23..68fbb83 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
@@ -38,7 +38,7 @@ import org.apache.kafka.streams.processor.ProcessorSupplier;
 import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.kafka.streams.state.Stores;
-import org.apache.rya.api.function.Filter.FilterEvaluator;
+import org.apache.rya.api.function.filter.FilterEvaluator;
 import org.apache.rya.api.function.join.IterativeJoin;
 import org.apache.rya.api.function.join.LeftOuterJoin;
 import org.apache.rya.api.function.join.NaturalJoin;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/923448f1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
index 4c44bae..3ff8e8d 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
@@ -24,7 +24,7 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
 import org.apache.kafka.streams.processor.ProcessorContext;
-import org.apache.rya.api.function.Filter.FilterEvaluator;
+import org.apache.rya.api.function.filter.FilterEvaluator;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.kafka.RdfTestUtil;
 import org.apache.rya.streams.kafka.processors.ProcessorResult;


[36/50] [abbrv] incubator-rya git commit: RYA-377 Temporal Within Function

Posted by ca...@apache.org.
RYA-377 Temporal Within Function


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

Branch: refs/heads/master
Commit: 5e5d7c8ec2279b00ab7090e2214c333c0300ec88
Parents: cf43360
Author: Andrew Smith <sm...@gmail.com>
Authored: Fri Dec 1 14:52:00 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../function/temporal/AfterTemporalInstant.java |  6 +-
 .../temporal/BeforeTemporalInstant.java         |  6 +-
 .../api/function/temporal/EqualsTemporal.java   | 49 -----------
 .../temporal/EqualsTemporalInstant.java         | 44 ++++++++++
 .../TemporalInstantRelationFunction.java        | 63 ++++++++++++++
 .../TemporalIntervalRelationFunction.java       | 70 ++++++++++++++++
 .../temporal/TemporalRelationFunction.java      | 65 ---------------
 .../rya/api/function/temporal/TemporalURIs.java | 23 +++++
 .../temporal/WithinTemporalInterval.java        | 45 ++++++++++
 ...f.query.algebra.evaluation.function.Function |  3 +-
 .../temporal/EqualsTemporalFunctionsTest.java   |  6 +-
 .../temporal/WithinTemporalFunctionsTest.java   | 88 ++++++++++++++++++++
 .../processors/filter/TemporalFilterIT.java     | 74 +++++++++++-----
 13 files changed, 396 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/AfterTemporalInstant.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/AfterTemporalInstant.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/AfterTemporalInstant.java
index 38c02ba..fe58e37 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/AfterTemporalInstant.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/AfterTemporalInstant.java
@@ -29,12 +29,10 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * Filter function in a SPARQL query used to filter when a point of time is after another.
  */
 @DefaultAnnotation(NonNull.class)
-public class AfterTemporalInstant extends TemporalRelationFunction {
-    public static final String URI = BASE_URI + "after";
-
+public class AfterTemporalInstant extends TemporalInstantRelationFunction {
     @Override
     public String getURI() {
-        return URI;
+        return TemporalURIs.AFTER;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/BeforeTemporalInstant.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/BeforeTemporalInstant.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/BeforeTemporalInstant.java
index c9f31c3..4cb12e0 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/BeforeTemporalInstant.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/BeforeTemporalInstant.java
@@ -29,12 +29,10 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * Filter function in a SPARQL query used to filter when a point of time is before another.
  */
 @DefaultAnnotation(NonNull.class)
-public class BeforeTemporalInstant extends TemporalRelationFunction {
-    public static final String URI = BASE_URI + "before";
-
+public class BeforeTemporalInstant extends TemporalInstantRelationFunction {
     @Override
     public String getURI() {
-        return URI;
+        return TemporalURIs.BEFORE;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
deleted file mode 100644
index 07391c8..0000000
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.rya.api.function.temporal;
-
-import java.time.ZonedDateTime;
-import java.util.Objects;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-
-/**
- * Filter function in a SPARQL query used to filter equality over time.
- */
-@DefaultAnnotation(NonNull.class)
-public class EqualsTemporal extends TemporalRelationFunction {
-    private static final String URI = BASE_URI + "equals";
-
-    @Override
-    public String getURI() {
-        return URI;
-    }
-
-    @Override
-    protected boolean relation(final ZonedDateTime d1, final ZonedDateTime d2) {
-        Objects.requireNonNull(d1);
-        Objects.requireNonNull(d2);
-        return d1.isEqual(d2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporalInstant.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporalInstant.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporalInstant.java
new file mode 100644
index 0000000..10fe40b
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporalInstant.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rya.api.function.temporal;
+
+import java.time.ZonedDateTime;
+import java.util.Objects;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Filter function in a SPARQL query used to filter equality over time.
+ */
+@DefaultAnnotation(NonNull.class)
+public class EqualsTemporalInstant extends TemporalInstantRelationFunction {
+    @Override
+    public String getURI() {
+        return TemporalURIs.EQUALS;
+    }
+
+    @Override
+    protected boolean relation(final ZonedDateTime d1, final ZonedDateTime d2) {
+        Objects.requireNonNull(d1);
+        Objects.requireNonNull(d2);
+        return d1.isEqual(d2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalInstantRelationFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalInstantRelationFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalInstantRelationFunction.java
new file mode 100644
index 0000000..d622329
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalInstantRelationFunction.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.temporal;
+
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeParseException;
+
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.query.algebra.evaluation.ValueExprEvaluationException;
+import org.openrdf.query.algebra.evaluation.function.Function;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Function for comparing 2 {@link ZonedDateTime} objects in a SPARQL filter.
+ */
+@DefaultAnnotation(NonNull.class)
+abstract class TemporalInstantRelationFunction implements Function {
+    @Override
+    public Value evaluate(final ValueFactory valueFactory, final Value... args) throws ValueExprEvaluationException {
+        if (args.length != 2) {
+            throw new ValueExprEvaluationException(getURI() + " requires exactly 2 arguments, got " + args.length);
+        }
+
+        try {
+            final ZonedDateTime date1 = ZonedDateTime.parse(args[0].stringValue());
+            final ZonedDateTime date2 = ZonedDateTime.parse(args[1].stringValue());
+            final boolean result = relation(date1, date2);
+
+            return valueFactory.createLiteral(result);
+        } catch (final DateTimeParseException e) {
+            throw new ValueExprEvaluationException("Date/Times provided must be of the ISO-8601 format. Example: 2007-04-05T14:30Z");
+        }
+    }
+
+    /**
+     * The comparison function to perform between 2 {@link ZonedDateTime}
+     * objects.
+     *
+     * @param d1 first {@link ZonedDateTime} to compare. (not null)
+     * @param d2 second {@link ZonedDateTime} to compare. (not null)
+     * @return The result of the comparison between {@link ZonedDateTime}s.
+     */
+    protected abstract boolean relation(ZonedDateTime d1, ZonedDateTime d2);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalIntervalRelationFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalIntervalRelationFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalIntervalRelationFunction.java
new file mode 100644
index 0000000..4a0d4bd
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalIntervalRelationFunction.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.temporal;
+
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeParseException;
+
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.query.algebra.evaluation.ValueExprEvaluationException;
+import org.openrdf.query.algebra.evaluation.function.Function;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Function for comparing a {@link ZonedDateTime} with an interval of {@link ZonedDateTime} objects in a SPARQL filter.
+ */
+@DefaultAnnotation(NonNull.class)
+abstract class TemporalIntervalRelationFunction implements Function {
+    @Override
+    public Value evaluate(final ValueFactory valueFactory, final Value... args) throws ValueExprEvaluationException {
+        if (args.length != 2) {
+            throw new ValueExprEvaluationException(getURI() + " requires exactly 2 arguments, got " + args.length);
+        }
+
+        final String[] strInterval = args[1].stringValue().split("/");
+        if (strInterval.length != 2) {
+            throw new ValueExprEvaluationException(getURI() + " requires the second argument: " + args[1] + " to be 2 dates seperated by a \'/\'");
+        }
+        try {
+            final ZonedDateTime date1 = ZonedDateTime.parse(args[0].stringValue());
+            final ZonedDateTime[] interval = new ZonedDateTime[] {
+                    ZonedDateTime.parse(strInterval[0]),
+                    ZonedDateTime.parse(strInterval[1])
+            };
+            final boolean result = relation(date1, interval);
+
+            return valueFactory.createLiteral(result);
+        } catch (final DateTimeParseException e) {
+            throw new ValueExprEvaluationException("Date/Times provided must be of the ISO-8601 format. Example: 2007-04-05T14:30Z");
+        }
+    }
+
+    /**
+     * The comparison function to perform between 2 {@link ZonedDateTime}
+     * objects.
+     *
+     * @param d1 first {@link ZonedDateTime} to compare. (not null)
+     * @param interval The interval represented by 2 {@link ZonedDateTime}s. (not null)
+     * @return The result of the comparison between {@link ZonedDateTime}s.
+     */
+    protected abstract boolean relation(ZonedDateTime d1, ZonedDateTime[] interval);
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalRelationFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalRelationFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalRelationFunction.java
deleted file mode 100644
index 02710d9..0000000
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalRelationFunction.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.api.function.temporal;
-
-import java.time.ZonedDateTime;
-import java.time.format.DateTimeParseException;
-
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.query.algebra.evaluation.ValueExprEvaluationException;
-import org.openrdf.query.algebra.evaluation.function.Function;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-
-/**
- * Function for comparing 2 {@link ZonedDateTime} objects in a SPARQL filter.
- */
-@DefaultAnnotation(NonNull.class)
-abstract class TemporalRelationFunction implements Function {
-    public static final String BASE_URI = "http://rya.apache.org/ns/temporal/";
-
-    @Override
-    public Value evaluate(final ValueFactory valueFactory, final Value... args) throws ValueExprEvaluationException {
-        if (args.length != 2) {
-            throw new ValueExprEvaluationException(getURI() + " requires exactly 2 arguments, got " + args.length);
-        }
-
-        try {
-            final ZonedDateTime date1 = ZonedDateTime.parse(args[0].stringValue());
-            final ZonedDateTime date2 = ZonedDateTime.parse(args[1].stringValue());
-            final boolean result = relation(date1, date2);
-
-            return valueFactory.createLiteral(result);
-        } catch (final DateTimeParseException e) {
-            throw new ValueExprEvaluationException("Date/Times provided must be of the ISO-8601 format. Example: 2007-04-05T14:30Z");
-        }
-    }
-
-    /**
-     * The comparison function to perform between 2 {@link ZonedDateTime}
-     * objects.
-     *
-     * @param d1 first {@link ZonedDateTime} to compare. (not null)
-     * @param d2 second {@link ZonedDateTime} to compare. (not null)
-     * @return The result of the comparison between {@link ZonedDateTime}s.
-     */
-    protected abstract boolean relation(ZonedDateTime d1, ZonedDateTime d2);
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalURIs.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalURIs.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalURIs.java
new file mode 100644
index 0000000..7902a5f
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/TemporalURIs.java
@@ -0,0 +1,23 @@
+package org.apache.rya.api.function.temporal;
+
+/**
+ * Constants for the Temporal Functions used in rya.
+ */
+public class TemporalURIs {
+    /**
+     * All temporal functions have the namespace (<tt>http://rya.apache.org/ns/temporal#</tt>).
+     */
+    public static final String NAMESPACE = "http://rya.apache.org/ns/temporal#";
+
+    /** <tt>http://rya.apache.org/ns/temporal#equals</tt> */
+    public final static String EQUALS = NAMESPACE + "equals";
+
+    /** <tt>http://rya.apache.org/ns/temporal#before</tt> */
+    public final static String BEFORE = NAMESPACE + "before";
+
+    /** <tt>http://rya.apache.org/ns/temporal#after</tt> */
+    public final static String AFTER = NAMESPACE + "after";
+
+    /** <tt>http://rya.apache.org/ns/temporal#within</tt> */
+    public final static String WITHIN = NAMESPACE + "within";
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/WithinTemporalInterval.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/WithinTemporalInterval.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/WithinTemporalInterval.java
new file mode 100644
index 0000000..5fc3e03
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/WithinTemporalInterval.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rya.api.function.temporal;
+
+import java.time.ZonedDateTime;
+import java.util.Objects;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Filter function in a SPARQL query used to filter when a point of time is
+ * within an interval.
+ */
+@DefaultAnnotation(NonNull.class)
+public class WithinTemporalInterval extends TemporalIntervalRelationFunction {
+    @Override
+    public String getURI() {
+        return TemporalURIs.WITHIN;
+    }
+
+    @Override
+    protected boolean relation(final ZonedDateTime d1, final ZonedDateTime[] interval) {
+        Objects.requireNonNull(d1);
+        Objects.requireNonNull(interval);
+        return d1.isAfter(interval[0]) && d1.isBefore(interval[1]);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function b/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
index 2ec01d5..1172844 100644
--- a/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
+++ b/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
@@ -16,6 +16,7 @@
 # specific language governing permissions and limitations
 # under the License.
 #
-org.apache.rya.api.function.temporal.EqualsTemporal
+org.apache.rya.api.function.temporal.EqualsTemporalInstant
 org.apache.rya.api.function.temporal.BeforeTemporalInstant
 org.apache.rya.api.function.temporal.AfterTemporalInstant
+org.apache.rya.api.function.temporal.WithinTemporalInterval

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/EqualsTemporalFunctionsTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/EqualsTemporalFunctionsTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/EqualsTemporalFunctionsTest.java
index f32904a..0d14d48 100644
--- a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/EqualsTemporalFunctionsTest.java
+++ b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/EqualsTemporalFunctionsTest.java
@@ -36,7 +36,7 @@ public class EqualsTemporalFunctionsTest {
 
     @Test
     public void testEquals_equal() throws Exception {
-        final EqualsTemporal function = new EqualsTemporal();
+        final EqualsTemporalInstant function = new EqualsTemporalInstant();
 
         // 2 times equal
         final Value[] args = new Value[2];
@@ -49,7 +49,7 @@ public class EqualsTemporalFunctionsTest {
 
     @Test
     public void testEquals_before() throws Exception {
-        final EqualsTemporal function = new EqualsTemporal();
+        final EqualsTemporalInstant function = new EqualsTemporalInstant();
 
         // first time is before
         final Value[] args = new Value[2];
@@ -62,7 +62,7 @@ public class EqualsTemporalFunctionsTest {
 
     @Test
     public void testEquals_after() throws Exception {
-        final EqualsTemporal function = new EqualsTemporal();
+        final EqualsTemporalInstant function = new EqualsTemporalInstant();
 
         // first time is after
         final Value[] args = new Value[2];

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/WithinTemporalFunctionsTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/WithinTemporalFunctionsTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/WithinTemporalFunctionsTest.java
new file mode 100644
index 0000000..6dee711
--- /dev/null
+++ b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/WithinTemporalFunctionsTest.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.temporal;
+
+import static org.junit.Assert.assertEquals;
+
+import java.time.ZonedDateTime;
+
+import org.junit.Test;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.evaluation.ValueExprEvaluationException;
+
+public class WithinTemporalFunctionsTest {
+    private static final ZonedDateTime TIME = ZonedDateTime.parse("2015-12-30T12:00:00Z");
+    private static final ZonedDateTime TIME_10 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
+    private static final ZonedDateTime TIME_20 = ZonedDateTime.parse("2015-12-30T12:00:20Z");
+
+    final ValueFactory VF = ValueFactoryImpl.getInstance();
+
+    @Test(expected = ValueExprEvaluationException.class)
+    public void within_NotInterval() throws Exception {
+        // correct date formats are ensured through other tests
+        final WithinTemporalInterval function = new WithinTemporalInterval();
+
+        // 2 dates are provided
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME.toString());
+        function.evaluate(VF, args);
+    }
+
+    @Test
+    public void testWithin_beginning() throws Exception {
+        final WithinTemporalInterval function = new WithinTemporalInterval();
+
+        // 2 times equal
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME.toString() + "/" + TIME_20.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+
+    @Test
+    public void testWithin_within() throws Exception {
+        final WithinTemporalInterval function = new WithinTemporalInterval();
+
+        // first time is before
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME_10.toString());
+        args[1] = VF.createLiteral(TIME.toString() + "/" + TIME_20.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(true), rez);
+    }
+
+    @Test
+    public void testWithin_end() throws Exception {
+        final WithinTemporalInterval function = new WithinTemporalInterval();
+
+        // first time is after
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME_20.toString());
+        args[1] = VF.createLiteral(TIME.toString() + "/" + TIME_20.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/5e5d7c8e/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
index 0a2185d..22a883b 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
@@ -74,8 +74,8 @@ public class TemporalFilterIT {
             }
         }
 
-        // There are 3 temporal functions registered, ensure that there are 3.
-        assertEquals(3, count);
+        // There are 4 temporal functions registered, ensure that there are 4.
+        assertEquals(4, count);
     }
 
     @Test
@@ -89,12 +89,12 @@ public class TemporalFilterIT {
         // Get the RDF model objects that will be used to build the query.
         final String sparql =
                 "PREFIX time: <http://www.w3.org/2006/time/> \n"
-            + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
-            + "SELECT * \n"
-            + "WHERE { \n"
-            + "  <urn:time> time:atTime ?date .\n"
-            + " FILTER(tempf:equals(?date, \"" + TIME.toString() + "\")) "
-            + "}";
+                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:time> time:atTime ?date .\n"
+                        + " FILTER(tempf:equals(?date, \"" + TIME.toString() + "\")) "
+                        + "}";
         // Setup a topology.
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
@@ -123,12 +123,12 @@ public class TemporalFilterIT {
         // Get the RDF model objects that will be used to build the query.
         final String sparql =
                 "PREFIX time: <http://www.w3.org/2006/time/> \n"
-            + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
-            + "SELECT * \n"
-            + "WHERE { \n"
-            + "  <urn:time> time:atTime ?date .\n"
-            + " FILTER(tempf:before(?date, \"" + TIME_10.toString() + "\")) "
-            + "}";
+                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:time> time:atTime ?date .\n"
+                        + " FILTER(tempf:before(?date, \"" + TIME_10.toString() + "\")) "
+                        + "}";
         // Setup a topology.
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
@@ -157,12 +157,12 @@ public class TemporalFilterIT {
         // Get the RDF model objects that will be used to build the query.
         final String sparql =
                 "PREFIX time: <http://www.w3.org/2006/time/> \n"
-            + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
-            + "SELECT * \n"
-            + "WHERE { \n"
-            + "  <urn:time> time:atTime ?date .\n"
-            + " FILTER(tempf:after(?date, \"" + TIME_10.toString() + "\")) "
-            + "}";
+                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:time> time:atTime ?date .\n"
+                        + " FILTER(tempf:after(?date, \"" + TIME_10.toString() + "\")) "
+                        + "}";
         // Setup a topology.
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
@@ -180,6 +180,40 @@ public class TemporalFilterIT {
         RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
+    @Test
+    public void showWithinWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX time: <http://www.w3.org/2006/time/> \n"
+                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:time> time:atTime ?date .\n"
+                        + " FILTER(tempf:within(?date, \"" + TIME.toString() + "/" + TIME_20.toString() + "\")) "
+                        + "}";
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("date", vf.createLiteral(TIME_10.toString()));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
     private List<VisibilityStatement> getStatements() throws Exception {
         final List<VisibilityStatement> statements = new ArrayList<>();
         statements.add(new VisibilityStatement(statement(TIME), "a"));


[10/50] [abbrv] incubator-rya git commit: RYA-377 ProjectionProcessor

Posted by ca...@apache.org.
RYA-377 ProjectionProcessor


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

Branch: refs/heads/master
Commit: 98af7aa7fd22784855a1f45b15b342db3f44eaea
Parents: 4acbe5a
Author: Andrew Smith <sm...@gmail.com>
Authored: Mon Nov 13 14:51:28 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../projection/ProjectionProcessorSupplier.java | 128 ++++++++++++++++
 .../kafka/processors/ProjectionProcessorIT.java | 152 +++++++++++++++++++
 2 files changed, 280 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/98af7aa7/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorSupplier.java
new file mode 100644
index 0000000..67a777f
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorSupplier.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.projection;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.ResultType;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResultFactory;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessorSupplier;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.algebra.ProjectionElem;
+import org.openrdf.query.algebra.ProjectionElemList;
+import org.openrdf.query.impl.MapBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Supplies {@link ProjectionProcessor} instances.
+ */
+@DefaultAnnotation(NonNull.class)
+public class ProjectionProcessorSupplier extends RyaStreamsProcessorSupplier {
+
+    private final ProjectionElemList projectionElems;
+
+    /**
+     * Constructs an instance of {@link ProjectionProcessorSupplier}.
+     *
+     * @param projectionElems - The {@link ProjectionElemList} that defines which bindings get forwarded or changed. (not null)
+     * @param resultFactory - The factory that the supplied processors will use to create results. (not null)
+     */
+    public ProjectionProcessorSupplier(
+            final ProjectionElemList projectionElems,
+            final ProcessorResultFactory resultFactory) {
+        super(resultFactory);
+        this.projectionElems = requireNonNull(projectionElems);
+    }
+
+    @Override
+    public Processor<Object, ProcessorResult> get() {
+        return new ProjectionProcessor(projectionElems, super.getResultFactory());
+    }
+
+    /**
+     * Evaluates {@link ProcessorResult}s against a {@link Projection}.  Any results found in
+     * the {@link ProjectionElemList} will be modified and/or forwarded.  A {@link ProjectionElemList} defines
+     * a source and target name for a binding, so if a binding name appears in the source list of the {@link ProjectionElemList},
+     * then it will be renamed with the associated target name.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static final class ProjectionProcessor implements Processor<Object, ProcessorResult> {
+
+        private final ProjectionElemList projectionElems;
+        private final ProcessorResultFactory resultFactory;
+
+        private ProcessorContext context;
+
+        /**
+         * Constructs an instance of {@link ProjectionProcessor}.
+         *
+         * @param projectionElems - The projection elems that will determine what to do with the bindings. (not null)
+         * @param resultFactory - The factory that the processor will use to create results. (not null)
+         */
+        public ProjectionProcessor(final ProjectionElemList projectionElems, final ProcessorResultFactory resultFactory) {
+            this.projectionElems = requireNonNull(projectionElems);
+            this.resultFactory = requireNonNull(resultFactory);
+        }
+
+        @Override
+        public void init(final ProcessorContext context) {
+            this.context = context;
+        }
+
+        @Override
+        public void process(final Object key, final ProcessorResult result) {
+            // projections can only be unary
+            if (result.getType() != ResultType.UNARY) {
+                throw new RuntimeException("The ProcessorResult to be processed must be Unary.");
+            }
+
+            final UnaryResult unary = result.getUnary();
+            final VisibilityBindingSet bindingSet = unary.getResult();
+
+            final MapBindingSet newBindingSet = new MapBindingSet(bindingSet.size());
+            for (final ProjectionElem elem : projectionElems.getElements()) {
+                if (bindingSet.hasBinding(elem.getSourceName())) {
+                    newBindingSet.addBinding(elem.getTargetName(), bindingSet.getValue(elem.getSourceName()));
+                }
+            }
+
+            // wrap the new binding set with the original's visibility.
+            final VisibilityBindingSet newVisiSet = new VisibilityBindingSet(newBindingSet, bindingSet.getVisibility());
+            final ProcessorResult resultValue = resultFactory.make(newVisiSet);
+            context.forward(key, resultValue);
+        }
+
+        @Override
+        public void punctuate(final long timestamp) {
+            // Nothing to do.
+        }
+
+        @Override
+        public void close() {
+            // Nothing to do.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/98af7aa7/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
new file mode 100644
index 0000000..53d1765
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RdfTestUtil;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.RyaStreamsSinkFormatterSupplier.RyaStreamsSinkFormatter;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
+import org.apache.rya.streams.kafka.processors.projection.ProjectionProcessorSupplier;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.ProjectionElem;
+import org.openrdf.query.algebra.ProjectionElemList;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.evaluation.QueryBindingSet;
+
+import com.google.common.collect.Sets;
+
+/**
+ * Integration tests the methods of {@link StatementPatternProcessor}.
+ */
+public class ProjectionProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void projection_renameOne() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPattern object that will be evaluated.
+        final StatementPattern sp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
+
+        // Add a processor that handles the projection.
+        final ProjectionElemList elems = new ProjectionElemList();
+        elems.addElement(new ProjectionElem("otherPerson", "dog"));
+        elems.addElement(new ProjectionElem("person", "person"));
+        builder.addProcessor("P1", new ProjectionProcessorSupplier(elems, result -> ProcessorResult.make(new UnaryResult(result))), "SP1");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "P1");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Load some data into the input topic.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Sparky")), "a") );
+
+        // Show the correct binding set results from the job.
+        final QueryBindingSet bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("dog", vf.createURI("urn:Sparky"));
+        final VisibilityBindingSet binding = new VisibilityBindingSet(bs, "a");
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        expected.add(binding);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected));
+    }
+
+    @Test
+    public void projection_keepOneDropOne() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the StatementPattern object that will be evaluated.
+        final StatementPattern sp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        // The topic that Statements are written to is used as a source.
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+
+        // Add a processor that handles the first statement pattern.
+        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
+
+        // Add a processor that handles the projection.
+        final ProjectionElemList elems = new ProjectionElemList();
+        elems.addElement(new ProjectionElem("otherPerson", "otherPerson"));
+        builder.addProcessor("P1", new ProjectionProcessorSupplier(elems, result -> ProcessorResult.make(new UnaryResult(result))), "SP1");
+
+        // Add a processor that formats the VisibilityBindingSet for output.
+        builder.addProcessor("SINK_FORMATTER", RyaStreamsSinkFormatter::new, "P1");
+
+        // Add a sink that writes the data out to a new Kafka topic.
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Load some data into the input topic.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+
+        // Show the correct binding set results from the job.
+        final QueryBindingSet bs = new QueryBindingSet();
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        final VisibilityBindingSet binding = new VisibilityBindingSet(bs, "a");
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        expected.add(binding);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected));
+    }
+}
\ No newline at end of file


[40/50] [abbrv] incubator-rya git commit: RYA-377 Added Construct query support to Rya Streams.

Posted by ca...@apache.org.
RYA-377 Added Construct query support to Rya Streams.


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

Branch: refs/heads/master
Commit: da63fd125e16779df3536b4172fa66e36561e4ff
Parents: 538393f
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Nov 21 18:49:13 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../projection/MultiProjectionEvaluator.java    |   4 +-
 .../projection/ProjectionEvaluator.java         |   5 +-
 .../kafka/topology/TopologyBuilderFactory.java  |  40 ++-
 .../streams/kafka/topology/TopologyFactory.java | 178 +++++++++++---
 .../apache/rya/streams/kafka/KafkaTestUtil.java |  18 +-
 .../processors/StatementPatternProcessorIT.java |  18 +-
 .../kafka/processors/join/JoinProcessorIT.java  | 243 ++++---------------
 .../projection/MultiProjectionProcessorIT.java  |  87 +------
 .../projection/ProjectionProcessorIT.java       |  42 +---
 .../kafka/topology/TopologyFactoryTest.java     |  32 ++-
 10 files changed, 280 insertions(+), 387 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/MultiProjectionEvaluator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/MultiProjectionEvaluator.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/MultiProjectionEvaluator.java
index e2b7046..0e9093d 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/MultiProjectionEvaluator.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/MultiProjectionEvaluator.java
@@ -57,9 +57,9 @@ public class MultiProjectionEvaluator {
     /**
      * Constructs an instance of {@link MultiProjection}.
      *
-     * @param projections - The {@link ProjectionEvaluators} that handle each projection within the multi. (not null)
+     * @param projections - The {@link ProjectionEvaluators} that handle each projection within the MultiProjection. (not null)
      * @param blankNodeSourceNames - If there are blank nodes in the projection, this is a set of their names
-     *   so that they may be re-label to have the same node IDs. (not null)
+     *   so that they may be re-labeled to have the same node IDs. (not null)
      * @param bNodeIdFactory - Creates the IDs for Blank Nodes. (not null)
      */
     public MultiProjectionEvaluator(

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/ProjectionEvaluator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/ProjectionEvaluator.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/ProjectionEvaluator.java
index a0b59c1..4b37448 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/ProjectionEvaluator.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/ProjectionEvaluator.java
@@ -179,7 +179,10 @@ public class ProjectionEvaluator {
                 }
             }
 
-            result.addBinding(elem.getTargetName(), value);
+            // Only add the value if there is one. There may not be one if a binding is optional.
+            if(value != null) {
+                result.addBinding(elem.getTargetName(), value);
+            }
         }
 
         return new VisibilityBindingSet(result, bs.getVisibility());

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyBuilderFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyBuilderFactory.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyBuilderFactory.java
index 9e9dd92..666cbb0 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyBuilderFactory.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyBuilderFactory.java
@@ -19,8 +19,8 @@
 package org.apache.rya.streams.kafka.topology;
 
 import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.BNodeIdFactory;
 import org.openrdf.query.MalformedQueryException;
-import org.openrdf.query.algebra.TupleExpr;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -32,17 +32,33 @@ import edu.umd.cs.findbugs.annotations.NonNull;
 public interface TopologyBuilderFactory {
 
     /**
-     * Builds a {@link TopologyBuilder} based on the provided sparql query where
-     * each {@link TupleExpr} in the parsed query is a processor in the
-     * topology.
+     * Builds a {@link TopologyBuilder} based on the provided SPARQL query that
+     * pulls from {@code statementsTopic} for input and writes the query's results
+     * to {@code resultsTopic}.
      *
      * @param sparqlQuery - The SPARQL query to build a topology for. (not null)
-     * @param statementTopic - The topic for the source to read from. (not null)
-     * @param statementTopic - The topic for the sink to write to. (not null)
-     * @return - The created {@link TopologyBuilder}.
-     * @throws MalformedQueryException - The provided query is not a valid
-     *         SPARQL query.
+     * @param statementsTopic - The topic for the source to read from. (not null)
+     * @param resultsTopic - The topic for the sink to write to. (not null)
+     * @param bNodeIdFactory - A factory that generates Blank Node IDs if any are required. (not null)
+     * @return The created {@link TopologyBuilder}.
+     * @throws MalformedQueryException - The provided query is not a valid SPARQL query.
+     * @throws TopologyBuilderException - A problem occurred while constructing the topology.
      */
-    public TopologyBuilder build(final String sparqlQuery, final String statementTopic, final String resultTopic)
-            throws Exception;
-}
+    public TopologyBuilder build(
+            final String sparqlQuery,
+            final String statementsTopic,
+            final String resultsTopic,
+            final BNodeIdFactory bNodeIdFactory) throws MalformedQueryException, TopologyBuilderException;
+
+    /**
+     * An Exception thrown when a problem occurs when constructing the processor
+     * topology in the {@link TopologyFactory}.
+     */
+    public static class TopologyBuilderException extends Exception {
+        private static final long serialVersionUID = 1L;
+
+        public TopologyBuilderException(final String message, final Throwable cause) {
+            super(message, cause);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
index 782a58b..08f3625 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
@@ -30,6 +30,7 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 
+import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.streams.processor.Processor;
@@ -40,6 +41,9 @@ import org.apache.kafka.streams.state.Stores;
 import org.apache.rya.api.function.join.IterativeJoin;
 import org.apache.rya.api.function.join.LeftOuterJoin;
 import org.apache.rya.api.function.join.NaturalJoin;
+import org.apache.rya.api.function.projection.BNodeIdFactory;
+import org.apache.rya.api.function.projection.MultiProjectionEvaluator;
+import org.apache.rya.api.function.projection.ProjectionEvaluator;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.kafka.processors.ProcessorResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
@@ -48,17 +52,22 @@ import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier;
+import org.apache.rya.streams.kafka.processors.output.StatementOutputFormatterSupplier;
+import org.apache.rya.streams.kafka.processors.projection.MultiProjectionProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.projection.ProjectionProcessorSupplier;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerde;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
 import org.openrdf.query.MalformedQueryException;
 import org.openrdf.query.algebra.BinaryTupleOperator;
 import org.openrdf.query.algebra.Extension;
 import org.openrdf.query.algebra.Join;
 import org.openrdf.query.algebra.LeftJoin;
+import org.openrdf.query.algebra.MultiProjection;
 import org.openrdf.query.algebra.Projection;
 import org.openrdf.query.algebra.QueryModelNode;
+import org.openrdf.query.algebra.Reduced;
 import org.openrdf.query.algebra.StatementPattern;
 import org.openrdf.query.algebra.TupleExpr;
 import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
@@ -85,34 +94,28 @@ public class TopologyFactory implements TopologyBuilderFactory {
 
     private List<ProcessorEntry> processorEntryList;
 
-    /**
-     * Builds a {@link TopologyBuilder} based on the provided sparql query.
-     *
-     * @param sparqlQuery - The SPARQL query to build a topology for. (not null)
-     * @param statementTopic - The topic for the source to read from. (not null)
-     * @param resultTopic - The topic for the sink to write to. (not null)
-     * @return - The created {@link TopologyBuilder}.
-     * @throws MalformedQueryException - The provided query is not a valid SPARQL query.
-     * @throws TopologyBuilderException - A problem occurred while constructing the topology.
-     */
     @Override
-    public TopologyBuilder build(final String sparqlQuery, final String statementTopic, final String resultTopic)
+    public TopologyBuilder build(
+            final String sparqlQuery,
+            final String statementsTopic,
+            final String resultsTopic,
+            final BNodeIdFactory bNodeIdFactory)
             throws MalformedQueryException, TopologyBuilderException {
         requireNonNull(sparqlQuery);
-        requireNonNull(statementTopic);
-        requireNonNull(resultTopic);
+        requireNonNull(statementsTopic);
+        requireNonNull(resultsTopic);
 
         final ParsedQuery parsedQuery = new SPARQLParser().parseQuery(sparqlQuery, null);
         final TopologyBuilder builder = new TopologyBuilder();
 
         final TupleExpr expr = parsedQuery.getTupleExpr();
-        final QueryVisitor visitor = new QueryVisitor();
+        final QueryVisitor visitor = new QueryVisitor(bNodeIdFactory);
         expr.visit(visitor);
 
         processorEntryList = visitor.getProcessorEntryList();
         final Map<TupleExpr, String> idMap = visitor.getIDs();
         // add source node
-        builder.addSource(SOURCE, new StringDeserializer(), new VisibilityStatementDeserializer(), statementTopic);
+        builder.addSource(SOURCE, new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
 
         // processing the processor entry list in reverse order means we go from leaf
         // nodes -> parent nodes.
@@ -146,11 +149,12 @@ public class TopologyFactory implements TopologyBuilderFactory {
             }
         }
 
-        // convert processing results to visibility binding sets
-        builder.addProcessor("OUTPUT_FORMATTER", new BindingSetOutputFormatterSupplier(), entry.getID());
+        // Add a formatter that converts the ProcessorResults into the output format.
+        final SinkEntry<?,?> sinkEntry = visitor.getSinkEntry();
+        builder.addProcessor("OUTPUT_FORMATTER", sinkEntry.getFormatterSupplier(), entry.getID());
 
-        // add sink
-        builder.addSink(SINK, resultTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "OUTPUT_FORMATTER");
+        // Add the sink.
+        builder.addSink(SINK, resultsTopic, sinkEntry.getKeySerializer(), sinkEntry.getValueSerializer(), "OUTPUT_FORMATTER");
 
         return builder;
     }
@@ -264,16 +268,82 @@ public class TopologyFactory implements TopologyBuilderFactory {
     }
 
     /**
+     * Information about how key/value pairs need to be written to the sink.
+     *
+     * @param <K> - The type of Key that the sink uses.
+     * @param <V> - The type of Value that the sink uses.
+     */
+    private final static class SinkEntry<K, V> {
+
+        private final ProcessorSupplier<Object, ProcessorResult> formatterSupplier;
+        private final Serializer<K> keySerializer;
+        private final Serializer<V> valueSerializer;
+
+        /**
+         * Constructs an instance of {@link SinkEntry}.
+         *
+         * @param formatterSupplier - Formats {@link ProcessingResult}s for output to the sink. (not null)
+         * @param keySerializer - Serializes keys that are used to write to the sink. (not null)
+         * @param valueSerializer - Serializes values that are used to write to the sink. (not null)
+         */
+        public SinkEntry(
+                final ProcessorSupplier<Object, ProcessorResult> formatterSupplier,
+                final Serializer<K> keySerializer,
+                final Serializer<V> valueSerializer) {
+            this.keySerializer = requireNonNull(keySerializer);
+            this.valueSerializer = requireNonNull(valueSerializer);
+            this.formatterSupplier = requireNonNull(formatterSupplier);
+        }
+
+        /**
+         * @return Formats {@link ProcessingResult}s for output to the sink.
+         */
+        public ProcessorSupplier<Object, ProcessorResult> getFormatterSupplier() {
+            return formatterSupplier;
+        }
+
+        /**
+         * @return Serializes keys that are used to write to the sink.
+         */
+        public Serializer<K> getKeySerializer() {
+            return keySerializer;
+        }
+
+        /**
+         * @return Serializes values that are used to write to the sink.
+         */
+        public Serializer<V> getValueSerializer() {
+            return valueSerializer;
+        }
+    }
+
+    /**
      * Visits each node in a {@link TupleExpr} and creates a
      * {@link ProcessorSupplier} and meta information needed for creating a
      * {@link TopologyBuilder}.
      */
     final static class QueryVisitor extends QueryModelVisitorBase<TopologyBuilderException> {
-        // Each node needs a ProcessorEntry to be a processor node in the
-        // TopologyBuilder.
+        // Each node needs a ProcessorEntry to be a processor node in the TopologyBuilder.
         private final List<ProcessorEntry> entries = new ArrayList<>();
         private final Map<TupleExpr, String> idMap = new HashMap<>();
 
+        // Default to a Binding Set outputting sink entry.
+        private SinkEntry<?, ?> sinkEntry = new SinkEntry<>(
+                new BindingSetOutputFormatterSupplier(),
+                new StringSerializer(),
+                new VisibilityBindingSetSerializer());
+
+        private final BNodeIdFactory bNodeIdFactory;
+
+        /**
+         * Constructs an instance of {@link QueryVisitor}.
+         *
+         * @param bNodeIdFactory - Builds Blank Node IDs for the query's results. (not null)
+         */
+        public QueryVisitor(final BNodeIdFactory bNodeIdFactory) {
+            this.bNodeIdFactory = requireNonNull(bNodeIdFactory);
+        }
+
         /**
          * @return The {@link ProcessorEntry}s used to create a Topology.
          */
@@ -288,6 +358,23 @@ public class TopologyFactory implements TopologyBuilderFactory {
             return idMap;
         }
 
+        /**
+         * @return Information about how values are to be output by the topology to the results sink.
+         */
+        public SinkEntry<?, ?> getSinkEntry() {
+            return sinkEntry;
+        }
+
+        @Override
+        public void meet(final Reduced node) throws TopologyBuilderException {
+            // This indicates we're outputting VisibilityStatements.
+            sinkEntry = new SinkEntry<>(
+                    new StatementOutputFormatterSupplier(),
+                    new StringSerializer(),
+                    new VisibilityStatementSerializer());
+            super.meet(node);
+        }
+
         @Override
         public void meet(final StatementPattern node) throws TopologyBuilderException {
             // topology parent for Statement Patterns will always be a source
@@ -303,14 +390,39 @@ public class TopologyFactory implements TopologyBuilderFactory {
         public void meet(final Projection node) throws TopologyBuilderException {
             final String id = PROJECTION_PREFIX + UUID.randomUUID();
             final Optional<Side> side = getSide(node);
-            TupleExpr arg = node.getArg();
+
             // If the arg is an Extension, there are rebindings that need to be
             // ignored since they do not have a processor node.
-            if (arg instanceof Extension) {
-                arg = ((Extension) arg).getArg();
+            TupleExpr downstreamNode = node.getArg();
+            if (downstreamNode instanceof Extension) {
+                downstreamNode = ((Extension) downstreamNode).getArg();
+            }
+
+            final ProjectionProcessorSupplier supplier = new ProjectionProcessorSupplier(
+                    ProjectionEvaluator.make(node),
+                    result -> getResult(side, result));
+
+            entries.add(new ProcessorEntry(node, id, side, supplier, Lists.newArrayList(downstreamNode)));
+            idMap.put(node, id);
+            super.meet(node);
+        }
+
+        @Override
+        public void meet(final MultiProjection node) throws TopologyBuilderException {
+            final String id = PROJECTION_PREFIX + UUID.randomUUID();
+            final Optional<Side> side = getSide(node);
+
+            final MultiProjectionProcessorSupplier supplier = new MultiProjectionProcessorSupplier(
+                    MultiProjectionEvaluator.make(node, bNodeIdFactory),
+                    result -> getResult(side, result));
+
+            // If the arg is an Extension, then this node's grandchild is the next processing node.
+            TupleExpr downstreamNode = node.getArg();
+            if (downstreamNode instanceof Extension) {
+                downstreamNode = ((Extension) downstreamNode).getArg();
             }
-            final ProjectionProcessorSupplier supplier = new ProjectionProcessorSupplier(node.getProjectionElemList(), result -> getResult(side, result));
-            entries.add(new ProcessorEntry(node, id, side, supplier, Lists.newArrayList(arg)));
+
+            entries.add(new ProcessorEntry(node, id, side, supplier, Lists.newArrayList(downstreamNode)));
             idMap.put(node, id);
             super.meet(node);
         }
@@ -399,16 +511,4 @@ public class TopologyFactory implements TopologyBuilderFactory {
             }
         }
     }
-
-    /**
-     * An Exception thrown when a problem occurs when constructing the processor
-     * topology in the {@link TopologyFactory}.
-     */
-    public class TopologyBuilderException extends Exception {
-        private static final long serialVersionUID = 1L;
-
-        public TopologyBuilderException(final String message, final Throwable cause) {
-            super(message, cause);
-        }
-    }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
index 0a1a8a4..8898284 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
@@ -42,10 +42,8 @@ import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 
@@ -144,29 +142,34 @@ public final class KafkaTestUtil {
      * Runs a Kafka Streams topology, loads statements into the input topic, read the binding sets that come out of
      * the results topic, and ensures the expected results match the read results.
      *
-     * @param kafka - The embedded kafka instance that is being tested with. (not null)
+     * @param <T> The type of value that will be consumed from the results topic.
+     * @param kafka - The embedded Kafka instance that is being tested with. (not null)
      * @param statementsTopic - The topic statements will be written to. (not null)
      * @param resultsTopic - The topic results will be read from. (not null)
      * @param builder - The streams topology that will be executed. (not null)
      * @param startupMs - How long to wait for the topology to start before writing the statements.
      * @param statements - The statements that will be loaded into the topic. (not null)
      * @param expected - The expected results. (not null)
+     * @param expectedDeserializerClass - The class of the deserializer that will be used when reading
+     *   values from the results topic. (not null)
      * @throws Exception If any exception was thrown while running the test.
      */
-    public static void runStreamProcessingTest(
+    public static <T> void runStreamProcessingTest(
             final KafkaTestInstanceRule kafka,
             final String statementsTopic,
             final String resultsTopic,
             final TopologyBuilder builder,
             final int startupMs,
             final List<VisibilityStatement> statements,
-            final Set<VisibilityBindingSet> expected) throws Exception {
+            final Set<T> expected,
+            final Class<? extends Deserializer<T>> expectedDeserializerClass) throws Exception {
         requireNonNull(kafka);
         requireNonNull(statementsTopic);
         requireNonNull(resultsTopic);
         requireNonNull(builder);
         requireNonNull(statements);
         requireNonNull(expected);
+        requireNonNull(expectedDeserializerClass);
 
         // Explicitly create the topics that are being used.
         kafka.createTopic(statementsTopic);
@@ -191,13 +194,12 @@ public final class KafkaTestUtil {
             }
 
             // Wait for the final results to appear in the output topic and verify the expected Binding Sets were found.
-            try(Consumer<String, VisibilityBindingSet> consumer = KafkaTestUtil.fromStartConsumer(
-                    kafka, StringDeserializer.class, VisibilityBindingSetDeserializer.class)) {
+            try(Consumer<String, T> consumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, expectedDeserializerClass)) {
                 // Register the topic.
                 consumer.subscribe(Arrays.asList(resultsTopic));
 
                 // Poll for the result.
-                final Set<VisibilityBindingSet> results = Sets.newHashSet( KafkaTestUtil.pollForResults(500, 6, expected.size(), consumer) );
+                final Set<T> results = Sets.newHashSet( KafkaTestUtil.pollForResults(500, 6, expected.size(), consumer) );
 
                 // Show the correct binding sets results from the job.
                 assertEquals(expected, results);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
index e55ec2e..3e0e64d 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
@@ -25,11 +25,13 @@ import java.util.Set;
 import java.util.UUID;
 
 import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.topology.TopologyFactory;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.Rule;
@@ -57,7 +59,7 @@ public class StatementPatternProcessorIT {
         // Setup a topology.
         final String query = "SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }";
         final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Create a statement that generate an SP result.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -73,7 +75,7 @@ public class StatementPatternProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -87,7 +89,7 @@ public class StatementPatternProcessorIT {
         // Setup a topology.
         final String query = "SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }";
         final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Create some statements where some generates SP results and others do not.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -111,7 +113,7 @@ public class StatementPatternProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a|b") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -128,7 +130,7 @@ public class StatementPatternProcessorIT {
                 + "?person ?action <urn:Bob>"
                 + "}";
         final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Create some statements where some generates SP results and others do not.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -145,7 +147,7 @@ public class StatementPatternProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -162,7 +164,7 @@ public class StatementPatternProcessorIT {
                 + "?person ?action <urn:Bob>"
                 + "}";
         final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Create some statements where some generates SP results and others do not.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -188,6 +190,6 @@ public class StatementPatternProcessorIT {
         expected.add(new VisibilityBindingSet(bs, "a"));
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
index dbad15c..b137a9a 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
@@ -24,35 +24,23 @@ import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.kafka.streams.state.Stores;
-import org.apache.rya.api.function.join.LeftOuterJoin;
 import org.apache.rya.api.function.join.NaturalJoin;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RdfTestUtil;
 import org.apache.rya.streams.kafka.processors.ProcessorResult;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
-import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier.JoinProcessor;
-import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerde;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
-import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.topology.TopologyFactory;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.algebra.StatementPattern;
 import org.openrdf.query.impl.MapBindingSet;
 
 import com.google.common.collect.Lists;
@@ -84,12 +72,13 @@ public class JoinProcessorIT {
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
         // Setup a topology.
-        final String query = "SELECT * WHERE { "
-                + "?person <urn:talksTo> ?employee ."
-                + "?employee <urn:worksAt> ?business"
-                + " }";
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "?employee <urn:worksAt> ?business" +
+                " }";
         final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Create some statements that generate a bunch of right SP results.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -122,7 +111,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -133,46 +122,14 @@ public class JoinProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the StatementPatterns that will be evaluated.
-        final StatementPattern leftSp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
-        final StatementPattern rightSp = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
-
         // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("LEFT_SP", new StatementPatternProcessorSupplier(leftSp,
-                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
-
-        // Add a processor that handles the second statement pattern.
-        builder.addProcessor("RIGHT_SP", new StatementPatternProcessorSupplier(rightSp,
-                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
-
-        // Add a processor that handles a natrual join over the SPs.
-        builder.addProcessor("NATURAL_JOIN", new JoinProcessorSupplier(
-                "NATURAL_JOIN",
-                new NaturalJoin(),
-                Lists.newArrayList("employee"),
-                Lists.newArrayList("employee", "person", "business"),
-                result -> ProcessorResult.make( new UnaryResult(result) )), "LEFT_SP", "RIGHT_SP");
-
-        // Add a state store for the join processor.
-        final StateStoreSupplier joinStoreSupplier =
-                Stores.create( "NATURAL_JOIN" )
-                .withStringKeys()
-                .withValues(new VisibilityBindingSetSerde())
-                .inMemory()
-                .build();
-        builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "NATURAL_JOIN");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "?employee <urn:worksAt> ?business" +
+                " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Create some statements that generate a bunch of right SP results.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -205,7 +162,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -216,46 +173,14 @@ public class JoinProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the StatementPatterns that will be evaluated.
-        final StatementPattern leftSp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
-        final StatementPattern rightSp = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
-
         // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("LEFT_SP", new StatementPatternProcessorSupplier(leftSp,
-                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
-
-        // Add a processor that handles the second statement pattern.
-        builder.addProcessor("RIGHT_SP", new StatementPatternProcessorSupplier(rightSp,
-                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
-
-        // Add a processor that handles a natrual join over the SPs.
-        builder.addProcessor("NATURAL_JOIN", new JoinProcessorSupplier(
-                "NATURAL_JOIN",
-                new NaturalJoin(),
-                Lists.newArrayList("employee"),
-                Lists.newArrayList("employee", "person", "business"),
-                result -> ProcessorResult.make( new UnaryResult(result) )), "LEFT_SP", "RIGHT_SP");
-
-        // Add a state store for the join processor.
-        final StateStoreSupplier joinStoreSupplier =
-                Stores.create( "NATURAL_JOIN" )
-                .withStringKeys()
-                .withValues(new VisibilityBindingSetSerde())
-                .inMemory()
-                .build();
-        builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "NATURAL_JOIN");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "?employee <urn:worksAt> ?business" +
+                " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Create some statements that generate a bunch of right SP results.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -294,7 +219,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a&c") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -305,67 +230,15 @@ public class JoinProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the StatementPatterns that will be evaluated.
-        final StatementPattern sp1 = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
-        final StatementPattern sp2 = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
-        final StatementPattern sp3 = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:hourlyWage> ?wage }");
-
         // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp1,
-                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
-
-        // Add a processor that handles the second statement pattern.
-        builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2,
-                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
-
-        // Add a processor that handles a natural join over SPs 1 and 2.
-        builder.addProcessor("JOIN1", new JoinProcessorSupplier(
-                "JOIN1",
-                new NaturalJoin(),
-                Lists.newArrayList("employee"),
-                Lists.newArrayList("employee", "person", "business"),
-                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "SP1", "SP2");
-
-        // Add a processor that handles the third statement pattern.
-        builder.addProcessor("SP3", new StatementPatternProcessorSupplier(sp3,
-                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
-
-        // Add a processor that handles a natural join over JOIN1 and SP3.
-        builder.addProcessor("JOIN2", new JoinProcessorSupplier(
-                "JOIN2",
-                new NaturalJoin(),
-                Lists.newArrayList("employee"),
-                Lists.newArrayList("employee", "business", "wage"),
-                result -> ProcessorResult.make( new UnaryResult(result) )), "JOIN1", "SP3");
-
-        // Setup the join state suppliers.
-        final StateStoreSupplier join1StoreSupplier =
-                Stores.create( "JOIN1" )
-                .withStringKeys()
-                .withValues(new VisibilityBindingSetSerde())
-                .inMemory()
-                .build();
-        builder.addStateStore(join1StoreSupplier, "JOIN1");
-
-        final StateStoreSupplier join2StoreSupplier =
-                Stores.create( "JOIN2" )
-                .withStringKeys()
-                .withValues(new VisibilityBindingSetSerde())
-                .inMemory()
-                .build();
-        builder.addStateStore(join2StoreSupplier, "JOIN2");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "JOIN2");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "?employee <urn:worksAt> ?business ." +
+                    "?employee <urn:hourlyWage> ?wage ." +
+                " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Create some statements that generate a bunch of right SP results.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -387,7 +260,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 3000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 3000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -398,46 +271,14 @@ public class JoinProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the StatementPatterns that will be evaluated.
-        final StatementPattern requiredSp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
-        final StatementPattern optionalSp = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
-
         // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("REQUIRED_SP", new StatementPatternProcessorSupplier(requiredSp,
-                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
-
-        // Add a processor that handles the second statement pattern.
-        builder.addProcessor("OPTIONAL_SP", new StatementPatternProcessorSupplier(optionalSp,
-                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
-
-        // Add a processor that handles a natrual join over the SPs.
-        builder.addProcessor("LEFT_JOIN", new JoinProcessorSupplier(
-                "LEFT_JOIN",
-                new LeftOuterJoin(),
-                Lists.newArrayList("employee"),
-                Lists.newArrayList("employee", "person", "business"),
-                result -> ProcessorResult.make( new UnaryResult(result) )), "REQUIRED_SP", "OPTIONAL_SP");
-
-        // Add a state store for the join processor.
-        final StateStoreSupplier joinStoreSupplier =
-                Stores.create( "LEFT_JOIN" )
-                .withStringKeys()
-                .withValues(new VisibilityBindingSetSerde())
-                .inMemory()
-                .build();
-        builder.addStateStore(joinStoreSupplier, "LEFT_JOIN");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "LEFT_JOIN");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "OPTIONAL{ ?employee <urn:worksAt> ?business } " +
+                " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Create some statements that generate a result that includes the optional value as well as one that does not.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -470,6 +311,6 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "c") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
index ee0e55b..d71577b 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
@@ -24,29 +24,13 @@ import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.kafka.streams.state.Stores;
-import org.apache.rya.api.function.join.NaturalJoin;
-import org.apache.rya.api.function.projection.MultiProjectionEvaluator;
-import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RdfTestUtil;
-import org.apache.rya.streams.kafka.processors.ProcessorResult;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
-import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
-import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier;
-import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
 import org.apache.rya.streams.kafka.processors.projection.MultiProjectionProcessorSupplier.MultiProjectionProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerde;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -54,11 +38,6 @@ import org.openrdf.model.BNode;
 import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.ValueFactoryImpl;
 import org.openrdf.model.vocabulary.RDF;
-import org.openrdf.query.algebra.MultiProjection;
-import org.openrdf.query.algebra.StatementPattern;
-import org.openrdf.query.impl.MapBindingSet;
-
-import com.google.common.collect.Lists;
 
 /**
  * Integration tests the methods of {@link MultiProjectionProcessor}.
@@ -76,10 +55,8 @@ public class MultiProjectionProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the RDF model objects that will be used to build the query.
-        final StatementPattern sp1 = RdfTestUtil.getSp("SELECT * WHERE { ?thing <urn:corner> ?location . }");
-        final StatementPattern sp2 = RdfTestUtil.getSp("SELECT * WHERE { ?thing <urn:compass> ?direction . }");
-        final MultiProjection multiProjection = RdfTestUtil.getMultiProjection(
+        // Create a topology for the Query that will be tested.
+        final String sparql =
                 "CONSTRUCT {" +
                     "_:b a <urn:movementObservation> ; " +
                     "<urn:location> ?location ; " +
@@ -88,38 +65,10 @@ public class MultiProjectionProcessorIT {
                 "WHERE {" +
                     "?thing <urn:corner> ?location ." +
                     "?thing <urn:compass> ?direction." +
-                "}");
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp1,
-                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
-        builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2,
-                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
-
-        builder.addProcessor("NATURAL_JOIN", new JoinProcessorSupplier(
-                "NATURAL_JOIN",
-                new NaturalJoin(),
-                Lists.newArrayList("thing"),
-                Lists.newArrayList("thing", "location", "direction"),
-                result -> ProcessorResult.make( new UnaryResult(result) )), "SP1", "SP2");
-
-        final StateStoreSupplier joinStoreSupplier =
-                Stores.create( "NATURAL_JOIN" )
-                  .withStringKeys()
-                  .withValues(new VisibilityBindingSetSerde())
-                  .inMemory()
-                  .build();
-        builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
+                "}";
 
-        final String blankNodeId = UUID.randomUUID().toString();
-        builder.addProcessor("MULTIPROJECTION", new MultiProjectionProcessorSupplier(
-                MultiProjectionEvaluator.make(multiProjection, () -> blankNodeId),
-                result -> ProcessorResult.make(new UnaryResult(result))), "NATURAL_JOIN");
-
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "MULTIPROJECTION");
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String bNodeId = UUID.randomUUID().toString();
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, () -> bNodeId);
 
         // Create the statements that will be input into the query.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -130,26 +79,14 @@ public class MultiProjectionProcessorIT {
                 vf.createStatement(vf.createURI("urn:car1"), vf.createURI("urn:corner"), vf.createURI("urn:corner1")), "a") );
 
         // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        final BNode blankNode = vf.createBNode(blankNodeId);
-
-        MapBindingSet expectedBs = new MapBindingSet();
-        expectedBs.addBinding("subject", blankNode);
-        expectedBs.addBinding("predicate", RDF.TYPE);
-        expectedBs.addBinding("object", vf.createURI("urn:movementObservation"));
-
-        expectedBs = new MapBindingSet();
-        expectedBs.addBinding("subject", blankNode);
-        expectedBs.addBinding("predicate", vf.createURI("urn:direction"));
-        expectedBs.addBinding("object", vf.createURI("urn:NW"));
-
+        final Set<VisibilityStatement> expected = new HashSet<>();
+        final BNode blankNode = vf.createBNode(bNodeId);
 
-        expectedBs = new MapBindingSet();
-        expectedBs.addBinding("subject", blankNode);
-        expectedBs.addBinding("predicate", vf.createURI("urn:location"));
-        expectedBs.addBinding("object", vf.createURI("urn:corner1"));
+        expected.add(new VisibilityStatement(vf.createStatement(blankNode, RDF.TYPE, vf.createURI("urn:movementObservation")), "a"));
+        expected.add(new VisibilityStatement(vf.createStatement(blankNode, vf.createURI("urn:direction"), vf.createURI("urn:NW")), "a"));
+        expected.add(new VisibilityStatement(vf.createStatement(blankNode, vf.createURI("urn:location"), vf.createURI("urn:corner1")), "a"));
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityStatementDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
index 99e2451..bc5f115 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
@@ -24,30 +24,20 @@ import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.function.projection.ProjectionEvaluator;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RdfTestUtil;
-import org.apache.rya.streams.kafka.processors.ProcessorResult;
-import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
-import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
-import org.apache.rya.streams.kafka.processors.projection.ProjectionProcessorSupplier;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
-import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.algebra.Projection;
-import org.openrdf.query.algebra.StatementPattern;
 import org.openrdf.query.impl.MapBindingSet;
 
 import com.google.common.collect.Sets;
@@ -68,34 +58,14 @@ public class ProjectionProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the RDF model objects that will be used to build the query.
+        // Create a topology for the Query that will be tested.
         final String sparql =
                 "SELECT (?person AS ?p) ?otherPerson " +
                 "WHERE { " +
                     "?person <urn:talksTo> ?otherPerson . " +
                 "}";
-        final Projection projection = RdfTestUtil.getProjection(sparql);
-        final StatementPattern sp = RdfTestUtil.getSp(sparql);
 
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
-
-        // Add a processor that handles the projection.
-        builder.addProcessor("P1", new ProjectionProcessorSupplier(
-                ProjectionEvaluator.make(projection),
-                result -> ProcessorResult.make(new UnaryResult(result))), "SP1");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "P1");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Load some data into the input topic.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -110,6 +80,6 @@ public class ProjectionProcessorIT {
         expectedBs.addBinding("otherPerson", vf.createURI("urn:Bob"));
         expected.add(new VisibilityBindingSet(expectedBs, "a"));
 
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected));
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected), VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/da63fd12/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/topology/TopologyFactoryTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/topology/TopologyFactoryTest.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/topology/TopologyFactoryTest.java
index eda4c89..31462ec 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/topology/TopologyFactoryTest.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/topology/TopologyFactoryTest.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.rya.streams.kafka.topology;
 
 import static org.junit.Assert.assertEquals;
@@ -5,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.List;
 
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
 import org.apache.rya.streams.kafka.topology.TopologyFactory.ProcessorEntry;
 import org.junit.Before;
 import org.junit.Test;
@@ -15,6 +34,9 @@ import org.openrdf.query.algebra.Projection;
 import org.openrdf.query.algebra.StatementPattern;
 import org.openrdf.query.algebra.Var;
 
+/**
+ * Unit tests the methods of {@link TopologyFactory}.
+ */
 public class TopologyFactoryTest {
     private static TopologyFactory FACTORY;
 
@@ -40,7 +62,7 @@ public class TopologyFactoryTest {
                 + "?person <urn:talksTo> ?otherPerson . "
                 + "}";
 
-        FACTORY.build(query, "source", "sink");
+        FACTORY.build(query, "source", "sink", new RandomUUIDFactory());
         final List<ProcessorEntry> entries = FACTORY.getProcessorEntry();
 
         assertTrue(entries.get(0).getNode() instanceof Projection);
@@ -57,7 +79,7 @@ public class TopologyFactoryTest {
                 + "?otherPerson <urn:talksTo> ?dog . "
                 + "}";
 
-        FACTORY.build(query, "source", "sink");
+        FACTORY.build(query, "source", "sink", new RandomUUIDFactory());
         final List<ProcessorEntry> entries = FACTORY.getProcessorEntry();
 
         assertTrue(entries.get(0).getNode() instanceof Projection);
@@ -76,7 +98,7 @@ public class TopologyFactoryTest {
                 + "?dog <urn:chews> ?toy . "
                 + "}";
 
-        FACTORY.build(query, "source", "sink");
+        FACTORY.build(query, "source", "sink", new RandomUUIDFactory());
         final List<ProcessorEntry> entries = FACTORY.getProcessorEntry();
 
         assertTrue(entries.get(0).getNode() instanceof Projection);
@@ -96,11 +118,11 @@ public class TopologyFactoryTest {
                 + "?person <urn:talksTo> ?otherPerson . "
             + "}";
 
-        FACTORY.build(query, "source", "sink");
+        FACTORY.build(query, "source", "sink", new RandomUUIDFactory());
         final List<ProcessorEntry> entries = FACTORY.getProcessorEntry();
 
         assertTrue(entries.get(0).getNode() instanceof Projection);
         final StatementPattern expected = new StatementPattern(new Var("person"), TALKS_TO, new Var("otherPerson"));
         assertEquals(expected, entries.get(1).getNode());
     }
-}
+}
\ No newline at end of file


[29/50] [abbrv] incubator-rya git commit: RYA-377 Implement the Aggregation Processor for Rya Streams.

Posted by ca...@apache.org.
RYA-377 Implement the Aggregation Processor for Rya Streams.


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

Branch: refs/heads/master
Commit: 95df37a30ff6e4f2e70a863872e1087a447de47f
Parents: 8363724
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Nov 21 16:53:38 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../aggregation/AggregationStateStore.java      |  49 ++
 .../aggregation/AggregationsEvaluator.java      | 175 +++++++
 .../AggregationProcessorSupplier.java           | 158 +++++++
 .../KeyValueAggregationStateStore.java          | 104 +++++
 .../streams/kafka/topology/TopologyFactory.java |  24 +-
 .../aggregation/AggregationProcessorIT.java     | 457 +++++++++++++++++++
 6 files changed, 962 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/95df37a3/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationStateStore.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationStateStore.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationStateStore.java
new file mode 100644
index 0000000..d37f4c7
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationStateStore.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import java.util.Optional;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Provides a mechanism for storing the updating {@link AggregationState} while using an {@link AggregationsEvaluator}.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface AggregationStateStore {
+
+    /**
+     * Stores a state. If this value updates a previously stored state, then it will overwrite the old value
+     * with the new one.
+     *
+     * @param state - The state that will be stored. (not null)
+     */
+    public void store(AggregationState state);
+
+    /**
+     * Get the {@link AggregationState} that may be updatted using the provided binding set.
+     *
+     * @param bs - A binding set that defines which state to fetch. (not null)
+     * @return The {@link AggregationState} that is updated by the binding set, if one has been stored.
+     */
+    public Optional<AggregationState> get(VisibilityBindingSet bs);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/95df37a3/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationsEvaluator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationsEvaluator.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationsEvaluator.java
new file mode 100644
index 0000000..2aa716f
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationsEvaluator.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.visibility.VisibilitySimplifier;
+import org.openrdf.query.algebra.AggregateOperator;
+import org.openrdf.query.algebra.Group;
+import org.openrdf.query.algebra.GroupElem;
+import org.openrdf.query.algebra.Var;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.ImmutableMap;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A stateful evaluator that processes aggregation functions over variables that are grouped together.
+ * </p>
+ * The following aggregation functions are supported:
+ * <ul>
+ *   <li>Count</li>
+ *   <li>Sum</li>
+ *   <li>Average</li>
+ *   <li>Min</li>
+ *   <li>Max</li>
+ * </ul>
+ * </p>
+ * The persistence of the aggregation's state is determined by the provided {@link AggregationStateStore}.
+ */
+@DefaultAnnotation(NonNull.class)
+public class AggregationsEvaluator {
+
+    private static final ImmutableMap<AggregationType, AggregationFunction> FUNCTIONS;
+    static {
+        final ImmutableMap.Builder<AggregationType, AggregationFunction> builder = ImmutableMap.builder();
+        builder.put(AggregationType.COUNT, new CountFunction());
+        builder.put(AggregationType.SUM, new SumFunction());
+        builder.put(AggregationType.AVERAGE, new AverageFunction());
+        builder.put(AggregationType.MIN, new MinFunction());
+        builder.put(AggregationType.MAX, new MaxFunction());
+        FUNCTIONS = builder.build();
+    }
+
+    private final AggregationStateStore aggStateStore;
+    private final Collection<AggregationElement> aggregations;
+    private final List<String> groupByVars;
+
+    /**
+     * Constructs an instance of {@link AggregationsEvaluator}.
+     *
+     * @param aggStateStore - The mechanism for storing aggregation state. (not null)
+     * @param aggregations - The aggregation functions that will be computed. (not null)
+     * @param groupByVars - The names of the binding whose values are used to group aggregation results. (not null)
+     */
+    public AggregationsEvaluator(
+            final AggregationStateStore aggStateStore,
+            final Collection<AggregationElement> aggregations,
+            final List<String> groupByVars) {
+        this.aggStateStore = requireNonNull(aggStateStore);
+        this.aggregations = requireNonNull(aggregations);
+        this.groupByVars = requireNonNull(groupByVars);
+    }
+
+    /**
+     * Make an instance of {@link AggregationsEvaluator} based on a {@link Group} node.
+     *
+     * @param aggStateStore - The mechanism for storing aggregation state. (not null)
+     * @param aggNode - Defines which aggregation functions need to be performed.
+     * @param groupByVars - The names of the binding whose values are used to group aggregation results. (not null)
+     * @return The evaluator that handles the node's aggregations.
+     */
+    public static  AggregationsEvaluator make(final AggregationStateStore aggStateStore, final Group aggNode, final List<String> groupByVars) {
+        requireNonNull(aggStateStore);
+        requireNonNull(aggNode);
+        requireNonNull(groupByVars);
+
+        // The aggregations that need to be performed are the Group Elements.
+        final List<AggregationElement> aggregations = new ArrayList<>();
+        for(final GroupElem groupElem : aggNode.getGroupElements()) {
+            // Figure out the type of the aggregation.
+            final AggregateOperator operator = groupElem.getOperator();
+            final Optional<AggregationType> type = AggregationType.byOperatorClass( operator.getClass() );
+
+            // If the type is one we support, create the AggregationElement.
+            if(type.isPresent()) {
+                final String resultBindingName = groupElem.getName();
+
+                final AtomicReference<String> aggregatedBindingName = new AtomicReference<>();
+                groupElem.visitChildren(new QueryModelVisitorBase<RuntimeException>() {
+                    @Override
+                    public void meet(final Var node) {
+                        aggregatedBindingName.set( node.getName() );
+                    }
+                });
+
+                aggregations.add( new AggregationElement(type.get(), aggregatedBindingName.get(), resultBindingName) );
+            }
+        }
+
+        return new AggregationsEvaluator(aggStateStore, aggregations, groupByVars);
+    }
+
+    /**
+     * Update the aggregation values using information found within {@code newBs}.
+     *
+     * @param newBs - A binding set whose values need to be incorporated within the aggregations. (not null)
+     * @return A binding set containing the updated aggregation values.
+     */
+    public VisibilityBindingSet update(final VisibilityBindingSet newBs) {
+        requireNonNull(newBs);
+
+        // Load the old state if one was previously stored; otherwise initialize the state.
+        final AggregationState state = aggStateStore.get(newBs).orElseGet(() -> {
+            // Initialize a new state.
+            final AggregationState newState = new AggregationState();
+
+            // If we have group by bindings, their values need to be added to the state's binding set.
+            final MapBindingSet bindingSet = newState.getBindingSet();
+            for(final String groupByVar : groupByVars) {
+                bindingSet.addBinding( newBs.getBinding(groupByVar) );
+            }
+
+            return newState;
+        });
+
+        // Update the visibilities of the result binding set based on the new result's visibilities.
+        final String oldVisibility = state.getVisibility();
+        final String updateVisibilities = VisibilitySimplifier.unionAndSimplify(oldVisibility, newBs.getVisibility());
+        state.setVisibility(updateVisibilities);
+
+        // Update the Aggregation State with each Aggregation function included within this group.
+        for(final AggregationElement aggregation : aggregations) {
+            final AggregationType type = aggregation.getAggregationType();
+            final AggregationFunction function = FUNCTIONS.get(type);
+            if(function == null) {
+                throw new RuntimeException("Unrecognized aggregation function: " + type);
+            }
+
+            function.update(aggregation, state, newBs);
+        }
+
+        // Store the updated state. This will write on top of any old state that was present for the Group By values.
+        aggStateStore.store(state);
+
+        // Return the updated binding set from the updated state.
+        return new VisibilityBindingSet(state.getBindingSet(), state.getVisibility());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/95df37a3/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorSupplier.java
new file mode 100644
index 0000000..c8e1049
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorSupplier.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.aggregation;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.rya.api.function.aggregation.AggregationState;
+import org.apache.rya.api.function.aggregation.AggregationStateStore;
+import org.apache.rya.api.function.aggregation.AggregationsEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.ResultType;
+import org.apache.rya.streams.kafka.processors.ProcessorResultFactory;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessor;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessorSupplier;
+import org.openrdf.query.algebra.Group;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.internal.Lists;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Supplies {@link AggregationProcessor} instances.
+ */
+@DefaultAnnotation(NonNull.class)
+public class AggregationProcessorSupplier extends RyaStreamsProcessorSupplier {
+
+    private final String stateStoreName;
+    private final Group aggNode;
+
+    /**
+     * Constructs an instance of {@link AggregationProcessorSupplier}.
+     *
+     * @param stateStoreName - The name of the state store the processor will use. (not null)
+     * @param aggNode - Defines which aggregations will be performed by the processor. (not null)
+     * @param resultFactory - The factory that the supplied processors will use to create results. (not null)
+     */
+    public AggregationProcessorSupplier(
+            final String stateStoreName,
+            final Group aggNode,
+            final ProcessorResultFactory resultFactory) {
+        super(resultFactory);
+        this.stateStoreName = requireNonNull(stateStoreName);
+        this.aggNode = requireNonNull(aggNode);
+    }
+
+    @Override
+    public Processor<Object, ProcessorResult> get() {
+        return new AggregationProcessor(stateStoreName, aggNode, super.getResultFactory());
+    }
+
+    /**
+     * Evaluates a {@link Group} node that contains a bunch of aggregations. Each aggregation will have a binding
+     * within the resulting binding sets that contains the aggregation value.
+     *
+     * @see AggregationsEvaluator
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static class AggregationProcessor extends RyaStreamsProcessor {
+        private static final Logger log = LoggerFactory.getLogger(AggregationProcessor.class);
+
+        private final String stateStoreName;
+        private final Group aggNode;
+
+        private ProcessorContext context;
+        private AggregationStateStore aggStateStore;
+        private AggregationsEvaluator evaluator;
+
+        /**
+         * Constructs an instance of {@link AggregationProcessor}.
+         *
+         * @param stateStoreName - The name of the Kafka Streams state store that this processor will use. (not null)
+         * @param aggNode - The group by node that configures how the aggregations will be performed. (not null)
+         * @param resultFactory - The factory that will format this processor's final results for the downstream
+         *   processor. (not null)
+         */
+        public AggregationProcessor(
+                final String stateStoreName,
+                final Group aggNode,
+                final ProcessorResultFactory resultFactory) {
+            super(resultFactory);
+            this.stateStoreName = requireNonNull(stateStoreName);
+            this.aggNode = requireNonNull(aggNode);
+        }
+
+        @Override
+        public void init(final ProcessorContext context) {
+            this.context = context;
+
+            // Sort the group by vars so that they will be written to the state store in the same order every time.
+            final List<String> groupByVars = Lists.newArrayList(aggNode.getGroupBindingNames());
+            groupByVars.sort(Comparator.naturalOrder());
+
+            // Get a reference to the state store that keeps track of aggregation state.
+            final KeyValueStore<String, AggregationState> stateStore =
+                    (KeyValueStore<String, AggregationState>) context.getStateStore( stateStoreName );
+            aggStateStore = new KeyValueAggregationStateStore(stateStore, groupByVars);
+
+            // Create the aggregation evaluator.
+            evaluator = AggregationsEvaluator.make(aggStateStore, aggNode, groupByVars);
+        }
+
+        @Override
+        public void process(final Object key, final ProcessorResult value) {
+            // Aggregations can only be unary.
+            if (value.getType() != ResultType.UNARY) {
+                throw new RuntimeException("The ProcessorResult to be processed must be Unary.");
+            }
+
+            // Log the binding set that has been input.
+            log.debug("\nINPUT:\nBinding Set: {}", value.getUnary().getResult());
+
+            // Update the aggregations values.
+            final VisibilityBindingSet resultBs = evaluator.update(value.getUnary().getResult());
+
+            // Log the binding set that will be output.
+            log.debug("\nOUTPUT:\nBinding Set: {}", resultBs);
+
+            // Forward to the updated aggregation binding set to the downstream processors.
+            context.forward(key, super.getResultFactory().make(resultBs));
+        }
+
+        @Override
+        public void punctuate(final long timestamp) {
+            // Do nothing.
+        }
+
+        @Override
+        public void close() {
+            // Do nothing.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/95df37a3/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/KeyValueAggregationStateStore.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/KeyValueAggregationStateStore.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/KeyValueAggregationStateStore.java
new file mode 100644
index 0000000..3300590
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/KeyValueAggregationStateStore.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.aggregation;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.rya.api.function.aggregation.AggregationState;
+import org.apache.rya.api.function.aggregation.AggregationStateStore;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.query.BindingSet;
+
+import com.google.common.base.Joiner;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A {@link KeyValueStore} implementation of {@link AggregationStateStore}.
+ * </p>
+ * This is a key/value store, so we need to store the {@link AggregationState} for each set of group by values
+ * using a key that is composed with those values. We use the following pattern to accomplish this:
+ * <pre>
+ * [groupByVar1 value],[groupByVar2 value],...,[groupByVarN value]
+ * </pre>
+ */
+@DefaultAnnotation(NonNull.class)
+public class KeyValueAggregationStateStore implements AggregationStateStore {
+
+    private final KeyValueStore<String, AggregationState> store;
+    private final List<String> groupByVars;
+
+    /**
+     * Constructs an instance of {@link KeyValueAggregationStateStore}.
+     *
+     * @param store - The state store that will be used. (not null)
+     * @param groupByVars - An ordered list of group by variable names. (not null)
+     */
+    public KeyValueAggregationStateStore(
+            final KeyValueStore<String, AggregationState> store,
+            final List<String> groupByVars) {
+        this.store = requireNonNull(store);
+        this.groupByVars = requireNonNull(groupByVars);
+    }
+
+    @Override
+    public void store(final AggregationState state) {
+        requireNonNull(state);
+
+        // Aggregations group their states by their group by variables, so the key is the resulting binding
+        // set's values for the group by variables.
+        final String key = makeCommaDelimitedValues(groupByVars, state.getBindingSet());
+        store.put(key, state);
+    }
+
+    @Override
+    public Optional<AggregationState> get(final VisibilityBindingSet bs) {
+        requireNonNull(bs);
+
+        final String key = makeCommaDelimitedValues(groupByVars, bs);
+        return Optional.ofNullable(store.get(key));
+    }
+
+    /**
+     * A utility function that helps construct the keys used by {@link KeyValueAggregationStateStore}.
+     *
+     * @param vars - Which variables within the binding set to use for the key's values. (not null)
+     * @param bindingSet - The binding set the key is being constructed from. (not null)
+     * @return A comma delimited list of the binding values, leading with the side.
+     */
+    private static String makeCommaDelimitedValues(final List<String> vars, final BindingSet bindingSet) {
+        requireNonNull(vars);
+        requireNonNull(bindingSet);
+
+        // Make a an ordered list of the binding set variables.
+        final List<String> values = new ArrayList<>();
+        for(final String var : vars) {
+            values.add( bindingSet.hasBinding(var) ? bindingSet.getBinding(var).getValue().toString() : "" );
+        }
+
+        // Return a comma delimited list of those values.
+        return Joiner.on(",").join(values);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/95df37a3/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
index 426b041..4046e23 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
@@ -51,6 +51,7 @@ import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.aggregation.AggregationProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier;
 import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier;
@@ -65,6 +66,7 @@ import org.openrdf.query.MalformedQueryException;
 import org.openrdf.query.algebra.BinaryTupleOperator;
 import org.openrdf.query.algebra.Extension;
 import org.openrdf.query.algebra.Filter;
+import org.openrdf.query.algebra.Group;
 import org.openrdf.query.algebra.Join;
 import org.openrdf.query.algebra.LeftJoin;
 import org.openrdf.query.algebra.MultiProjection;
@@ -94,6 +96,7 @@ public class TopologyFactory implements TopologyBuilderFactory {
     private static final String JOIN_PREFIX = "JOIN_";
     private static final String PROJECTION_PREFIX = "PROJECTION_";
     private static final String FILTER_PREFIX = "FILTER_";
+    private static final String AGGREGATION_PREFIX = "AGGREGATION_";
     private static final String SINK = "SINK";
 
     private List<ProcessorEntry> processorEntryList;
@@ -141,14 +144,15 @@ public class TopologyFactory implements TopologyBuilderFactory {
                 builder.addProcessor(entry.getID(), entry.getSupplier(), parentIDs);
             }
 
-            if (entry.getNode() instanceof Join || entry.getNode() instanceof LeftJoin) {
+            // Add a state store for any node type that requires one.
+            if (entry.getNode() instanceof Join ||  entry.getNode() instanceof LeftJoin || entry.getNode() instanceof Group) {
                 // Add a state store for the join processor.
                 final StateStoreSupplier joinStoreSupplier =
                         Stores.create( entry.getID() )
-                        .withStringKeys()
-                        .withValues(new VisibilityBindingSetSerde())
-                        .persistent()
-                        .build();
+                            .withStringKeys()
+                            .withValues(new VisibilityBindingSetSerde())
+                            .persistent()
+                            .build();
                 builder.addStateStore(joinStoreSupplier, entry.getID());
             }
         }
@@ -459,6 +463,16 @@ public class TopologyFactory implements TopologyBuilderFactory {
             super.meet(node);
         }
 
+        @Override
+        public void meet(final Group node) throws TopologyBuilderException {
+            final String id = AGGREGATION_PREFIX + UUID.randomUUID();
+            final Optional<Side> side = getSide(node);
+            final AggregationProcessorSupplier supplier = new AggregationProcessorSupplier(id, node, (result) -> getResult(side, result));
+            entries.add( new ProcessorEntry(node, id, side, supplier, Lists.newArrayList(node.getArg())) );
+            idMap.put(node, id);
+            super.meet(node);
+        }
+
         /**
          * Gets the {@link Side} the current node in the visitor is on relative to the provided node.
          * @param node - The node used to determine the side of the current visitor node.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/95df37a3/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
new file mode 100644
index 0000000..ccf5c0c
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
@@ -0,0 +1,457 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.aggregation;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.aggregation.AggregationProcessorSupplier.AggregationProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.XMLSchema;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Integration tests {@link AggregationProcessor}.
+ */
+public class AggregationProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(false);
+
+    @Test
+    public void count() throws Exception {
+        // A query that figures out how many books each person has.
+        final String sparql =
+                "SELECT ?person (count(?book) as ?bookCount) " +
+                "WHERE { " +
+                    "?person <urn:hasBook> ?book " +
+                "} GROUP BY ?person";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasBook"), vf.createLiteral("Book 1")), "a"));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasBook"), vf.createLiteral("Book 1")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasBook"), vf.createLiteral("Book 2")), "b"));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("bookCount", vf.createLiteral("1", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, "a"));
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("bookCount", vf.createLiteral("1", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("bookCount", vf.createLiteral("2", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, "a&b"));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void sum() throws Exception {
+        // A query that figures out how much food each person has.
+        final String sparql =
+                "SELECT ?person (sum(?foodCount) as ?totalFood) " +
+                "WHERE { " +
+                    "?person <urn:hasFoodType> ?food . " +
+                    "?food <urn:count> ?foodCount . " +
+                "} GROUP BY ?person";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasFoodType"), vf.createURI("urn:corn")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasFoodType"), vf.createURI("urn:apple")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:corn"), vf.createURI("urn:count"), vf.createLiteral(4)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:apple"), vf.createURI("urn:count"), vf.createLiteral(3)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("totalFood", vf.createLiteral("4", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("totalFood", vf.createLiteral("7", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void average() throws Exception {
+        // A query that figures out the average age across all people.
+        final String sparql =
+                "SELECT (avg(?age) as ?avgAge) " +
+                "WHERE { " +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(3)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(2)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("avgAge", vf.createLiteral("3", XMLSchema.DECIMAL));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("avgAge", vf.createLiteral("5", XMLSchema.DECIMAL));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("avgAge", vf.createLiteral("4", XMLSchema.DECIMAL));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void min() throws Exception {
+        // A query that figures out what the youngest age is across all people.
+        final String sparql =
+                "SELECT (min(?age) as ?youngest) " +
+                "WHERE { " +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(13)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(14)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:age"), vf.createLiteral(5)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:age"), vf.createLiteral(25)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(13));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(7));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(5));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void max() throws Exception {
+        // A query that figures out what the oldest age is across all people.
+        final String sparql =
+                "SELECT (max(?age) as ?oldest) " +
+                "WHERE { " +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(13)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(14)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:age"), vf.createLiteral(5)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:age"), vf.createLiteral(25)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("oldest", vf.createLiteral(13));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("oldest", vf.createLiteral(14));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("oldest", vf.createLiteral(25));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void multipleGroupByVars() throws Exception {
+        // A query that contains more than one group by variable.
+        final String sparql =
+                "SELECT ?business ?employee (sum(?hours) AS ?totalHours) " +
+                "WHERE {" +
+                    "?employee <urn:worksAt> ?business . " +
+                    "?business <urn:hasTimecardId> ?timecardId . " +
+                    "?employee <urn:hasTimecardId> ?timecardId . " +
+                    "?timecardId <urn:hours> ?hours . " +
+                "} GROUP BY ?business ?employee";
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoJoint")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard1")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard1")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard1"), vf.createURI("urn:hours"), vf.createLiteral(40)), ""));
+
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard2")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard2")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard2"), vf.createURI("urn:hours"), vf.createLiteral(25)), ""));
+
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoJoint")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard3")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard3")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard3"), vf.createURI("urn:hours"), vf.createLiteral(28)), ""));
+
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard4")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard4")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard4"), vf.createURI("urn:hours"), vf.createLiteral(28)), ""));
+
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:CoffeeShop"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard5")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard5")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard5"), vf.createURI("urn:hours"), vf.createLiteral(12)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        bs.addBinding("employee", vf.createURI("urn:Alice"));
+        bs.addBinding("totalHours", vf.createLiteral("40", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        bs.addBinding("employee", vf.createURI("urn:Alice"));
+        bs.addBinding("totalHours", vf.createLiteral("65", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("totalHours", vf.createLiteral("28", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("totalHours", vf.createLiteral("56", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:CoffeeShop"));
+        bs.addBinding("employee", vf.createURI("urn:Alice"));
+        bs.addBinding("totalHours", vf.createLiteral("12", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 4000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void multipleAggregations() throws Exception {
+        // A query that figures out what the youngest and oldest ages are across all people.
+        final String sparql =
+                "SELECT (min(?age) as ?youngest) (max(?age) as ?oldest) " +
+                "WHERE { " +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(13)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(14)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:age"), vf.createLiteral(5)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:age"), vf.createLiteral(25)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(13));
+        bs.addBinding("oldest", vf.createLiteral(13));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(13));
+        bs.addBinding("oldest", vf.createLiteral(14));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(7));
+        bs.addBinding("oldest", vf.createLiteral(14));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(5));
+        bs.addBinding("oldest", vf.createLiteral(14));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(5));
+        bs.addBinding("oldest", vf.createLiteral(25));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+}
\ No newline at end of file


[41/50] [abbrv] incubator-rya git commit: RYA-377 Repackaged the common Aggregation code into the rya.api.functions project.

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/SparqlFluoQueryBuilder.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/SparqlFluoQueryBuilder.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/SparqlFluoQueryBuilder.java
index 7b21575..57ae9d2 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/SparqlFluoQueryBuilder.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/SparqlFluoQueryBuilder.java
@@ -42,13 +42,13 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.rya.api.client.CreatePCJ.ExportStrategy;
 import org.apache.rya.api.client.CreatePCJ.QueryType;
+import org.apache.rya.api.function.aggregation.AggregationElement;
+import org.apache.rya.api.function.aggregation.AggregationType;
 import org.apache.rya.indexing.pcj.fluo.app.ConstructGraph;
 import org.apache.rya.indexing.pcj.fluo.app.ConstructProjection;
 import org.apache.rya.indexing.pcj.fluo.app.FluoStringConverter;
 import org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants;
 import org.apache.rya.indexing.pcj.fluo.app.NodeType;
-import org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata.AggregationElement;
-import org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata.AggregationType;
 import org.apache.rya.indexing.pcj.fluo.app.query.JoinMetadata.JoinType;
 import org.apache.rya.indexing.pcj.fluo.app.util.FilterSerializer;
 import org.apache.rya.indexing.pcj.fluo.app.util.FilterSerializer.FilterParseException;
@@ -103,73 +103,73 @@ public class SparqlFluoQueryBuilder {
     private String queryId;
     private NodeIds nodeIds;
     private Optional<Integer> joinBatchSize = Optional.empty();
-  
-    //Default behavior is to export to Kafka - subject to change when user can 
+
+    //Default behavior is to export to Kafka - subject to change when user can
     //specify their own export strategy
     private Set<ExportStrategy> exportStrategies = new HashSet<>(Arrays.asList(ExportStrategy.KAFKA));
-    
-    public SparqlFluoQueryBuilder setSparql(String sparql) {
+
+    public SparqlFluoQueryBuilder setSparql(final String sparql) {
         this.sparql = Preconditions.checkNotNull(sparql);
         return this;
     }
-    
-    public SparqlFluoQueryBuilder setTupleExpr(TupleExpr te) {
+
+    public SparqlFluoQueryBuilder setTupleExpr(final TupleExpr te) {
         this.te = Preconditions.checkNotNull(te);
         return this;
     }
-    
+
     /**
      * Sets the FluoQuery id as generated by {@link NodeType#generateNewFluoIdForType(NodeType)} or
      * {@link NodeType#generateNewIdForType(NodeType, String)}, where NodeType is of type Query.
      * @param queryId for the {@link FluoQuery}
      * @return SparqlFluoQueryBuilder for chaining method calls
      */
-    public SparqlFluoQueryBuilder setFluoQueryId(String queryId) {
+    public SparqlFluoQueryBuilder setFluoQueryId(final String queryId) {
         this.queryId = Preconditions.checkNotNull(queryId);
         return this;
     }
-    
-    public SparqlFluoQueryBuilder setNodeIds(NodeIds nodeIds) {
+
+    public SparqlFluoQueryBuilder setNodeIds(final NodeIds nodeIds) {
         this.nodeIds = Preconditions.checkNotNull(nodeIds);
         return this;
     }
-    
-    public SparqlFluoQueryBuilder setExportStrategies(Set<ExportStrategy> exportStrategies) {
+
+    public SparqlFluoQueryBuilder setExportStrategies(final Set<ExportStrategy> exportStrategies) {
         this.exportStrategies = exportStrategies;
         return this;
     }
-    
-    public SparqlFluoQueryBuilder setJoinBatchSize(int joinBatchSize) {
-        Preconditions.checkArgument(joinBatchSize > 0); 
+
+    public SparqlFluoQueryBuilder setJoinBatchSize(final int joinBatchSize) {
+        Preconditions.checkArgument(joinBatchSize > 0);
         this.joinBatchSize = Optional.of(joinBatchSize);
         return this;
     }
-    
+
     public FluoQuery build() throws UnsupportedQueryException {
         Preconditions.checkNotNull(sparql);
         Preconditions.checkNotNull(queryId);
         Preconditions.checkNotNull(exportStrategies);
-      
+
         if(nodeIds == null) {
             nodeIds = new NodeIds();
         }
-        
+
         if(te == null) {
-            SPARQLParser parser = new SPARQLParser();
+            final SPARQLParser parser = new SPARQLParser();
             ParsedQuery pq;
             try {
                 pq = parser.parseQuery(sparql, null);
-            } catch (MalformedQueryException e) {
+            } catch (final MalformedQueryException e) {
                throw new RuntimeException(e);
             }
             te = pq.getTupleExpr();
         }
-        
+
         PeriodicQueryUtil.placePeriodicQueryNode(te);
-        String childNodeId = nodeIds.getOrMakeId(te);
-        
+        final String childNodeId = nodeIds.getOrMakeId(te);
+
         final FluoQuery.Builder fluoQueryBuilder = FluoQuery.builder();
-        QueryMetadata.Builder queryBuilder = QueryMetadata.builder(queryId);
+        final QueryMetadata.Builder queryBuilder = QueryMetadata.builder(queryId);
         //sets {@link QueryType} and VariableOrder
         setVarOrderAndQueryType(queryBuilder, te);
         queryBuilder
@@ -177,18 +177,18 @@ public class SparqlFluoQueryBuilder {
             .setChildNodeId(childNodeId)
             .setExportStrategies(exportStrategies)
             .setJoinBatchSize(joinBatchSize);
-        
+
         fluoQueryBuilder.setQueryMetadata(queryBuilder);
-        
+
         setChildMetadata(fluoQueryBuilder, childNodeId, queryBuilder.getVariableOrder(), queryId);
-        
+
         final NewQueryVisitor visitor = new NewQueryVisitor(fluoQueryBuilder, nodeIds);
         te.visit( visitor );
-        
+
         final FluoQuery fluoQuery = fluoQueryBuilder.build();
         return fluoQuery;
     }
-    
+
     /**
      * A data structure that creates and keeps track of Node IDs for the nodes
      * of a {@link ParsedQuery}. This structure should only be used while creating
@@ -275,7 +275,7 @@ public class SparqlFluoQueryBuilder {
             // Put them together to create the Node ID.
             return prefix + "_" + unique;
         }
-        
+
     }
 
     /**
@@ -327,7 +327,7 @@ public class SparqlFluoQueryBuilder {
                 } else {
                     groupByVariableOrder = new VariableOrder();
                 }
-                
+
 
                 // The aggregations that need to be performed are the Group Elements.
                 final List<AggregationElement> aggregations = new ArrayList<>();
@@ -361,15 +361,15 @@ public class SparqlFluoQueryBuilder {
 
                 aggregationBuilder.setChildNodeId(childNodeId);
                 aggregationBuilder.setGroupByVariableOrder(groupByVariableOrder);
-                
-                Set<String> aggregationVars = getVarsToDelete(groupByVariableOrder.getVariableOrders(), aggregationBuilder.getVariableOrder().getVariableOrders());
+
+                final Set<String> aggregationVars = getVarsToDelete(groupByVariableOrder.getVariableOrders(), aggregationBuilder.getVariableOrder().getVariableOrders());
                 FluoQueryUtils.updateVarOrders(fluoQueryBuilder, UpdateAction.DeleteVariable, Lists.newArrayList(aggregationVars), aggregationId);
-                
+
                 for(final AggregationElement aggregation : aggregations) {
                     aggregationBuilder.addAggregation(aggregation);
                 }
-                
-                
+
+
 
                 // Update the child node's metadata.
                 final Set<String> childVars = getVars(child);
@@ -459,7 +459,7 @@ public class SparqlFluoQueryBuilder {
 
         @Override
         public void meet(final Filter node) {
-            
+
             // Get or create a builder for this node populated with the known metadata.
             final String filterId = nodeIds.getOrMakeId(node);
 
@@ -472,7 +472,7 @@ public class SparqlFluoQueryBuilder {
             String filterString;
             try {
                 filterString = FilterSerializer.serialize(node);
-            } catch (FilterParseException e) {
+            } catch (final FilterParseException e) {
                 throw new RuntimeException(e);
             }
             filterBuilder.setFilterSparql(filterString);
@@ -493,10 +493,11 @@ public class SparqlFluoQueryBuilder {
             // Walk to the next node.
             super.meet(node);
         }
-        
+
+        @Override
         public void meetOther(final QueryModelNode qNode) {
             if (qNode instanceof PeriodicQueryNode) {
-                PeriodicQueryNode node = (PeriodicQueryNode) qNode;
+                final PeriodicQueryNode node = (PeriodicQueryNode) qNode;
                 // Get or create a builder for this node populated with the
                 // known metadata.
                 final String periodicId = nodeIds.getOrMakeId(node);
@@ -531,9 +532,9 @@ public class SparqlFluoQueryBuilder {
                 FluoQueryUtils.updateVarOrders(fluoQueryBuilder, UpdateAction.AddVariable, Arrays.asList(IncrementalUpdateConstants.PERIODIC_BIN_ID), periodicId);
                 // Walk to the next node.
                 node.getArg().visit(this);
-            } 
+            }
         }
-        
+
 
         @Override
         public void meet(final Projection node) {
@@ -564,25 +565,26 @@ public class SparqlFluoQueryBuilder {
             // Walk to the next node.
             super.meet(node);
         }
-        
-        
-        public void meet(Reduced node) {
-            //create id, initialize ConstructQueryMetadata builder, register ConstructQueryMetadata 
+
+
+        @Override
+        public void meet(final Reduced node) {
+            //create id, initialize ConstructQueryMetadata builder, register ConstructQueryMetadata
             //builder with FluoQueryBuilder, and add metadata that we currently have
             final String constructId = nodeIds.getOrMakeId(node);
-            
+
             ConstructQueryMetadata.Builder constructBuilder = fluoQueryBuilder.getConstructQueryBuilder().orNull();
             if(constructBuilder == null) {
                 constructBuilder = ConstructQueryMetadata.builder();
                 constructBuilder.setNodeId(constructId);
                 fluoQueryBuilder.setConstructQueryMetadata(constructBuilder);
             }
-            
+
             //get child node
             QueryModelNode child = node.getArg();
             Preconditions.checkArgument(child instanceof Projection || child instanceof MultiProjection);
-            UnaryTupleOperator unary = (UnaryTupleOperator) child;
-            
+            final UnaryTupleOperator unary = (UnaryTupleOperator) child;
+
             //get ProjectionElemList to build ConstructGraph
             final List<ProjectionElemList> projections = new ArrayList<>();
             if(unary instanceof Projection) {
@@ -590,15 +592,15 @@ public class SparqlFluoQueryBuilder {
             } else {
                 projections.addAll(((MultiProjection)unary).getProjections());
             }
-            
+
             //get ExtensionElems to build ConstructGraph
-            QueryModelNode grandChild = unary.getArg();
+            final QueryModelNode grandChild = unary.getArg();
             Preconditions.checkArgument(grandChild instanceof Extension);
-            Extension extension = (Extension) grandChild;
+            final Extension extension = (Extension) grandChild;
             final List<ExtensionElem> extensionElems = extension.getElements();
             final ConstructGraph graph = getConstructGraph(projections, extensionElems);
             constructBuilder.setConstructGraph(graph);
-            
+
             //set child to the next node we care about in Fluo
             //if Extension's arg is a Group node, then it is an Aggregation, so set child to Extension
             //otherwise set child to Extension's child (only care about Extensions if they are Aggregations)
@@ -607,42 +609,42 @@ public class SparqlFluoQueryBuilder {
             } else {
                 child = extension.getArg();
             }
-            
+
             //Set the child node in the ConstructQueryMetadataBuilder
-            String childNodeId = nodeIds.getOrMakeId(child);
+            final String childNodeId = nodeIds.getOrMakeId(child);
             constructBuilder.setChildNodeId(childNodeId);
-            
+
             // Update the child node's metadata.
             final Set<String> childVars = getVars((TupleExpr)child);
             final VariableOrder childVarOrder = new VariableOrder(childVars);
             setChildMetadata(fluoQueryBuilder, childNodeId, childVarOrder, constructId);
-            
+
             //fast forward visitor to next node we care about
             child.visit(this);
         }
-        
-        private ConstructGraph getConstructGraph(List<ProjectionElemList> projections, List<ExtensionElem> extensionElems) {
-            Map<String, Value> valueMap = new HashMap<>();
+
+        private ConstructGraph getConstructGraph(final List<ProjectionElemList> projections, final List<ExtensionElem> extensionElems) {
+            final Map<String, Value> valueMap = new HashMap<>();
             //create valueMap to associate source names with Values
-            for(ExtensionElem elem: extensionElems) {
-                String name = elem.getName();
-                ValueExpr expr = elem.getExpr();
+            for(final ExtensionElem elem: extensionElems) {
+                final String name = elem.getName();
+                final ValueExpr expr = elem.getExpr();
                 if(expr instanceof ValueConstant) {
-                    Value value = ((ValueConstant) expr).getValue();
+                    final Value value = ((ValueConstant) expr).getValue();
                     valueMap.put(name, value);
                 } else if(expr instanceof BNodeGenerator) {
                     valueMap.put(name, new BNodeImpl(UUID.randomUUID().toString()));
                 }
             }
-            
-            Set<ConstructProjection> constructProj = new HashSet<>();
+
+            final Set<ConstructProjection> constructProj = new HashSet<>();
             //build ConstructProjection for each ProjectionElemList
-            for(ProjectionElemList list: projections) {
+            for(final ProjectionElemList list: projections) {
                 validateProjectionElemList(list);
-                List<Var> vars = new ArrayList<>();
-                for(ProjectionElem elem: list.getElements()) {
-                    String sourceName = elem.getSourceName();
-                    Var var = new Var(sourceName);
+                final List<Var> vars = new ArrayList<>();
+                for(final ProjectionElem elem: list.getElements()) {
+                    final String sourceName = elem.getSourceName();
+                    final Var var = new Var(sourceName);
                     if(valueMap.containsKey(sourceName)) {
                         var.setValue(valueMap.get(sourceName));
                     }
@@ -650,25 +652,25 @@ public class SparqlFluoQueryBuilder {
                 }
                 constructProj.add(new ConstructProjection(vars.get(0), vars.get(1), vars.get(2)));
             }
-            
+
             return new ConstructGraph(constructProj);
         }
-        
-        private Set<String> getVarsToDelete(Collection<String> groupByVars, Collection<String> varOrderVars) {
-            Set<String> groupBySet = Sets.newHashSet(groupByVars);
-            Set<String> varOrderSet = Sets.newHashSet(varOrderVars);
-            
+
+        private Set<String> getVarsToDelete(final Collection<String> groupByVars, final Collection<String> varOrderVars) {
+            final Set<String> groupBySet = Sets.newHashSet(groupByVars);
+            final Set<String> varOrderSet = Sets.newHashSet(varOrderVars);
+
             return Sets.difference(varOrderSet, groupBySet);
         }
-        
-        private void validateProjectionElemList(ProjectionElemList list) {
-            List<ProjectionElem> elements = list.getElements();
+
+        private void validateProjectionElemList(final ProjectionElemList list) {
+            final List<ProjectionElem> elements = list.getElements();
             checkArgument(elements.size() == 3);
             checkArgument(elements.get(0).getTargetName().equals("subject"));
             checkArgument(elements.get(1).getTargetName().equals("predicate"));
             checkArgument(elements.get(2).getTargetName().equals("object"));
         }
-        
+
         /**
          * Get the non-constant variables from a {@link TupleExpr}.
          *
@@ -769,55 +771,58 @@ public class SparqlFluoQueryBuilder {
             return shifted;
         }
     }
-    
-    private void setVarOrderAndQueryType(QueryMetadata.Builder builder, TupleExpr te) {
-        QueryMetadataLocator locator = new QueryMetadataLocator();
+
+    private void setVarOrderAndQueryType(final QueryMetadata.Builder builder, final TupleExpr te) {
+        final QueryMetadataLocator locator = new QueryMetadataLocator();
         try {
             te.visit(locator);
-        } catch (Exception e) {
+        } catch (final Exception e) {
             throw new RuntimeException(e);
         }
-        
+
         builder.setVarOrder(locator.getVarOrder());
         builder.setQueryType(locator.getQueryType());
     }
-    
+
     public static class QueryMetadataLocator extends QueryModelVisitorBase<Exception> {
-        
+
         private VariableOrder varOrder;
         private QueryType queryType;
-        
+
         public VariableOrder getVarOrder() {
             return varOrder;
         }
-        
+
         public QueryType getQueryType() {
             return queryType;
         }
-        
-        public void meet(Projection node) throws Exception {
-            Set<String> bindingNames = node.getBindingNames();
+
+        @Override
+        public void meet(final Projection node) throws Exception {
+            final Set<String> bindingNames = node.getBindingNames();
             if(varOrder == null) {
                 varOrder = new VariableOrder(bindingNames);
             }
-            
+
             if(queryType == null) {
                 queryType = QueryType.PROJECTION;
             }
             super.meet(node);
         }
-        
-        public void meet(Reduced node) throws Exception {
+
+        @Override
+        public void meet(final Reduced node) throws Exception {
             if(varOrder == null) {
                 varOrder = getConstructGraphVarOrder(node);
             }
-            
+
             if(queryType == null) {
                 queryType = QueryType.CONSTRUCT;
             }
             super.meet(node);
         }
-        
+
+        @Override
         public void meetOther(final QueryModelNode node) throws Exception {
             if (node instanceof PeriodicQueryNode) {
                 queryType = QueryType.PERIODIC;
@@ -826,14 +831,14 @@ public class SparqlFluoQueryBuilder {
             }
         }
     }
-    
-    private static VariableOrder getConstructGraphVarOrder(Reduced node) {
-        
+
+    private static VariableOrder getConstructGraphVarOrder(final Reduced node) {
+
         //get child node
-          QueryModelNode child = node.getArg();
+          final QueryModelNode child = node.getArg();
           Preconditions.checkArgument(child instanceof Projection || child instanceof MultiProjection);
-          UnaryTupleOperator unary = (UnaryTupleOperator) child;
-          
+          final UnaryTupleOperator unary = (UnaryTupleOperator) child;
+
           //get ProjectionElemList to build ConstructGraph
           final List<ProjectionElemList> projections = new ArrayList<>();
           if(unary instanceof Projection) {
@@ -841,26 +846,26 @@ public class SparqlFluoQueryBuilder {
           } else {
               projections.addAll(((MultiProjection)unary).getProjections());
           }
-          
+
           return getConstructGraphVarOrder(projections);
       }
-    
-    private static VariableOrder getConstructGraphVarOrder(List<ProjectionElemList> projections) {
-        Set<String> varOrders = new HashSet<>();
-        
-        for(ProjectionElemList elems: projections) {
-            for(ProjectionElem elem: elems.getElements()) {
-                String name = elem.getSourceName();
+
+    private static VariableOrder getConstructGraphVarOrder(final List<ProjectionElemList> projections) {
+        final Set<String> varOrders = new HashSet<>();
+
+        for(final ProjectionElemList elems: projections) {
+            for(final ProjectionElem elem: elems.getElements()) {
+                final String name = elem.getSourceName();
                 if(!name.startsWith("-const-") && !name.startsWith("-anon-")) {
                     varOrders.add(name);
                 }
             }
         }
-        
+
         return new VariableOrder(varOrders);
     }
-    
-    
+
+
     /**
      * Update a query node's metadata to include it's binding set variable order
      * and it's parent node id. This information is only known when handling
@@ -921,21 +926,21 @@ public class SparqlFluoQueryBuilder {
             aggregationBuilder.setVarOrder(childVarOrder);
             aggregationBuilder.setParentNodeId(parentNodeId);
             break;
-            
+
         case PROJECTION:
             ProjectionMetadata.Builder projectionBuilder = fluoQueryBuilder.getProjectionBuilder(childNodeId).orNull();
             if(projectionBuilder == null) {
                 projectionBuilder = ProjectionMetadata.builder(childNodeId);
                 fluoQueryBuilder.addProjectionBuilder(projectionBuilder);
             }
-            
+
             projectionBuilder.setVarOrder(childVarOrder);
             projectionBuilder.setParentNodeId(parentNodeId);
             break;
-            
+
         case QUERY:
             throw new IllegalArgumentException("A QUERY node cannot be the child of another node.");
-        
+
         case CONSTRUCT:
             ConstructQueryMetadata.Builder constructBuilder = fluoQueryBuilder.getConstructQueryBuilder().orNull();
             if(constructBuilder == null) {
@@ -943,12 +948,12 @@ public class SparqlFluoQueryBuilder {
                 constructBuilder.setNodeId(childNodeId);
                 fluoQueryBuilder.setConstructQueryMetadata(constructBuilder);
             }
-            
+
             Preconditions.checkArgument(childNodeId.equals(constructBuilder.getNodeId()));
             constructBuilder.setVarOrder(childVarOrder);
             constructBuilder.setParentNodeId(parentNodeId);
             break;
-        
+
         case PERIODIC_QUERY:
             PeriodicQueryMetadata.Builder periodicQueryBuilder = fluoQueryBuilder.getPeriodicQueryBuilder().orNull();
             if (periodicQueryBuilder == null) {
@@ -959,7 +964,7 @@ public class SparqlFluoQueryBuilder {
             periodicQueryBuilder.setVarOrder(childVarOrder);
             periodicQueryBuilder.setParentNodeId(parentNodeId);
             break;
-            
+
         default:
             throw new IllegalArgumentException("Unsupported NodeType: " + childType);
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAOIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAOIT.java b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAOIT.java
index 45492de..384ec5d 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAOIT.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAOIT.java
@@ -31,10 +31,10 @@ import org.apache.fluo.api.client.Snapshot;
 import org.apache.fluo.api.client.Transaction;
 import org.apache.rya.api.client.CreatePCJ.ExportStrategy;
 import org.apache.rya.api.client.CreatePCJ.QueryType;
+import org.apache.rya.api.function.aggregation.AggregationElement;
+import org.apache.rya.api.function.aggregation.AggregationType;
 import org.apache.rya.indexing.pcj.fluo.app.ConstructGraph;
 import org.apache.rya.indexing.pcj.fluo.app.NodeType;
-import org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata.AggregationElement;
-import org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata.AggregationType;
 import org.apache.rya.indexing.pcj.fluo.app.query.JoinMetadata.JoinType;
 import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
 import org.apache.rya.pcj.fluo.test.base.RyaExportITBase;
@@ -146,7 +146,7 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
         final FluoQueryMetadataDAO dao = new FluoQueryMetadataDAO();
 
         // Create the object that will be serialized.
-        String queryId = NodeType.generateNewFluoIdForType(NodeType.QUERY);
+        final String queryId = NodeType.generateNewFluoIdForType(NodeType.QUERY);
         final QueryMetadata.Builder builder = QueryMetadata.builder(queryId);
         builder.setQueryType(QueryType.PROJECTION);
         builder.setVarOrder(new VariableOrder("y;s;d"));
@@ -172,7 +172,7 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
             assertEquals(originalMetadata, storedMetdata);
         }
     }
-    
+
     @Test
     public void projectionMetadataTest() {
         final FluoQueryMetadataDAO dao = new FluoQueryMetadataDAO();
@@ -202,15 +202,15 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
             assertEquals(originalMetadata, storedMetdata);
         }
     }
-    
+
     @Test
     public void constructQueryMetadataTest() throws MalformedQueryException {
-        
-        String query = "select ?x ?y where {?x <uri:p1> ?y. ?y <uri:p2> <uri:o1> }";
-        SPARQLParser parser = new SPARQLParser();
-        ParsedQuery pq = parser.parseQuery(query, null);
-        List<StatementPattern> patterns = StatementPatternCollector.process(pq.getTupleExpr());
-        
+
+        final String query = "select ?x ?y where {?x <uri:p1> ?y. ?y <uri:p2> <uri:o1> }";
+        final SPARQLParser parser = new SPARQLParser();
+        final ParsedQuery pq = parser.parseQuery(query, null);
+        final List<StatementPattern> patterns = StatementPatternCollector.process(pq.getTupleExpr());
+
         final FluoQueryMetadataDAO dao = new FluoQueryMetadataDAO();
 
         // Create the object that will be serialized.
@@ -301,13 +301,13 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
             assertEquals(originalMetadata, storedMetadata);
         }
     }
-    
+
     @Test
     public void periodicQueryMetadataTest() {
         final FluoQueryMetadataDAO dao = new FluoQueryMetadataDAO();
 
         // Create the object that will be serialized.
-        PeriodicQueryMetadata originalMetadata =  PeriodicQueryMetadata.builder()
+        final PeriodicQueryMetadata originalMetadata =  PeriodicQueryMetadata.builder()
             .setNodeId("nodeId")
             .setParentNodeId("parentNodeId")
             .setVarOrder(new VariableOrder("a","b","c"))
@@ -317,7 +317,7 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
             .setUnit(TimeUnit.DAYS)
             .setTemporalVariable("a")
             .build();
-            
+
 
         try(FluoClient fluoClient = FluoFactory.newClient(super.getFluoConfiguration())) {
             // Write it to the Fluo table.
@@ -352,14 +352,14 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
                   "?worker <http://worksAt> <http://Chipotle>. " +
                 "}";
 
-        SparqlFluoQueryBuilder builder = new SparqlFluoQueryBuilder();
+        final SparqlFluoQueryBuilder builder = new SparqlFluoQueryBuilder();
         builder.setSparql(sparql);
         builder.setFluoQueryId(NodeType.generateNewFluoIdForType(NodeType.QUERY));
         final FluoQuery originalQuery = builder.build();
 
         assertEquals(QueryType.PROJECTION, originalQuery.getQueryType());
         assertEquals(false, originalQuery.getConstructQueryMetadata().isPresent());
-        
+
         try(FluoClient fluoClient = FluoFactory.newClient(super.getFluoConfiguration())) {
             // Write it to the Fluo table.
             try(Transaction tx = fluoClient.newTransaction()) {
@@ -377,7 +377,7 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
             assertEquals(originalQuery, storedQuery);
         }
     }
-    
+
     @Test
     public void fluoConstructQueryTest() throws MalformedQueryException, UnsupportedQueryException {
         final FluoQueryMetadataDAO dao = new FluoQueryMetadataDAO();
@@ -393,11 +393,11 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
                   "?worker <http://worksAt> <http://Chipotle>. " +
                 "}";
 
-        SparqlFluoQueryBuilder builder = new SparqlFluoQueryBuilder();
+        final SparqlFluoQueryBuilder builder = new SparqlFluoQueryBuilder();
         builder.setSparql(sparql);
         builder.setFluoQueryId(NodeType.generateNewFluoIdForType(NodeType.QUERY));
         final FluoQuery originalQuery = builder.build();
-        
+
         assertEquals(QueryType.CONSTRUCT, originalQuery.getQueryType());
         assertEquals(true, originalQuery.getConstructQueryMetadata().isPresent());
 
@@ -418,8 +418,8 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
             assertEquals(originalQuery, storedQuery);
         }
     }
-    
-    
+
+
     @Test
     public void fluoNestedQueryTest() throws MalformedQueryException, UnsupportedQueryException {
         final FluoQueryMetadataDAO dao = new FluoQueryMetadataDAO();
@@ -435,13 +435,13 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
                     "?id <urn:price> ?price ." +
                 "} " +
                 "GROUP BY ?type ?location }}";
-        
-        
-        SparqlFluoQueryBuilder builder = new SparqlFluoQueryBuilder();
+
+
+        final SparqlFluoQueryBuilder builder = new SparqlFluoQueryBuilder();
         builder.setSparql(sparql);
         builder.setFluoQueryId(NodeType.generateNewFluoIdForType(NodeType.QUERY));
         final FluoQuery originalQuery = builder.build();
-        
+
         assertEquals(QueryType.PROJECTION, originalQuery.getQueryType());
 
         try(FluoClient fluoClient = FluoFactory.newClient(super.getFluoConfiguration())) {
@@ -461,7 +461,7 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
             assertEquals(originalQuery, storedQuery);
         }
     }
-    
+
     @Test
     public void fluoNestedConstructQueryTest() throws MalformedQueryException, UnsupportedQueryException {
         final FluoQueryMetadataDAO dao = new FluoQueryMetadataDAO();
@@ -478,16 +478,16 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
                 + "SELECT ?location (count(?obs) AS ?obsCount) (avg(?velocity) AS ?avgVelocity) "
                 + "WHERE { "
                 + "FILTER(?velocity > 75) "
-                + "?obs <urn:hasVelocity> ?velocity. " 
-                + "?obs <urn:hasLocation> ?location. " 
+                + "?obs <urn:hasVelocity> ?velocity. "
+                + "?obs <urn:hasLocation> ?location. "
                 + "}GROUP BY ?location }}";
-        
-        
-        SparqlFluoQueryBuilder builder = new SparqlFluoQueryBuilder();
+
+
+        final SparqlFluoQueryBuilder builder = new SparqlFluoQueryBuilder();
         builder.setSparql(sparql);
         builder.setFluoQueryId(NodeType.generateNewFluoIdForType(NodeType.QUERY));
         final FluoQuery originalQuery = builder.build();
-        
+
         assertEquals(QueryType.CONSTRUCT, originalQuery.getQueryType());
 
         try(FluoClient fluoClient = FluoFactory.newClient(super.getFluoConfiguration())) {
@@ -507,5 +507,5 @@ public class FluoQueryMetadataDAOIT extends RyaExportITBase {
             assertEquals(originalQuery, storedQuery);
         }
     }
-    
+
 }
\ No newline at end of file


[39/50] [abbrv] incubator-rya git commit: RYA-377 After temporal isntant function

Posted by ca...@apache.org.
RYA-377 After temporal isntant function


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

Branch: refs/heads/master
Commit: 9e02a541c848defd027ff737f6149b66b73273b2
Parents: 5603a10
Author: Andrew Smith <sm...@gmail.com>
Authored: Fri Dec 1 13:30:25 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../function/temporal/AfterTemporalInstant.java | 46 ++++++++++++
 ...f.query.algebra.evaluation.function.Function |  1 +
 .../temporal/AfterTemporalFunctionsTest.java    | 75 ++++++++++++++++++++
 .../AggregationProcessorSupplier.java           |  2 +-
 .../processors/filter/TemporalFilterIT.java     | 72 ++++++++++++++++++-
 5 files changed, 193 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e02a541/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/AfterTemporalInstant.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/AfterTemporalInstant.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/AfterTemporalInstant.java
new file mode 100644
index 0000000..38c02ba
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/AfterTemporalInstant.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rya.api.function.temporal;
+
+import java.time.ZonedDateTime;
+import java.util.Objects;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Filter function in a SPARQL query used to filter when a point of time is after another.
+ */
+@DefaultAnnotation(NonNull.class)
+public class AfterTemporalInstant extends TemporalRelationFunction {
+    public static final String URI = BASE_URI + "after";
+
+    @Override
+    public String getURI() {
+        return URI;
+    }
+
+    @Override
+    protected boolean relation(final ZonedDateTime d1, final ZonedDateTime d2) {
+        Objects.requireNonNull(d1);
+        Objects.requireNonNull(d2);
+        return d1.isAfter(d2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e02a541/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function b/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
index 3cb1c56..2ec01d5 100644
--- a/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
+++ b/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
@@ -18,3 +18,4 @@
 #
 org.apache.rya.api.function.temporal.EqualsTemporal
 org.apache.rya.api.function.temporal.BeforeTemporalInstant
+org.apache.rya.api.function.temporal.AfterTemporalInstant

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e02a541/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/AfterTemporalFunctionsTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/AfterTemporalFunctionsTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/AfterTemporalFunctionsTest.java
new file mode 100644
index 0000000..f5f18f7
--- /dev/null
+++ b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/AfterTemporalFunctionsTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.temporal;
+
+import static org.junit.Assert.assertEquals;
+
+import java.time.ZonedDateTime;
+
+import org.junit.Test;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+public class AfterTemporalFunctionsTest {
+    private static final ZonedDateTime TIME = ZonedDateTime.parse("2015-12-30T12:00:00Z");
+    private static final ZonedDateTime TIME_10 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
+    private static final ZonedDateTime TIME_20 = ZonedDateTime.parse("2015-12-30T12:00:20Z");
+
+    final ValueFactory VF = ValueFactoryImpl.getInstance();
+
+    @Test
+    public void testAfter_same() throws Exception {
+        final AfterTemporalInstant function = new AfterTemporalInstant();
+
+        // 2 times equal
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+
+    @Test
+    public void testAfter_before() throws Exception {
+        final AfterTemporalInstant function = new AfterTemporalInstant();
+
+        // 2 times equal
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME_10.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+
+    @Test
+    public void testAfter_after() throws Exception {
+        final AfterTemporalInstant function = new AfterTemporalInstant();
+
+        // 2 times equal
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME_20.toString());
+        args[1] = VF.createLiteral(TIME.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(true), rez);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e02a541/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorSupplier.java
index c8e1049..c101914 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorSupplier.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorSupplier.java
@@ -39,7 +39,7 @@ import org.openrdf.query.algebra.Group;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9e02a541/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
index 837b57b..cac9559 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
@@ -74,8 +74,8 @@ public class TemporalFilterIT {
             }
         }
 
-        // There are 2 temporal functions registered, ensure that there are 2.
-        assertEquals(2, count);
+        // There are 3 temporal functions registered, ensure that there are 3.
+        assertEquals(3, count);
     }
 
     @Test
@@ -111,6 +111,74 @@ public class TemporalFilterIT {
         // Run the test.
         RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
+    
+    @Test
+    public void showBeforeWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX time: <http://www.w3.org/2006/time/> \n"
+            + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+            + "SELECT * \n"
+            + "WHERE { \n"
+            + "  <urn:time> time:atTime ?date .\n"
+            + " FILTER(tempf:before(?date, \"" + TIME_10.toString() + "\")) "
+            + "}";
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("date", vf.createLiteral(TIME.toString()));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+    
+    @Test
+    public void showAfterWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX time: <http://www.w3.org/2006/time/> \n"
+            + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+            + "SELECT * \n"
+            + "WHERE { \n"
+            + "  <urn:time> time:atTime ?date .\n"
+            + " FILTER(tempf:after(?date, \"" + TIME_10.toString() + "\")) "
+            + "}";
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("date", vf.createLiteral(TIME_20.toString()));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
 
     private List<VisibilityStatement> getStatements() throws Exception {
         final List<VisibilityStatement> statements = new ArrayList<>();


[28/50] [abbrv] incubator-rya git commit: rya-377 Fixing broken build.

Posted by ca...@apache.org.
rya-377 Fixing broken build.


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

Branch: refs/heads/master
Commit: 1535b4620c821aa171f2b5f26646c400344c5363
Parents: 9cd0c56
Author: kchilton2 <ke...@gmail.com>
Authored: Thu Nov 30 16:37:36 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../apache/rya/streams/kafka/processors/filter/GeoFilterIT.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/1535b462/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
index 4057e18..b9d81ef 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
@@ -32,8 +32,8 @@ import org.apache.rya.api.function.projection.RandomUUIDFactory;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.indexing.GeoConstants;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
 import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.topology.TopologyFactory;
@@ -117,7 +117,7 @@ public class GeoFilterIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     private List<VisibilityStatement> getStatements() throws Exception {


[43/50] [abbrv] incubator-rya git commit: RYA-377 Implement a command for running a Rya Streams query out of the command line client.

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
deleted file mode 100644
index 8898284..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/KafkaTestUtil.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka;
-
-import static java.util.Objects.requireNonNull;
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.serialization.Deserializer;
-import org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.apache.kafka.streams.KafkaStreams;
-import org.apache.kafka.streams.StreamsConfig;
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
-import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-
-import com.google.common.collect.Sets;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-
-/**
- * A set of utility functions that are useful when writing tests against a Kafka instance.
- */
-@DefaultAnnotation(NonNull.class)
-public final class KafkaTestUtil {
-
-    private KafkaTestUtil() { }
-
-    /**
-     * Create a {@link Producer} that is able to write to a topic that is hosted within an embedded instance of Kafka.
-     *
-     * @param kafka - The Kafka rule used to connect to the embedded Kafkfa instance. (not null)
-     * @param keySerializerClass - Serializes the keys. (not null)
-     * @param valueSerializerClass - Serializes the values. (not null)
-     * @return A {@link Producer} that can be used to write records to a topic.
-     */
-    public static <K, V> Producer<K, V> makeProducer(
-            final KafkaTestInstanceRule kafka,
-            final Class<? extends Serializer<K>> keySerializerClass,
-            final Class<? extends Serializer<V>> valueSerializerClass) {
-        requireNonNull(kafka);
-        requireNonNull(keySerializerClass);
-        requireNonNull(valueSerializerClass);
-
-        final Properties props = kafka.createBootstrapServerConfig();
-        props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass.getName());
-        props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName());
-        return new KafkaProducer<>(props);
-    }
-
-    /**
-     * Create a {@link Consumer} that has a unique group ID and reads everything from a topic that is hosted within an
-     * embedded instance of Kafka starting at the earliest point by default.
-     *
-     * @param kafka - The Kafka rule used to connect to the embedded Kafkfa instance. (not null)
-     * @param keyDeserializerClass - Deserializes the keys. (not null)
-     * @param valueDeserializerClass - Deserializes the values. (not null)
-     * @return A {@link Consumer} that can be used to read records from a topic.
-     */
-    public static <K, V> Consumer<K, V> fromStartConsumer(
-            final KafkaTestInstanceRule kafka,
-            final Class<? extends Deserializer<K>> keyDeserializerClass,
-            final Class<? extends Deserializer<V>> valueDeserializerClass) {
-        requireNonNull(kafka);
-        requireNonNull(keyDeserializerClass);
-        requireNonNull(valueDeserializerClass);
-
-        final Properties props = kafka.createBootstrapServerConfig();
-        props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
-        props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
-        props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass.getName());
-        props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass.getName());
-        return new KafkaConsumer<>(props);
-    }
-
-    /**
-     * Polls a {@link Consumer> until it has either polled too many times without hitting the target number
-     * of results, or it hits the target number of results.
-     *
-     * @param pollMs - How long each poll could take.
-     * @param pollIterations - The maximum number of polls that will be attempted.
-     * @param targetSize - The number of results to read before stopping.
-     * @param consumer - The consumer that will be polled.
-     * @return The results that were read frmo the consumer.
-     * @throws Exception If the poll failed.
-     */
-    public static <K, V> List<V> pollForResults(
-            final int pollMs,
-            final int pollIterations,
-            final int targetSize,
-            final Consumer<K, V> consumer) throws Exception {
-        requireNonNull(consumer);
-
-        final List<V> values = new ArrayList<>();
-
-        int i = 0;
-        while(values.size() < targetSize && i < pollIterations) {
-            for(final ConsumerRecord<K, V> record : consumer.poll(pollMs)) {
-                values.add( record.value() );
-            }
-            i++;
-        }
-
-        return values;
-    }
-
-    /**
-     * Runs a Kafka Streams topology, loads statements into the input topic, read the binding sets that come out of
-     * the results topic, and ensures the expected results match the read results.
-     *
-     * @param <T> The type of value that will be consumed from the results topic.
-     * @param kafka - The embedded Kafka instance that is being tested with. (not null)
-     * @param statementsTopic - The topic statements will be written to. (not null)
-     * @param resultsTopic - The topic results will be read from. (not null)
-     * @param builder - The streams topology that will be executed. (not null)
-     * @param startupMs - How long to wait for the topology to start before writing the statements.
-     * @param statements - The statements that will be loaded into the topic. (not null)
-     * @param expected - The expected results. (not null)
-     * @param expectedDeserializerClass - The class of the deserializer that will be used when reading
-     *   values from the results topic. (not null)
-     * @throws Exception If any exception was thrown while running the test.
-     */
-    public static <T> void runStreamProcessingTest(
-            final KafkaTestInstanceRule kafka,
-            final String statementsTopic,
-            final String resultsTopic,
-            final TopologyBuilder builder,
-            final int startupMs,
-            final List<VisibilityStatement> statements,
-            final Set<T> expected,
-            final Class<? extends Deserializer<T>> expectedDeserializerClass) throws Exception {
-        requireNonNull(kafka);
-        requireNonNull(statementsTopic);
-        requireNonNull(resultsTopic);
-        requireNonNull(builder);
-        requireNonNull(statements);
-        requireNonNull(expected);
-        requireNonNull(expectedDeserializerClass);
-
-        // Explicitly create the topics that are being used.
-        kafka.createTopic(statementsTopic);
-        kafka.createTopic(resultsTopic);
-
-        // Start the streams program.
-        final Properties props = kafka.createBootstrapServerConfig();
-        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "StatementPatternProcessorIT");
-
-        final KafkaStreams streams = new KafkaStreams(builder, new StreamsConfig(props));
-        streams.cleanUp();
-        try {
-            streams.start();
-
-            // Wait for the streams application to start. Streams only see data after their consumers are connected.
-            Thread.sleep(startupMs);
-
-            // Load the statements into the input topic.
-            try(Producer<String, VisibilityStatement> producer = KafkaTestUtil.makeProducer(
-                    kafka, StringSerializer.class, VisibilityStatementSerializer.class)) {
-                new KafkaLoadStatements(statementsTopic, producer).fromCollection(statements);
-            }
-
-            // Wait for the final results to appear in the output topic and verify the expected Binding Sets were found.
-            try(Consumer<String, T> consumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, expectedDeserializerClass)) {
-                // Register the topic.
-                consumer.subscribe(Arrays.asList(resultsTopic));
-
-                // Poll for the result.
-                final Set<T> results = Sets.newHashSet( KafkaTestUtil.pollForResults(500, 6, expected.size(), consumer) );
-
-                // Show the correct binding sets results from the job.
-                assertEquals(expected, results);
-            }
-        } finally {
-            streams.close();
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
new file mode 100644
index 0000000..b7e2be2
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka;
+
+import static java.util.Objects.requireNonNull;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
+
+import com.google.common.collect.Sets;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Utility functions that make it easier to test Rya Streams applications.
+ */
+@DefaultAnnotation(NonNull.class)
+public class RyaStreamsTestUtil {
+
+    /**
+     * Runs a Kafka Streams topology, loads statements into the input topic, read the binding sets that come out of
+     * the results topic, and ensures the expected results match the read results.
+     *
+     * @param <T> The type of value that will be consumed from the results topic.
+     * @param kafka - The embedded Kafka instance that is being tested with. (not null)
+     * @param statementsTopic - The topic statements will be written to. (not null)
+     * @param resultsTopic - The topic results will be read from. (not null)
+     * @param builder - The streams topology that will be executed. (not null)
+     * @param startupMs - How long to wait for the topology to start before writing the statements.
+     * @param statements - The statements that will be loaded into the topic. (not null)
+     * @param expected - The expected results. (not null)
+     * @param expectedDeserializerClass - The class of the deserializer that will be used when reading
+     *   values from the results topic. (not null)
+     * @throws Exception If any exception was thrown while running the test.
+     */
+    public static <T> void runStreamProcessingTest(
+            final KafkaTestInstanceRule kafka,
+            final String statementsTopic,
+            final String resultsTopic,
+            final TopologyBuilder builder,
+            final int startupMs,
+            final List<VisibilityStatement> statements,
+            final Set<T> expected,
+            final Class<? extends Deserializer<T>> expectedDeserializerClass) throws Exception {
+        requireNonNull(kafka);
+        requireNonNull(statementsTopic);
+        requireNonNull(resultsTopic);
+        requireNonNull(builder);
+        requireNonNull(statements);
+        requireNonNull(expected);
+        requireNonNull(expectedDeserializerClass);
+
+        // Explicitly create the topics that are being used.
+        kafka.createTopic(statementsTopic);
+        kafka.createTopic(resultsTopic);
+
+        // Start the streams program.
+        final Properties props = kafka.createBootstrapServerConfig();
+        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "StatementPatternProcessorIT");
+
+        final KafkaStreams streams = new KafkaStreams(builder, new StreamsConfig(props));
+        streams.cleanUp();
+        try {
+            streams.start();
+
+            // Wait for the streams application to start. Streams only see data after their consumers are connected.
+            Thread.sleep(startupMs);
+
+            // Load the statements into the input topic.
+            try(Producer<String, VisibilityStatement> producer = KafkaTestUtil.makeProducer(
+                    kafka, StringSerializer.class, VisibilityStatementSerializer.class)) {
+                new KafkaLoadStatements(statementsTopic, producer).fromCollection(statements);
+            }
+
+            // Wait for the final results to appear in the output topic and verify the expected Binding Sets were found.
+            try(Consumer<String, T> consumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, expectedDeserializerClass)) {
+                // Register the topic.
+                consumer.subscribe(Arrays.asList(resultsTopic));
+
+                // Poll for the result.
+                final Set<T> results = Sets.newHashSet( KafkaTestUtil.pollForResults(500, 6, expected.size(), consumer) );
+
+                // Show the correct binding sets results from the job.
+                assertEquals(expected, results);
+            }
+        } finally {
+            streams.close();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
index 67889e9..c740ba2 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
@@ -31,10 +31,10 @@ import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.api.entity.QueryResultStream;
 import org.apache.rya.streams.api.interactor.GetQueryResultStream;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.Rule;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
index b48addd..7bfa560 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatementsIT.java
@@ -31,11 +31,10 @@ import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
-import org.apache.rya.test.kafka.KafkaITBase;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.Rule;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;
@@ -45,7 +44,7 @@ import org.openrdf.rio.UnsupportedRDFormatException;
 /**
  * Integration tests the {@link KafkaLoadStatements} command
  */
-public class KafkaLoadStatementsIT extends KafkaITBase {
+public class KafkaLoadStatementsIT {
     private static final Path TURTLE_FILE = Paths.get("src/test/resources/statements.ttl");
 
     private static final Path INVALID = Paths.get("src/test/resources/invalid.INVALID");

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaRunQueryIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaRunQueryIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaRunQueryIT.java
new file mode 100644
index 0000000..33b3a92
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaRunQueryIT.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.interactor;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.api.entity.StreamsQuery;
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+import org.apache.rya.streams.api.interactor.LoadStatements;
+import org.apache.rya.streams.api.interactor.RunQuery;
+import org.apache.rya.streams.api.queries.InMemoryQueryChangeLog;
+import org.apache.rya.streams.api.queries.InMemoryQueryRepository;
+import org.apache.rya.streams.api.queries.QueryRepository;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Integration tests the methods of {@link KafkaRunQuery}.
+ */
+public class KafkaRunQueryIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    private Producer<String, VisibilityStatement> producer;
+    private Consumer<String, VisibilityBindingSet> consumer;
+
+    @Before
+    public void setup() {
+        producer = KafkaTestUtil.makeProducer(kafka, StringSerializer.class, VisibilityStatementSerializer.class);
+        consumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, VisibilityBindingSetDeserializer.class);
+    }
+
+    @After
+    public void cleanup() {
+        producer.close();
+        consumer.close();
+    }
+
+    @Test
+    public void runQuery() throws Exception {
+        // Setup some constant that will be used through the test.
+        final String ryaInstance = UUID.randomUUID().toString();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+
+        // This query is completely in memory, so it doesn't need to be closed.
+        final QueryRepository queries = new InMemoryQueryRepository( new InMemoryQueryChangeLog() );
+
+        // Add the query to the query repository.
+        final StreamsQuery sQuery = queries.add("SELECT * WHERE { ?person <urn:worksAt> ?business . }");
+        final UUID queryId = sQuery.getQueryId();
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // The thread that will run the tested interactor.
+        final Thread testThread = new Thread() {
+            @Override
+            public void run() {
+                final RunQuery runQuery = new KafkaRunQuery(
+                        kafka.getKafkaHostname(),
+                        kafka.getKafkaPort(),
+                        statementsTopic,
+                        resultsTopic,
+                        queries,
+                        new TopologyFactory());
+                try {
+                    runQuery.run(queryId);
+                } catch (final RyaStreamsException e) {
+                    // Do nothing. Test will still fail because the expected results will be missing.
+                }
+            }
+        };
+
+        // Create the topics.
+        kafka.createTopic(statementsTopic);
+        kafka.createTopic(resultsTopic);
+
+        // Create the statements that will be loaded.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(vf.createStatement(
+                vf.createURI("urn:Alice"),
+                vf.createURI("urn:worksAt"),
+                vf.createURI("urn:BurgerJoint")), "a"));
+        statements.add(new VisibilityStatement(vf.createStatement(
+                vf.createURI("urn:Bob"),
+                vf.createURI("urn:worksAt"),
+                vf.createURI("urn:TacoShop")), "a"));
+        statements.add(new VisibilityStatement(vf.createStatement(
+                vf.createURI("urn:Charlie"),
+                vf.createURI("urn:worksAt"),
+                vf.createURI("urn:TacoShop")), "a"));
+
+        // Create the expected results.
+        final List<VisibilityBindingSet> expected = new ArrayList<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add(new VisibilityBindingSet(bs, "a"));
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoShop"));
+        expected.add(new VisibilityBindingSet(bs, "a"));
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Charlie"));
+        bs.addBinding("business", vf.createURI("urn:TacoShop"));
+        expected.add(new VisibilityBindingSet(bs, "a"));
+
+        // Execute the test. This will result in a set of results that were read from the results topic.
+        final List<VisibilityBindingSet> results;
+        try {
+            // Wait for the program to start.
+            testThread.start();
+            Thread.sleep(2000);
+
+            // Write some statements to the program.
+            final LoadStatements loadStatements = new KafkaLoadStatements(statementsTopic, producer);
+            loadStatements.fromCollection(statements);
+
+            // Read the output of the streams program.
+            consumer.subscribe( Lists.newArrayList(resultsTopic) );
+            results = KafkaTestUtil.pollForResults(500, 6, 3, consumer);
+        } finally {
+            // Tear down the test.
+            testThread.interrupt();
+            testThread.join(3000);
+        }
+
+        // Show the read results matched the expected ones.
+        assertEquals(expected, results);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
index 3e0e64d..80b6e42 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
@@ -28,8 +28,8 @@ import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.rya.api.function.projection.RandomUUIDFactory;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.topology.TopologyFactory;
@@ -75,7 +75,7 @@ public class StatementPatternProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -113,7 +113,7 @@ public class StatementPatternProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a|b") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -147,7 +147,7 @@ public class StatementPatternProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -190,6 +190,6 @@ public class StatementPatternProcessorIT {
         expected.add(new VisibilityBindingSet(bs, "a"));
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
index 0348dcd..fb5305f 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
@@ -28,8 +28,8 @@ import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.rya.api.function.projection.RandomUUIDFactory;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
 import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.topology.TopologyFactory;
@@ -81,6 +81,6 @@ public class FilterProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
index b137a9a..51bb0ae 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
@@ -29,8 +29,8 @@ import org.apache.rya.api.function.join.NaturalJoin;
 import org.apache.rya.api.function.projection.RandomUUIDFactory;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
 import org.apache.rya.streams.kafka.processors.ProcessorResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
 import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier.JoinProcessor;
@@ -111,7 +111,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -162,7 +162,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -219,7 +219,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a&c") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -260,7 +260,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 3000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 3000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -311,6 +311,6 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "c") );
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
index d71577b..c96919c 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
@@ -26,8 +26,8 @@ import java.util.UUID;
 
 import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
 import org.apache.rya.streams.kafka.processors.projection.MultiProjectionProcessorSupplier.MultiProjectionProcessor;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
 import org.apache.rya.streams.kafka.topology.TopologyFactory;
@@ -87,6 +87,6 @@ public class MultiProjectionProcessorIT {
         expected.add(new VisibilityStatement(vf.createStatement(blankNode, vf.createURI("urn:location"), vf.createURI("urn:corner1")), "a"));
 
         // Run the test.
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityStatementDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityStatementDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
index bc5f115..63c2cc7 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
@@ -28,8 +28,8 @@ import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.rya.api.function.projection.RandomUUIDFactory;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
 import org.apache.rya.streams.kafka.topology.TopologyFactory;
@@ -80,6 +80,6 @@ public class ProjectionProcessorIT {
         expectedBs.addBinding("otherPerson", vf.createURI("urn:Bob"));
         expected.add(new VisibilityBindingSet(expectedBs, "a"));
 
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected), VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected), VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
index ff2b59b..04c81ed 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/queries/KafkaQueryChangeLogIT.java
@@ -33,11 +33,11 @@ import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.streams.api.queries.ChangeLogEntry;
 import org.apache.rya.streams.api.queries.QueryChange;
 import org.apache.rya.streams.api.queries.QueryChangeLog.QueryChangeLogException;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeDeserializer;
 import org.apache.rya.streams.kafka.serialization.queries.QueryChangeSerializer;
 import org.apache.rya.test.kafka.KafkaITBase;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java
index f9129ff..70cba1c 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityBindingSetKafkaIT.java
@@ -29,8 +29,8 @@ import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.Rule;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java
index b85eb0c..9e85f52 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/serialization/VisibilityStatementKafkaIT.java
@@ -29,8 +29,8 @@ import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
 import org.junit.Rule;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/test/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/test/kafka/pom.xml b/test/kafka/pom.xml
index 44773a7..8492629 100644
--- a/test/kafka/pom.xml
+++ b/test/kafka/pom.xml
@@ -70,6 +70,11 @@ under the License.
                 </exclusion>
             </exclusions>
         </dependency>
+        
+        <dependency>
+            <groupId>com.github.stephenc.findbugs</groupId>
+            <artifactId>findbugs-annotations</artifactId>
+        </dependency>
     
         <!-- Testing dependencies. -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java
----------------------------------------------------------------------
diff --git a/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java b/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java
index f76fa2b..252c288 100644
--- a/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java
+++ b/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java
@@ -105,6 +105,13 @@ public class KafkaTestInstanceRule extends ExternalResource {
     }
 
     /**
+     * @return The hostnames of the Zookeeper servers.
+     */
+    public String getZookeeperServers() {
+        return kafkaInstance.getZookeeperConnect();
+    }
+
+    /**
      * @return The hostname of the Kafka Broker.
      */
     public String getKafkaHostname() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/94423229/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java
----------------------------------------------------------------------
diff --git a/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java b/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java
new file mode 100644
index 0000000..4b41f1a
--- /dev/null
+++ b/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestUtil.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.test.kafka;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serializer;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A set of utility functions that are useful when writing tests against a Kafka instance.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class KafkaTestUtil {
+
+    private KafkaTestUtil() { }
+
+    /**
+     * Create a {@link Producer} that is able to write to a topic that is hosted within an embedded instance of Kafka.
+     *
+     * @param kafka - The Kafka rule used to connect to the embedded Kafkfa instance. (not null)
+     * @param keySerializerClass - Serializes the keys. (not null)
+     * @param valueSerializerClass - Serializes the values. (not null)
+     * @return A {@link Producer} that can be used to write records to a topic.
+     */
+    public static <K, V> Producer<K, V> makeProducer(
+            final KafkaTestInstanceRule kafka,
+            final Class<? extends Serializer<K>> keySerializerClass,
+            final Class<? extends Serializer<V>> valueSerializerClass) {
+        requireNonNull(kafka);
+        requireNonNull(keySerializerClass);
+        requireNonNull(valueSerializerClass);
+
+        final Properties props = kafka.createBootstrapServerConfig();
+        props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass.getName());
+        props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName());
+        return new KafkaProducer<>(props);
+    }
+
+    /**
+     * Create a {@link Consumer} that has a unique group ID and reads everything from a topic that is hosted within an
+     * embedded instance of Kafka starting at the earliest point by default.
+     *
+     * @param kafka - The Kafka rule used to connect to the embedded Kafkfa instance. (not null)
+     * @param keyDeserializerClass - Deserializes the keys. (not null)
+     * @param valueDeserializerClass - Deserializes the values. (not null)
+     * @return A {@link Consumer} that can be used to read records from a topic.
+     */
+    public static <K, V> Consumer<K, V> fromStartConsumer(
+            final KafkaTestInstanceRule kafka,
+            final Class<? extends Deserializer<K>> keyDeserializerClass,
+            final Class<? extends Deserializer<V>> valueDeserializerClass) {
+        requireNonNull(kafka);
+        requireNonNull(keyDeserializerClass);
+        requireNonNull(valueDeserializerClass);
+
+        final Properties props = kafka.createBootstrapServerConfig();
+        props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
+        props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
+        props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass.getName());
+        props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass.getName());
+        return new KafkaConsumer<>(props);
+    }
+
+    /**
+     * Polls a {@link Consumer> until it has either polled too many times without hitting the target number
+     * of results, or it hits the target number of results.
+     *
+     * @param pollMs - How long each poll could take.
+     * @param pollIterations - The maximum number of polls that will be attempted.
+     * @param targetSize - The number of results to read before stopping.
+     * @param consumer - The consumer that will be polled.
+     * @return The results that were read frmo the consumer.
+     * @throws Exception If the poll failed.
+     */
+    public static <K, V> List<V> pollForResults(
+            final int pollMs,
+            final int pollIterations,
+            final int targetSize,
+            final Consumer<K, V> consumer) throws Exception {
+        requireNonNull(consumer);
+
+        final List<V> values = new ArrayList<>();
+
+        int i = 0;
+        while(values.size() < targetSize && i < pollIterations) {
+            for(final ConsumerRecord<K, V> record : consumer.poll(pollMs)) {
+                values.add( record.value() );
+            }
+            i++;
+        }
+
+        return values;
+    }
+}
\ No newline at end of file


[32/50] [abbrv] incubator-rya git commit: RYA-377 Fixed the project layout so that it builds even when the geoindexing profile is not enabled.

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/sp/StatementPatternProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/sp/StatementPatternProcessorIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/sp/StatementPatternProcessorIT.java
new file mode 100644
index 0000000..ba11e57
--- /dev/null
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/sp/StatementPatternProcessorIT.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.sp;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.evaluation.QueryBindingSet;
+
+/**
+ * Integration tests the methods of {@link StatementPatternProcessor}.
+ */
+public class StatementPatternProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void singlePattern_singleStatement() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final String query = "SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create a statement that generate an SP result.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        final QueryBindingSet bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void singlePattern_manyStatements() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final String query = "SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create some statements where some generates SP results and others do not.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoJoin")), "b") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:talksTo"), vf.createURI("urn:Alice")), "a|b") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "c") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        QueryBindingSet bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Alice"));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void multiplePatterns_singleStatement() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final String query = "SELECT * WHERE { "
+                + "?person <urn:talksTo> ?otherPerson . "
+                + "?person ?action <urn:Bob>"
+                + "}";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create some statements where some generates SP results and others do not.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        final QueryBindingSet bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("action", vf.createURI("urn:talksTo"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void multiplePatterns_multipleStatements() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final String query = "SELECT * WHERE { "
+                + "?person <urn:talksTo> ?otherPerson ."
+                + "?person ?action <urn:Bob>"
+                + "}";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create some statements where some generates SP results and others do not.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Charlie")), "a|b") );
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:walksWith"), vf.createURI("urn:Bob")), "b") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        QueryBindingSet bs = new QueryBindingSet();
+        bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("action", vf.createURI("urn:talksTo"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Charlie"));
+        expected.add(new VisibilityBindingSet(bs, "a&(a|b)"));
+
+        bs = new QueryBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("action", vf.createURI("urn:talksTo"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add(new VisibilityBindingSet(bs, "a"));
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka-test/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka-test/pom.xml b/extras/rya.streams/kafka-test/pom.xml
new file mode 100644
index 0000000..4a423e2
--- /dev/null
+++ b/extras/rya.streams/kafka-test/pom.xml
@@ -0,0 +1,51 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+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">
+
+    <parent>
+        <groupId>org.apache.rya</groupId>
+        <artifactId>rya.streams.parent</artifactId>
+        <version>3.2.12-incubating-SNAPSHOT</version>
+    </parent>
+    
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>rya.streams.kafka-test</artifactId>
+    
+    <name>Apache Rya Streams Kafka Test</name>
+    <description>
+        A common test jar containing utilities used to run Kafka based Rya 
+        Streams integration tests.
+    </description>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.test.kafka</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.streams.kafka</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka-test/src/main/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka-test/src/main/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java b/extras/rya.streams/kafka-test/src/main/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
new file mode 100644
index 0000000..ee25f8c
--- /dev/null
+++ b/extras/rya.streams/kafka-test/src/main/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka;
+
+import static java.util.Objects.requireNonNull;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.apache.rya.test.kafka.KafkaTestUtil;
+
+import com.google.common.collect.Sets;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Utility functions that make it easier to test Rya Streams applications.
+ */
+@DefaultAnnotation(NonNull.class)
+public class RyaStreamsTestUtil {
+
+    /**
+     * Runs a Kafka Streams topology, loads statements into the input topic, read the binding sets that come out of
+     * the results topic, and ensures the expected results match the read results.
+     *
+     * @param <T> The type of value that will be consumed from the results topic.
+     * @param kafka - The embedded Kafka instance that is being tested with. (not null)
+     * @param statementsTopic - The topic statements will be written to. (not null)
+     * @param resultsTopic - The topic results will be read from. (not null)
+     * @param builder - The streams topology that will be executed. (not null)
+     * @param statements - The statements that will be loaded into the topic. (not null)
+     * @param expected - The expected results. (not null)
+     * @param expectedDeserializerClass - The class of the deserializer that will be used when reading
+     *   values from the results topic. (not null)
+     * @throws Exception If any exception was thrown while running the test.
+     */
+    public static <T> void runStreamProcessingTest(
+            final KafkaTestInstanceRule kafka,
+            final String statementsTopic,
+            final String resultsTopic,
+            final TopologyBuilder builder,
+            final List<VisibilityStatement> statements,
+            final Set<T> expected,
+            final Class<? extends Deserializer<T>> expectedDeserializerClass) throws Exception {
+        requireNonNull(kafka);
+        requireNonNull(statementsTopic);
+        requireNonNull(resultsTopic);
+        requireNonNull(builder);
+        requireNonNull(statements);
+        requireNonNull(expected);
+        requireNonNull(expectedDeserializerClass);
+
+        // Explicitly create the topics that are being used.
+        kafka.createTopic(statementsTopic);
+        kafka.createTopic(resultsTopic);
+
+        // Start the streams program.
+        final Properties props = kafka.createBootstrapServerConfig();
+        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "StatementPatternProcessorIT");
+
+        final KafkaStreams streams = new KafkaStreams(builder, new StreamsConfig(props));
+        streams.cleanUp();
+        try {
+            streams.start();
+
+            // Wait for the streams application to start. Streams only see data after their consumers are connected.
+            Thread.sleep(6000);
+
+            // Load the statements into the input topic.
+            try(Producer<String, VisibilityStatement> producer = KafkaTestUtil.makeProducer(
+                    kafka, StringSerializer.class, VisibilityStatementSerializer.class)) {
+                new KafkaLoadStatements(statementsTopic, producer).fromCollection(statements);
+            }
+
+            // Wait for the final results to appear in the output topic and verify the expected Binding Sets were found.
+            try(Consumer<String, T> consumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, expectedDeserializerClass)) {
+                // Register the topic.
+                consumer.subscribe(Arrays.asList(resultsTopic));
+
+                // Poll for the result.
+                final Set<T> results = Sets.newHashSet( KafkaTestUtil.pollForResults(500, 6, expected.size(), consumer) );
+
+                // Show the correct binding sets results from the job.
+                assertEquals(expected, results);
+            }
+        } finally {
+            streams.close();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/pom.xml b/extras/rya.streams/kafka/pom.xml
index 8926870..778630d 100644
--- a/extras/rya.streams/kafka/pom.xml
+++ b/extras/rya.streams/kafka/pom.xml
@@ -36,23 +36,22 @@ under the License.
     <profiles>
         <profile>
             <id>geoindexing</id>
-                <dependencies>
-                    <!-- Rya dependencies -->
-                    <dependency>
-                        <groupId>org.apache.rya</groupId>
-                        <artifactId>rya.pcj.functions.geo</artifactId>
-                        <version>${project.version}</version>
-                    </dependency>
-                    <dependency>
-                        <groupId>org.apache.rya</groupId>
-                        <artifactId>rya.geo.common</artifactId>
-                        <version>${project.version}</version>
-                    </dependency>
-                </dependencies>
+            <dependencies>
+                <!-- Rya dependencies -->
+                <dependency>
+                    <groupId>org.apache.rya</groupId>
+                    <artifactId>rya.pcj.functions.geo</artifactId>
+                    <version>${project.version}</version>
+                </dependency>
+                <dependency>
+                    <groupId>org.apache.rya</groupId>
+                    <artifactId>rya.geo.common</artifactId>
+                    <version>${project.version}</version>
+                </dependency>
+            </dependencies>
         </profile>
     </profiles>
 
-
     <dependencies>
         <!-- Rya dependencies -->
         <dependency>
@@ -106,6 +105,11 @@ under the License.
         </dependency>
         <dependency>
             <groupId>org.apache.rya</groupId>
+            <artifactId>rya.test.rdf</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
             <artifactId>rya.test.kafka</artifactId>
             <scope>test</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
deleted file mode 100644
index b4388c3..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.openrdf.query.algebra.Filter;
-import org.openrdf.query.algebra.MultiProjection;
-import org.openrdf.query.algebra.Projection;
-import org.openrdf.query.algebra.StatementPattern;
-import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
-import org.openrdf.query.parser.ParsedQuery;
-import org.openrdf.query.parser.sparql.SPARQLParser;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-import edu.umd.cs.findbugs.annotations.Nullable;
-
-/**
- * A set of utility functions that are useful when writing tests RDF functions.
- */
-@DefaultAnnotation(NonNull.class)
-public final class RdfTestUtil {
-
-    private RdfTestUtil() { }
-
-    /**
-     * Fetch the {@link StatementPattern} from a SPARQL string.
-     *
-     * @param sparql - A SPARQL query that contains only a single Statement Patern. (not nul)
-     * @return The {@link StatementPattern} that was in the query, if it could be found. Otherwise {@code null}
-     * @throws Exception The statement pattern could not be found in the parsed SPARQL query.
-     */
-    public static @Nullable StatementPattern getSp(final String sparql) throws Exception {
-        requireNonNull(sparql);
-
-        final AtomicReference<StatementPattern> statementPattern = new AtomicReference<>();
-        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
-        parsed.getTupleExpr().visitChildren(new QueryModelVisitorBase<Exception>() {
-            @Override
-            public void meet(final StatementPattern node) throws Exception {
-                statementPattern.set(node);
-            }
-        });
-        return statementPattern.get();
-    }
-
-    /**
-     * Get the first {@link Projection} node from a SPARQL query.
-     *
-     * @param sparql - The query that contains a single Projection node.
-     * @return The first {@link Projection} that is encountered.
-     * @throws Exception The query could not be parsed.
-     */
-    public static @Nullable Projection getProjection(final String sparql) throws Exception {
-        requireNonNull(sparql);
-
-        final AtomicReference<Projection> projection = new AtomicReference<>();
-        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
-        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
-            @Override
-            public void meet(final Projection node) throws Exception {
-                projection.set(node);
-            }
-        });
-
-        return projection.get();
-    }
-
-    /**
-     * Get the first {@link MultiProjection} node from a SPARQL query.
-     *
-     * @param sparql - The query that contains a single Projection node.
-     * @return The first {@link MultiProjection} that is encountered.
-     * @throws Exception The query could not be parsed.
-     */
-    public static @Nullable MultiProjection getMultiProjection(final String sparql) throws Exception {
-        requireNonNull(sparql);
-
-        final AtomicReference<MultiProjection> multiProjection = new AtomicReference<>();
-        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
-        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
-            @Override
-            public void meet(final MultiProjection node) throws Exception {
-                multiProjection.set(node);
-            }
-        });
-
-        return multiProjection.get();
-    }
-
-    /**
-     * Get the first {@link Filter} node from a SPARQL query.
-     *
-     * @param sparql - The query that contains a single Projection node.
-     * @return The first {@link Filter} that is encountered.
-     * @throws Exception The query could not be parsed.
-     */
-    public static @Nullable Filter getFilter(final String sparql) throws Exception {
-        requireNonNull(sparql);
-
-        final AtomicReference<Filter> filter = new AtomicReference<>();
-        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
-        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
-            @Override
-            public void meet(final Filter node) throws Exception {
-                filter.set(node);
-            }
-        });
-
-        return filter.get();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
deleted file mode 100644
index ee25f8c..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka;
-
-import static java.util.Objects.requireNonNull;
-import static org.junit.Assert.assertEquals;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-
-import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.common.serialization.Deserializer;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.apache.kafka.streams.KafkaStreams;
-import org.apache.kafka.streams.StreamsConfig;
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.interactor.KafkaLoadStatements;
-import org.apache.rya.streams.kafka.serialization.VisibilityStatementSerializer;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.apache.rya.test.kafka.KafkaTestUtil;
-
-import com.google.common.collect.Sets;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-
-/**
- * Utility functions that make it easier to test Rya Streams applications.
- */
-@DefaultAnnotation(NonNull.class)
-public class RyaStreamsTestUtil {
-
-    /**
-     * Runs a Kafka Streams topology, loads statements into the input topic, read the binding sets that come out of
-     * the results topic, and ensures the expected results match the read results.
-     *
-     * @param <T> The type of value that will be consumed from the results topic.
-     * @param kafka - The embedded Kafka instance that is being tested with. (not null)
-     * @param statementsTopic - The topic statements will be written to. (not null)
-     * @param resultsTopic - The topic results will be read from. (not null)
-     * @param builder - The streams topology that will be executed. (not null)
-     * @param statements - The statements that will be loaded into the topic. (not null)
-     * @param expected - The expected results. (not null)
-     * @param expectedDeserializerClass - The class of the deserializer that will be used when reading
-     *   values from the results topic. (not null)
-     * @throws Exception If any exception was thrown while running the test.
-     */
-    public static <T> void runStreamProcessingTest(
-            final KafkaTestInstanceRule kafka,
-            final String statementsTopic,
-            final String resultsTopic,
-            final TopologyBuilder builder,
-            final List<VisibilityStatement> statements,
-            final Set<T> expected,
-            final Class<? extends Deserializer<T>> expectedDeserializerClass) throws Exception {
-        requireNonNull(kafka);
-        requireNonNull(statementsTopic);
-        requireNonNull(resultsTopic);
-        requireNonNull(builder);
-        requireNonNull(statements);
-        requireNonNull(expected);
-        requireNonNull(expectedDeserializerClass);
-
-        // Explicitly create the topics that are being used.
-        kafka.createTopic(statementsTopic);
-        kafka.createTopic(resultsTopic);
-
-        // Start the streams program.
-        final Properties props = kafka.createBootstrapServerConfig();
-        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "StatementPatternProcessorIT");
-
-        final KafkaStreams streams = new KafkaStreams(builder, new StreamsConfig(props));
-        streams.cleanUp();
-        try {
-            streams.start();
-
-            // Wait for the streams application to start. Streams only see data after their consumers are connected.
-            Thread.sleep(6000);
-
-            // Load the statements into the input topic.
-            try(Producer<String, VisibilityStatement> producer = KafkaTestUtil.makeProducer(
-                    kafka, StringSerializer.class, VisibilityStatementSerializer.class)) {
-                new KafkaLoadStatements(statementsTopic, producer).fromCollection(statements);
-            }
-
-            // Wait for the final results to appear in the output topic and verify the expected Binding Sets were found.
-            try(Consumer<String, T> consumer = KafkaTestUtil.fromStartConsumer(kafka, StringDeserializer.class, expectedDeserializerClass)) {
-                // Register the topic.
-                consumer.subscribe(Arrays.asList(resultsTopic));
-
-                // Poll for the result.
-                final Set<T> results = Sets.newHashSet( KafkaTestUtil.pollForResults(500, 6, expected.size(), consumer) );
-
-                // Show the correct binding sets results from the job.
-                assertEquals(expected, results);
-            }
-        } finally {
-            streams.close();
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
deleted file mode 100644
index 33dc945..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
+++ /dev/null
@@ -1,195 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.function.projection.RandomUUIDFactory;
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
-import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
-import org.apache.rya.streams.kafka.topology.TopologyFactory;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.algebra.evaluation.QueryBindingSet;
-
-/**
- * Integration tests the methods of {@link StatementPatternProcessor}.
- */
-public class StatementPatternProcessorIT {
-
-    @Rule
-    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
-
-    @Test
-    public void singlePattern_singleStatement() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final String query = "SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }";
-        final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create a statement that generate an SP result.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
-
-        // Show the correct binding set results from the job.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-
-        final QueryBindingSet bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void singlePattern_manyStatements() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final String query = "SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }";
-        final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create some statements where some generates SP results and others do not.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoJoin")), "b") );
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:talksTo"), vf.createURI("urn:Alice")), "a|b") );
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "c") );
-
-        // Show the correct binding set results from the job.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-
-        QueryBindingSet bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Bob"));
-        bs.addBinding("otherPerson", vf.createURI("urn:Alice"));
-        expected.add( new VisibilityBindingSet(bs, "a|b") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void multiplePatterns_singleStatement() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final String query = "SELECT * WHERE { "
-                + "?person <urn:talksTo> ?otherPerson . "
-                + "?person ?action <urn:Bob>"
-                + "}";
-        final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create some statements where some generates SP results and others do not.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
-
-        // Show the correct binding set results from the job.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-
-        final QueryBindingSet bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("action", vf.createURI("urn:talksTo"));
-        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void multiplePatterns_multipleStatements() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final String query = "SELECT * WHERE { "
-                + "?person <urn:talksTo> ?otherPerson ."
-                + "?person ?action <urn:Bob>"
-                + "}";
-        final TopologyFactory factory = new TopologyFactory();
-        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create some statements where some generates SP results and others do not.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Charlie")), "a|b") );
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:walksWith"), vf.createURI("urn:Bob")), "b") );
-
-        // Show the correct binding set results from the job.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-
-        QueryBindingSet bs = new QueryBindingSet();
-        bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("action", vf.createURI("urn:talksTo"));
-        bs.addBinding("otherPerson", vf.createURI("urn:Charlie"));
-        expected.add(new VisibilityBindingSet(bs, "a&(a|b)"));
-
-        bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("action", vf.createURI("urn:talksTo"));
-        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
-        expected.add(new VisibilityBindingSet(bs, "a"));
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
deleted file mode 100644
index 072469a..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
+++ /dev/null
@@ -1,457 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors.aggregation;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.function.projection.RandomUUIDFactory;
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
-import org.apache.rya.streams.kafka.processors.aggregation.AggregationProcessorSupplier.AggregationProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
-import org.apache.rya.streams.kafka.topology.TopologyFactory;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.model.vocabulary.XMLSchema;
-import org.openrdf.query.impl.MapBindingSet;
-
-/**
- * Integration tests {@link AggregationProcessor}.
- */
-public class AggregationProcessorIT {
-
-    @Rule
-    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(false);
-
-    @Test
-    public void count() throws Exception {
-        // A query that figures out how many books each person has.
-        final String sparql =
-                "SELECT ?person (count(?book) as ?bookCount) " +
-                "WHERE { " +
-                    "?person <urn:hasBook> ?book " +
-                "} GROUP BY ?person";
-
-        // Create the statements that will be input into the query..
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasBook"), vf.createLiteral("Book 1")), "a"));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasBook"), vf.createLiteral("Book 1")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasBook"), vf.createLiteral("Book 2")), "b"));
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("bookCount", vf.createLiteral("1", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, "a"));
-
-        bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Bob"));
-        bs.addBinding("bookCount", vf.createLiteral("1", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("bookCount", vf.createLiteral("2", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, "a&b"));
-
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void sum() throws Exception {
-        // A query that figures out how much food each person has.
-        final String sparql =
-                "SELECT ?person (sum(?foodCount) as ?totalFood) " +
-                "WHERE { " +
-                    "?person <urn:hasFoodType> ?food . " +
-                    "?food <urn:count> ?foodCount . " +
-                "} GROUP BY ?person";
-
-        // Create the statements that will be input into the query..
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasFoodType"), vf.createURI("urn:corn")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasFoodType"), vf.createURI("urn:apple")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:corn"), vf.createURI("urn:count"), vf.createLiteral(4)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:apple"), vf.createURI("urn:count"), vf.createLiteral(3)), ""));
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("totalFood", vf.createLiteral("4", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("totalFood", vf.createLiteral("7", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void average() throws Exception {
-        // A query that figures out the average age across all people.
-        final String sparql =
-                "SELECT (avg(?age) as ?avgAge) " +
-                "WHERE { " +
-                    "?person <urn:age> ?age " +
-                "}";
-
-        // Create the statements that will be input into the query..
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(3)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(2)), ""));
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("avgAge", vf.createLiteral("3", XMLSchema.DECIMAL));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("avgAge", vf.createLiteral("5", XMLSchema.DECIMAL));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("avgAge", vf.createLiteral("4", XMLSchema.DECIMAL));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void min() throws Exception {
-        // A query that figures out what the youngest age is across all people.
-        final String sparql =
-                "SELECT (min(?age) as ?youngest) " +
-                "WHERE { " +
-                    "?person <urn:age> ?age " +
-                "}";
-
-        // Create the statements that will be input into the query..
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(13)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(14)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:age"), vf.createLiteral(5)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:age"), vf.createLiteral(25)), ""));
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("youngest", vf.createLiteral(13));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("youngest", vf.createLiteral(7));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("youngest", vf.createLiteral(5));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void max() throws Exception {
-        // A query that figures out what the oldest age is across all people.
-        final String sparql =
-                "SELECT (max(?age) as ?oldest) " +
-                "WHERE { " +
-                    "?person <urn:age> ?age " +
-                "}";
-
-        // Create the statements that will be input into the query..
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(13)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(14)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:age"), vf.createLiteral(5)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:age"), vf.createLiteral(25)), ""));
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("oldest", vf.createLiteral(13));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("oldest", vf.createLiteral(14));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("oldest", vf.createLiteral(25));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void multipleGroupByVars() throws Exception {
-        // A query that contains more than one group by variable.
-        final String sparql =
-                "SELECT ?business ?employee (sum(?hours) AS ?totalHours) " +
-                "WHERE {" +
-                    "?employee <urn:worksAt> ?business . " +
-                    "?business <urn:hasTimecardId> ?timecardId . " +
-                    "?employee <urn:hasTimecardId> ?timecardId . " +
-                    "?timecardId <urn:hours> ?hours . " +
-                "} GROUP BY ?business ?employee";
-
-        // Create the statements that will be input into the query.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoJoint")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard1")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard1")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:timecard1"), vf.createURI("urn:hours"), vf.createLiteral(40)), ""));
-
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard2")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard2")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:timecard2"), vf.createURI("urn:hours"), vf.createLiteral(25)), ""));
-
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoJoint")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard3")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard3")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:timecard3"), vf.createURI("urn:hours"), vf.createLiteral(28)), ""));
-
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard4")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard4")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:timecard4"), vf.createURI("urn:hours"), vf.createLiteral(28)), ""));
-
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:CoffeeShop"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard5")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard5")), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:timecard5"), vf.createURI("urn:hours"), vf.createLiteral(12)), ""));
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
-        bs.addBinding("employee", vf.createURI("urn:Alice"));
-        bs.addBinding("totalHours", vf.createLiteral("40", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
-        bs.addBinding("employee", vf.createURI("urn:Alice"));
-        bs.addBinding("totalHours", vf.createLiteral("65", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("totalHours", vf.createLiteral("28", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
-        bs.addBinding("employee", vf.createURI("urn:Bob"));
-        bs.addBinding("totalHours", vf.createLiteral("56", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("business", vf.createURI("urn:CoffeeShop"));
-        bs.addBinding("employee", vf.createURI("urn:Alice"));
-        bs.addBinding("totalHours", vf.createLiteral("12", XMLSchema.INTEGER));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    @Test
-    public void multipleAggregations() throws Exception {
-        // A query that figures out what the youngest and oldest ages are across all people.
-        final String sparql =
-                "SELECT (min(?age) as ?youngest) (max(?age) as ?oldest) " +
-                "WHERE { " +
-                    "?person <urn:age> ?age " +
-                "}";
-
-        // Create the statements that will be input into the query..
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(13)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(14)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:age"), vf.createLiteral(5)), ""));
-        statements.add(new VisibilityStatement(
-                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:age"), vf.createLiteral(25)), ""));
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("youngest", vf.createLiteral(13));
-        bs.addBinding("oldest", vf.createLiteral(13));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("youngest", vf.createLiteral(13));
-        bs.addBinding("oldest", vf.createLiteral(14));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("youngest", vf.createLiteral(7));
-        bs.addBinding("oldest", vf.createLiteral(14));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("youngest", vf.createLiteral(5));
-        bs.addBinding("oldest", vf.createLiteral(14));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        bs = new MapBindingSet();
-        bs.addBinding("youngest", vf.createLiteral(5));
-        bs.addBinding("oldest", vf.createLiteral(25));
-        expected.add(new VisibilityBindingSet(bs, ""));
-
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
deleted file mode 100644
index aaa67ea..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors.filter;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.function.projection.RandomUUIDFactory;
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
-import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
-import org.apache.rya.streams.kafka.topology.TopologyFactory;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.impl.MapBindingSet;
-
-/**
- * Integration tests the methods of {@link FilterProcessor}.
- */
-public class FilterProcessorIT {
-
-    @Rule
-    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
-
-    @Test
-    public void showProcessorWorks() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Get the RDF model objects that will be used to build the query.
-        final String sparql =
-                "SELECT * " +
-                "WHERE { " +
-                    "FILTER(?age < 10)" +
-                    "?person <urn:age> ?age " +
-                "}";
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create the statements that will be input into the query.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(11)), "a"));
-        statements.add(new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(9)), "a"));
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        final MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("age", vf.createLiteral(9));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
deleted file mode 100644
index 3ff8e8d..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors.filter;
-
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
-import org.apache.kafka.streams.processor.ProcessorContext;
-import org.apache.rya.api.function.filter.FilterEvaluator;
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.streams.kafka.RdfTestUtil;
-import org.apache.rya.streams.kafka.processors.ProcessorResult;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
-import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.algebra.Filter;
-import org.openrdf.query.impl.MapBindingSet;
-
-/**
- * Unit tests the methods of {@link FilterProcessor}.
- */
-public class FilterProcessorTest {
-
-    @Test
-    public void showFilterFunctionIsCalled() throws Exception {
-        // Read the filter object from a SPARQL query.
-        final Filter filter = RdfTestUtil.getFilter(
-                "SELECT * " +
-                "WHERE { " +
-                    "FILTER(?age < 10)" +
-                    "?person <urn:age> ?age " +
-                "}");
-
-        // Create a Binding Set that will be passed into the Filter function based on the where clause.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("age", vf.createLiteral(9));
-        final VisibilityBindingSet inputVisBs = new VisibilityBindingSet(bs, "a");
-
-        // Mock the processor context that will be invoked.
-        final ProcessorContext context = mock(ProcessorContext.class);
-
-        // Run the test.
-        final FilterProcessor processor = new FilterProcessor(
-                FilterEvaluator.make(filter),
-                result -> ProcessorResult.make(new UnaryResult(result)));
-        processor.init(context);
-        processor.process("key", ProcessorResult.make(new UnaryResult(inputVisBs)));
-
-        // Verify the binding set was passed through.
-        verify(context, times(1)).forward(eq("key"), eq(ProcessorResult.make(new UnaryResult(inputVisBs))));
-
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
deleted file mode 100644
index c090afa..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors.filter;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.function.projection.RandomUUIDFactory;
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.indexing.GeoConstants;
-import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
-import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
-import org.apache.rya.streams.kafka.topology.TopologyFactory;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.StatementImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.algebra.evaluation.function.Function;
-import org.openrdf.query.algebra.evaluation.function.FunctionRegistry;
-import org.openrdf.query.impl.MapBindingSet;
-
-import com.vividsolutions.jts.geom.Coordinate;
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.geom.GeometryFactory;
-import com.vividsolutions.jts.io.WKTWriter;
-
-/**
- * Integration tests the geo methods of {@link FilterProcessor}.
- */
-public class GeoFilterIT {
-    private static final String GEO = "http://www.opengis.net/def/function/geosparql/";
-    private static final GeometryFactory GF = new GeometryFactory();
-    private static final Geometry ZERO = GF.createPoint(new Coordinate(0, 0));
-    private static final Geometry ONE = GF.createPoint(new Coordinate(1, 1));
-
-    @Rule
-    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
-
-    @Test
-    public void showGeoFunctionsRegistered() {
-        int count = 0;
-        final Collection<Function> funcs = FunctionRegistry.getInstance().getAll();
-        for (final Function fun : funcs) {
-            if (fun.getURI().startsWith(GEO)) {
-                count++;
-            }
-        }
-
-        // There are 30 geo functions registered, ensure that there are 30.
-        assertEquals(30, count);
-    }
-
-    @Test
-    public void showProcessorWorks() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Get the RDF model objects that will be used to build the query.
-        final String sparql =
-                "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-                        + "PREFIX geof: <" + GEO + ">\n"
-                        + "SELECT * \n"
-                        + "WHERE { \n"
-                        + "  <urn:event1> geo:asWKT ?point .\n"
-                        + " FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
-                        + "}";
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
-
-        // Create the statements that will be input into the query.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = getStatements();
-
-        // Make the expected results.
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        final MapBindingSet bs = new MapBindingSet();
-        final WKTWriter w = new WKTWriter();
-        bs.addBinding("point", vf.createLiteral(w.write(ZERO), GeoConstants.XMLSCHEMA_OGC_WKT));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
-    }
-
-    private List<VisibilityStatement> getStatements() throws Exception {
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        // geo 2x2 points
-        statements.add(new VisibilityStatement(statement(ZERO), "a"));
-        statements.add(new VisibilityStatement(statement(ONE), "a"));
-        return statements;
-    }
-
-    private static Statement statement(final Geometry geo) {
-        final ValueFactory vf = new ValueFactoryImpl();
-        final Resource subject = vf.createURI("urn:event1");
-        final URI predicate = GeoConstants.GEO_AS_WKT;
-        final WKTWriter w = new WKTWriter();
-        final Value object = vf.createLiteral(w.write(geo), GeoConstants.XMLSCHEMA_OGC_WKT);
-        return new StatementImpl(subject, predicate, object);
-    }
-}
\ No newline at end of file


[02/50] [abbrv] incubator-rya git commit: RYA-377 Create the GetQueryResultStream interactor.

Posted by ca...@apache.org.
RYA-377 Create the GetQueryResultStream interactor.


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

Branch: refs/heads/master
Commit: 3ccfbadc3e4b50726fb19213c43f58d3445bfc75
Parents: a95fe5a
Author: kchilton2 <ke...@gmail.com>
Authored: Thu Oct 26 15:35:42 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:12:59 2018 -0500

----------------------------------------------------------------------
 .../streams/api/entity/QueryResultStream.java   | 50 +++++++++++++++++++
 .../api/interactor/GetQueryResultStream.java    | 52 ++++++++++++++++++++
 2 files changed, 102 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3ccfbadc/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
new file mode 100644
index 0000000..fdd62df
--- /dev/null
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.api.entity;
+
+import java.util.Collection;
+import java.util.UUID;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * An infinite stream of {@link VisibilityBindingSet}s that are the results of a query within Rya Streams.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface QueryResultStream extends AutoCloseable {
+
+    /**
+     * @return Identifies which query in Rya Streams this result stream is over.
+     */
+    public UUID getQueryId();
+
+    /**
+     * Wait at most {@code timeoutMs} milliseconds for the next collection of results.
+     *
+     * @param timeoutMs - The number of milliseconds to at most wait for the next collection of results. (not null)
+     * @return The next collection of {@link VisibilityBindingSet}s that are the result of the query. Empty if
+     *   there where no new results within the timout period.
+     * @throws RyaStreamsException Could not fetch the next set of results.
+     */
+    public Collection<VisibilityBindingSet> poll(long timeoutMs) throws RyaStreamsException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/3ccfbadc/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/GetQueryResultStream.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/GetQueryResultStream.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/GetQueryResultStream.java
new file mode 100644
index 0000000..9ca577c
--- /dev/null
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/interactor/GetQueryResultStream.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.api.interactor;
+
+import java.util.UUID;
+
+import org.apache.rya.streams.api.entity.QueryResultStream;
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Get a {@link QueryResultStream} over the results of a query that is being managed by Rya Streams.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface GetQueryResultStream {
+
+    /**
+     * Stream all of the results that have been produced by a query.
+     *
+     * @param queryId - Indicates which query results to stream. (not null)
+     * @return A {@link QueryResultStream} that starts with the first result that was ever produced.
+     * @throws RyaStreamsException Could not create the result stream.
+     */
+    public QueryResultStream fromStart(UUID queryId) throws RyaStreamsException;
+
+    /**
+     * Stream results that have been produced by a query after this method was invoked.
+     *
+     * @param queryId - Indicates which query results to stream. (not null)
+     * @return A {@link QueryResultStream} that only returns results that were produced after this method is invoked.
+     * @throws RyaStreamsException Could not create the result stream.
+     */
+    public QueryResultStream fromNow(UUID queryId) throws RyaStreamsException;
+}
\ No newline at end of file


[49/50] [abbrv] incubator-rya git commit: RYA-377 Code review.

Posted by ca...@apache.org.
RYA-377 Code review.


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

Branch: refs/heads/master
Commit: f0725df5921a45adc37bc5cf73b5f70dfd886ac5
Parents: f365521
Author: kchilton2 <ke...@gmail.com>
Authored: Mon Jan 8 16:41:30 2018 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:02 2018 -0500

----------------------------------------------------------------------
 common/rya.api.function/pom.xml                 |  4 +-
 .../function/aggregation/AverageFunction.java   |  3 +
 .../api/function/aggregation/CountFunction.java |  3 +
 .../api/function/aggregation/MaxFunction.java   |  3 +
 .../api/function/aggregation/MinFunction.java   |  3 +
 .../api/function/aggregation/SumFunction.java   |  3 +
 .../apache/rya/api/utils/CloseableIterator.java | 31 ++++++++
 .../accumulo/AccumuloBatchUpdatePCJIT.java      |  2 +-
 .../TimestampedNotificationProcessor.java       |  2 +-
 .../PeriodicNotificationApplicationIT.java      |  2 +-
 .../pruner/PeriodicNotificationBinPrunerIT.java | 14 ++--
 .../pcj/storage/PeriodicQueryResultStorage.java | 28 +++----
 .../pcj/storage/PrecomputedJoinStorage.java     | 11 +--
 .../storage/accumulo/AccumuloPcjStorage.java    |  1 +
 .../AccumuloPeriodicQueryResultStorage.java     |  2 +-
 .../AccumuloValueBindingSetIterator.java        |  2 +-
 .../pcj/storage/accumulo/PcjTables.java         |  3 +-
 .../accumulo/ScannerBindingSetIterator.java     |  2 +-
 .../pcj/storage/accumulo/PcjTablesIT.java       |  4 +-
 .../integration/AccumuloPcjStorageIT.java       |  2 +-
 .../AccumuloPeriodicQueryResultStorageIT.java   | 84 ++++++++++----------
 extras/rya.pcj.fluo/pcj.fluo.app/pom.xml        |  2 +-
 .../pcj/fluo/demo/FluoAndHistoricPcjsDemo.java  |  2 +-
 .../indexing/pcj/fluo/integration/InputIT.java  |  2 +-
 .../indexing/pcj/fluo/integration/QueryIT.java  |  2 +-
 .../RyaInputIncrementalUpdateIT.java            |  2 +-
 .../pcj/fluo/integration/StreamingTestIT.java   |  2 +-
 .../pcj/fluo/visibility/PcjVisibilityIT.java    |  2 +-
 extras/rya.streams/api/pom.xml                  |  5 --
 extras/rya.streams/kafka/pom.xml                |  2 +-
 .../kafka/processors/ProcessorResult.java       |  2 +-
 .../processors/join/CloseableIterator.java      | 32 --------
 .../processors/join/JoinProcessorSupplier.java  |  3 +-
 .../kafka/processors/join/JoinStateStore.java   |  1 +
 .../processors/join/KeyValueJoinStateStore.java |  1 +
 extras/rya.streams/pom.xml                      | 10 ++-
 pom.xml                                         |  2 +-
 37 files changed, 146 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/common/rya.api.function/pom.xml
----------------------------------------------------------------------
diff --git a/common/rya.api.function/pom.xml b/common/rya.api.function/pom.xml
index ce88e36..5b7ee0a 100644
--- a/common/rya.api.function/pom.xml
+++ b/common/rya.api.function/pom.xml
@@ -27,8 +27,8 @@ under the License.
         <version>3.2.12-incubating-SNAPSHOT</version>
     </parent>
 
-    <artifactId>rya.api.function</artifactId>
-    <name>Apache Rya Common API - Functions</name>
+    <artifactId>rya.api.evaluation</artifactId>
+    <name>Apache Rya Common API - Evaluation Functions</name>
 
     <dependencies>
         <!-- Rya dependencies. -->        

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageFunction.java
index a73d5ac..4a31fce 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageFunction.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageFunction.java
@@ -19,6 +19,7 @@
 package org.apache.rya.api.function.aggregation;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -51,6 +52,8 @@ public final class AverageFunction implements AggregationFunction {
     @Override
     public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
         checkArgument(aggregation.getAggregationType() == AggregationType.AVERAGE, "The AverageFunction only accepts AVERAGE AggregationElements.");
+        requireNonNull(state);
+        requireNonNull(childBindingSet);
 
         // Only update the average if the child contains the binding that we are averaging.
         final String aggregatedName = aggregation.getAggregatedBindingName();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/CountFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/CountFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/CountFunction.java
index 7dd5b21..879df5e 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/CountFunction.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/CountFunction.java
@@ -19,6 +19,7 @@
 package org.apache.rya.api.function.aggregation;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
 
 import java.math.BigInteger;
 
@@ -39,6 +40,8 @@ public final class CountFunction implements AggregationFunction {
     @Override
     public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
         checkArgument(aggregation.getAggregationType() == AggregationType.COUNT, "The CountFunction only accepts COUNT AggregationElements.");
+        requireNonNull(state);
+        requireNonNull(childBindingSet);
 
         // Only add one to the count if the child contains the binding that we are counting.
         final String aggregatedName = aggregation.getAggregatedBindingName();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MaxFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MaxFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MaxFunction.java
index 3295fbb..5b5d493 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MaxFunction.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MaxFunction.java
@@ -19,6 +19,7 @@
 package org.apache.rya.api.function.aggregation;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
 
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.openrdf.model.Value;
@@ -40,6 +41,8 @@ public final class MaxFunction implements AggregationFunction {
     @Override
     public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
         checkArgument(aggregation.getAggregationType() == AggregationType.MAX, "The MaxFunction only accepts MAX AggregationElements.");
+        requireNonNull(state);
+        requireNonNull(childBindingSet);
 
         // Only update the max if the child contains the binding that we are finding the max value for.
         final String aggregatedName = aggregation.getAggregatedBindingName();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MinFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MinFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MinFunction.java
index d6bf751..f1b083c 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MinFunction.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MinFunction.java
@@ -19,6 +19,7 @@
 package org.apache.rya.api.function.aggregation;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
 
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.openrdf.model.Value;
@@ -40,6 +41,8 @@ public final class MinFunction implements AggregationFunction {
     @Override
     public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
         checkArgument(aggregation.getAggregationType() == AggregationType.MIN, "The MinFunction only accepts MIN AggregationElements.");
+        requireNonNull(state);
+        requireNonNull(childBindingSet);
 
         // Only update the min if the child contains the binding that we are finding the min value for.
         final String aggregatedName = aggregation.getAggregatedBindingName();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/SumFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/SumFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/SumFunction.java
index 97735f2..7ddc9ae 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/SumFunction.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/SumFunction.java
@@ -19,6 +19,7 @@
 package org.apache.rya.api.function.aggregation;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static java.util.Objects.requireNonNull;
 
 import java.math.BigInteger;
 
@@ -48,6 +49,8 @@ public final class SumFunction implements AggregationFunction {
     @Override
     public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
         checkArgument(aggregation.getAggregationType() == AggregationType.SUM, "The SumFunction only accepts SUM AggregationElements.");
+        requireNonNull(state);
+        requireNonNull(childBindingSet);
 
         // Only add values to the sum if the child contains the binding that we are summing.
         final String aggregatedName = aggregation.getAggregatedBindingName();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/common/rya.api/src/main/java/org/apache/rya/api/utils/CloseableIterator.java
----------------------------------------------------------------------
diff --git a/common/rya.api/src/main/java/org/apache/rya/api/utils/CloseableIterator.java b/common/rya.api/src/main/java/org/apache/rya/api/utils/CloseableIterator.java
new file mode 100644
index 0000000..c29f5e0
--- /dev/null
+++ b/common/rya.api/src/main/java/org/apache/rya/api/utils/CloseableIterator.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.utils;
+
+import java.util.Iterator;
+
+/**
+ * An {@link Iterator} that also extends {@link AutoCloseable} because it has reference to resources
+ * that need to be released once you are done iterating.
+ *
+ * @param <T> The type of object that is iterated over.
+ */
+public interface CloseableIterator<T> extends Iterator<T>, AutoCloseable {
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/AccumuloBatchUpdatePCJIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/AccumuloBatchUpdatePCJIT.java b/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/AccumuloBatchUpdatePCJIT.java
index 40941c8..5028454 100644
--- a/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/AccumuloBatchUpdatePCJIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/api/client/accumulo/AccumuloBatchUpdatePCJIT.java
@@ -27,11 +27,11 @@ import org.apache.rya.accumulo.AccumuloITBase;
 import org.apache.rya.accumulo.AccumuloRdfConfiguration;
 import org.apache.rya.api.client.Install.InstallConfiguration;
 import org.apache.rya.api.client.RyaClient;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.external.PrecomputedJoinIndexerConfig.PrecomputedJoinStorageType;
 import org.apache.rya.indexing.external.PrecomputedJoinIndexerConfig.PrecomputedJoinUpdaterType;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
 import org.apache.rya.sail.config.RyaSailFactory;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/periodic.notification/service/src/main/java/org/apache/rya/periodic/notification/processor/TimestampedNotificationProcessor.java
----------------------------------------------------------------------
diff --git a/extras/periodic.notification/service/src/main/java/org/apache/rya/periodic/notification/processor/TimestampedNotificationProcessor.java b/extras/periodic.notification/service/src/main/java/org/apache/rya/periodic/notification/processor/TimestampedNotificationProcessor.java
index ae586da..dcc47b6 100644
--- a/extras/periodic.notification/service/src/main/java/org/apache/rya/periodic/notification/processor/TimestampedNotificationProcessor.java
+++ b/extras/periodic.notification/service/src/main/java/org/apache/rya/periodic/notification/processor/TimestampedNotificationProcessor.java
@@ -22,8 +22,8 @@ import java.util.Optional;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryResultStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.periodic.notification.api.BinPruner;
 import org.apache.rya.periodic.notification.api.BindingSetRecord;
 import org.apache.rya.periodic.notification.api.NodeBin;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/application/PeriodicNotificationApplicationIT.java
----------------------------------------------------------------------
diff --git a/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/application/PeriodicNotificationApplicationIT.java b/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/application/PeriodicNotificationApplicationIT.java
index 92e3276..cd06f2a 100644
--- a/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/application/PeriodicNotificationApplicationIT.java
+++ b/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/application/PeriodicNotificationApplicationIT.java
@@ -54,6 +54,7 @@ import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.rya.api.resolver.RdfToRyaConversions;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.pcj.fluo.api.CreatePeriodicQuery;
 import org.apache.rya.indexing.pcj.fluo.api.InsertTriples;
@@ -61,7 +62,6 @@ import org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants;
 import org.apache.rya.indexing.pcj.fluo.app.util.FluoClientFactory;
 import org.apache.rya.indexing.pcj.fluo.app.util.FluoQueryUtils;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryResultStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPeriodicQueryResultStorage;
 import org.apache.rya.pcj.fluo.test.base.RyaExportITBase;
 import org.apache.rya.periodic.notification.notification.CommandNotification;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/pruner/PeriodicNotificationBinPrunerIT.java
----------------------------------------------------------------------
diff --git a/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/pruner/PeriodicNotificationBinPrunerIT.java b/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/pruner/PeriodicNotificationBinPrunerIT.java
index 830fa46..ac2202c 100644
--- a/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/pruner/PeriodicNotificationBinPrunerIT.java
+++ b/extras/periodic.notification/tests/src/test/java/org/apache/rya/periodic/notification/pruner/PeriodicNotificationBinPrunerIT.java
@@ -39,6 +39,7 @@ import org.apache.fluo.api.data.ColumnValue;
 import org.apache.fluo.api.data.Span;
 import org.apache.fluo.core.client.FluoClientImpl;
 import org.apache.rya.api.resolver.RdfToRyaConversions;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.fluo.api.CreatePeriodicQuery;
 import org.apache.rya.indexing.pcj.fluo.api.InsertTriples;
 import org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants;
@@ -48,7 +49,6 @@ import org.apache.rya.indexing.pcj.fluo.app.util.PeriodicQueryUtil;
 import org.apache.rya.indexing.pcj.fluo.app.util.RowKeyUtil;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryResultStorage;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryStorageException;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPeriodicQueryResultStorage;
 import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
 import org.apache.rya.pcj.fluo.test.base.RyaExportITBase;
@@ -68,7 +68,7 @@ import com.google.common.collect.Sets;
 
 public class PeriodicNotificationBinPrunerIT extends RyaExportITBase {
 
-    
+
     @Test
     public void periodicPrunerTest() throws Exception {
 
@@ -238,7 +238,7 @@ public class PeriodicNotificationBinPrunerIT extends RyaExportITBase {
         pruner.stop();
 
     }
-    
+
     private void compareResults(PeriodicQueryResultStorage periodicStorage, String queryId, long bin, Set<BindingSet> expected) throws PeriodicQueryStorageException, Exception {
         try(CloseableIterator<BindingSet> iter = periodicStorage.listResults(queryId, Optional.of(bin))) {
             Set<BindingSet> actual = new HashSet<>();
@@ -248,13 +248,13 @@ public class PeriodicNotificationBinPrunerIT extends RyaExportITBase {
             Assert.assertEquals(expected, actual);
         }
     }
-    
+
     private void compareFluoCounts(FluoClient client, String pcjId, long bin) {
         QueryBindingSet bs = new QueryBindingSet();
         bs.addBinding(IncrementalUpdateConstants.PERIODIC_BIN_ID, new LiteralImpl(Long.toString(bin), XMLSchema.LONG));
-        
+
         VariableOrder varOrder = new VariableOrder(IncrementalUpdateConstants.PERIODIC_BIN_ID);
-        
+
         try(Snapshot sx = client.newSnapshot()) {
             String fluoQueryId = NodeType.generateNewIdForType(NodeType.QUERY, pcjId);
             Set<String> ids = new HashSet<>();
@@ -279,5 +279,5 @@ public class PeriodicNotificationBinPrunerIT extends RyaExportITBase {
             }
         }
     }
-    
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PeriodicQueryResultStorage.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PeriodicQueryResultStorage.java b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PeriodicQueryResultStorage.java
index 6637dde..2936738 100644
--- a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PeriodicQueryResultStorage.java
+++ b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PeriodicQueryResultStorage.java
@@ -23,7 +23,7 @@ import java.util.List;
 import java.util.Optional;
 
 import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
 import org.openrdf.query.BindingSet;
 
@@ -32,7 +32,7 @@ import org.openrdf.query.BindingSet;
  *
  */
 public interface PeriodicQueryResultStorage {
-    
+
     /**
      * Binding name for the periodic bin id
      */
@@ -45,27 +45,27 @@ public interface PeriodicQueryResultStorage {
      * @throws PeriodicQueryStorageException
      */
     public String createPeriodicQuery(String sparql) throws PeriodicQueryStorageException;
-    
+
     /**
      * Creates a PeriodicQuery result storage layer for the given SPARQL query with the given id
      * @param queryId - id of the storage layer for the given SPARQL query
      * @param sparql - SPARQL query whose periodic results will be stored
-     * @return - id of the storage layer 
+     * @return - id of the storage layer
      * @throws PeriodicQueryStorageException
      */
     public String createPeriodicQuery(String queryId, String sparql) throws PeriodicQueryStorageException;
-    
+
     /**
      * Creates a PeriodicQuery result storage layer for the given SPARQL query with the given id
      * whose results are written in the order indicated by the specified VariableOrder.
      * @param queryId - id of the storage layer for the given SPARQL query
      * @param sparql - SPARQL query whose periodic results will be stored
      * @param varOrder - VariableOrder indicating the order that results will be written in
-     * @return - id of the storage layer 
+     * @return - id of the storage layer
      * @throws PeriodicQueryStorageException
      */
     public void createPeriodicQuery(String queryId, String sparql, VariableOrder varOrder) throws PeriodicQueryStorageException;
-    
+
     /**
      * Retrieve the {@link PeriodicQueryStorageMetdata} for the give query id
      * @param queryID - id of the query whose metadata will be returned
@@ -73,7 +73,7 @@ public interface PeriodicQueryResultStorage {
      * @throws PeriodicQueryStorageException
      */
     public PeriodicQueryStorageMetadata getPeriodicQueryMetadata(String queryID) throws PeriodicQueryStorageException;;
-    
+
     /**
      * Add periodic query results to the storage layer indicated by the given query id
      * @param queryId - id indicating the storage layer that results will be added to
@@ -81,7 +81,7 @@ public interface PeriodicQueryResultStorage {
      * @throws PeriodicQueryStorageException
      */
     public void addPeriodicQueryResults(String queryId, Collection<VisibilityBindingSet> results) throws PeriodicQueryStorageException;;
-    
+
     /**
      * Deletes periodic query results from the storage layer
      * @param queryId - id indicating the storage layer that results will be deleted from
@@ -89,14 +89,14 @@ public interface PeriodicQueryResultStorage {
      * @throws PeriodicQueryStorageException
      */
     public void deletePeriodicQueryResults(String queryId, long binID) throws PeriodicQueryStorageException;;
-    
+
     /**
-     * Deletes all results for the storage layer indicated by the given query id 
+     * Deletes all results for the storage layer indicated by the given query id
      * @param queryID - id indicating the storage layer whose results will be deleted
      * @throws PeriodicQueryStorageException
      */
     public void deletePeriodicQuery(String queryID) throws PeriodicQueryStorageException;;
-    
+
     /**
      * List results in the given storage layer indicated by the query id
      * @param queryId - id indicating the storage layer whose results will be listed
@@ -105,11 +105,11 @@ public interface PeriodicQueryResultStorage {
      * @throws PeriodicQueryStorageException
      */
     public CloseableIterator<BindingSet> listResults(String queryId, Optional<Long> binID) throws PeriodicQueryStorageException;;
-    
+
     /**
      * List all storage tables containing periodic results.
      * @return List of Strings with names of all tables containing periodic results
      */
     public List<String> listPeriodicTables();
-    
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PrecomputedJoinStorage.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PrecomputedJoinStorage.java b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PrecomputedJoinStorage.java
index 4988035..70c8b0e 100644
--- a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PrecomputedJoinStorage.java
+++ b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/PrecomputedJoinStorage.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.openrdf.query.BindingSet;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
@@ -103,16 +104,6 @@ public interface PrecomputedJoinStorage extends AutoCloseable {
     public void close() throws PCJStorageException;
 
     /**
-     * An {@link Iterator} that also extends {@link AutoCloseable} because it has reference to resources
-     * that need to be released once you are done iterating.
-     *
-     * @param <T> The type of object that is iterated over.
-     */
-    public static interface CloseableIterator<T> extends Iterator<T>, AutoCloseable {
-
-    }
-
-    /**
      * An operation of {@link PrecomputedJoinStorage} failed.
      */
     public static class PCJStorageException extends PcjException {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPcjStorage.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPcjStorage.java b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPcjStorage.java
index 3d0f11b..f3d078d 100644
--- a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPcjStorage.java
+++ b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPcjStorage.java
@@ -39,6 +39,7 @@ import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryExce
 import org.apache.rya.api.instance.RyaDetailsUpdater;
 import org.apache.rya.api.instance.RyaDetailsUpdater.RyaDetailsMutator.CouldNotApplyMutationException;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PCJIdFactory;
 import org.apache.rya.indexing.pcj.storage.PcjMetadata;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPeriodicQueryResultStorage.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPeriodicQueryResultStorage.java b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPeriodicQueryResultStorage.java
index f8547f5..8124aff 100644
--- a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPeriodicQueryResultStorage.java
+++ b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloPeriodicQueryResultStorage.java
@@ -38,11 +38,11 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PCJIdFactory;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryResultStorage;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryStorageException;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryStorageMetadata;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetConverter.BindingSetConversionException;
 import org.openrdf.model.impl.LiteralImpl;
 import org.openrdf.model.vocabulary.XMLSchema;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloValueBindingSetIterator.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloValueBindingSetIterator.java b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloValueBindingSetIterator.java
index ff8ff14..c488d36 100644
--- a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloValueBindingSetIterator.java
+++ b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/AccumuloValueBindingSetIterator.java
@@ -25,8 +25,8 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.fluo.api.data.Bytes;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryResultStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.openrdf.query.BindingSet;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTables.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTables.java b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTables.java
index 40db32a..9346c00 100644
--- a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTables.java
+++ b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTables.java
@@ -56,10 +56,9 @@ import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PcjMetadata;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
-import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetConverter.BindingSetConversionException;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.MalformedQueryException;
 import org.openrdf.query.QueryEvaluationException;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/ScannerBindingSetIterator.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/ScannerBindingSetIterator.java b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/ScannerBindingSetIterator.java
index 26fd8c9..b457dfd 100644
--- a/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/ScannerBindingSetIterator.java
+++ b/extras/rya.indexing.pcj/src/main/java/org/apache/rya/indexing/pcj/storage/accumulo/ScannerBindingSetIterator.java
@@ -27,7 +27,7 @@ import java.util.Map.Entry;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetConverter.BindingSetConversionException;
 import org.openrdf.query.BindingSet;
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIT.java b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIT.java
index e689f9d..b95c812 100644
--- a/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIT.java
+++ b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIT.java
@@ -45,9 +45,9 @@ import org.apache.rya.accumulo.MiniAccumuloSingleton;
 import org.apache.rya.accumulo.RyaTestInstanceRule;
 import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PcjException;
 import org.apache.rya.indexing.pcj.storage.PcjMetadata;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
 import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetConverter.BindingSetConversionException;
 import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
@@ -120,7 +120,7 @@ public class PcjTablesIT {
     private String getRyaInstanceName() {
         return testInstance.getRyaInstanceName();
     }
-    
+
     /**
      * Format a Mini Accumulo to be a Rya repository.
      *

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPcjStorageIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPcjStorageIT.java b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPcjStorageIT.java
index 5ba5e40..33571f7 100644
--- a/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPcjStorageIT.java
+++ b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPcjStorageIT.java
@@ -39,9 +39,9 @@ import org.apache.rya.api.instance.RyaDetailsRepository;
 import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
 import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PcjMetadata;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
 import org.apache.rya.indexing.pcj.storage.accumulo.ShiftVarOrderFactory;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPeriodicQueryResultStorageIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPeriodicQueryResultStorageIT.java b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPeriodicQueryResultStorageIT.java
index 1e21bf2..2d9da4d 100644
--- a/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPeriodicQueryResultStorageIT.java
+++ b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/integration/AccumuloPeriodicQueryResultStorageIT.java
@@ -30,10 +30,10 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.rya.accumulo.AccumuloITBase;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryResultStorage;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryStorageException;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryStorageMetadata;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPeriodicQueryResultStorage;
 import org.apache.rya.indexing.pcj.storage.accumulo.PeriodicQueryTableNameFactory;
 import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
@@ -53,86 +53,86 @@ public class AccumuloPeriodicQueryResultStorageIT extends AccumuloITBase {
     private static final String RYA = "rya_";
     private static final PeriodicQueryTableNameFactory nameFactory = new PeriodicQueryTableNameFactory();
     private static final ValueFactory vf = new ValueFactoryImpl();
-    
+
     @Before
     public void init() throws AccumuloException, AccumuloSecurityException {
         super.getConnector().securityOperations().changeUserAuthorizations("root", new Authorizations("U"));
         periodicStorage = new AccumuloPeriodicQueryResultStorage(super.getConnector(), RYA);
     }
-    
-    
+
+
     @Test
     public void testCreateAndMeta() throws PeriodicQueryStorageException {
-        
+
         String sparql = "select ?x where { ?x <urn:pred> ?y.}";
         VariableOrder varOrder = new VariableOrder("periodicBinId", "x");
         PeriodicQueryStorageMetadata expectedMeta = new PeriodicQueryStorageMetadata(sparql, varOrder);
-        
+
         String id = periodicStorage.createPeriodicQuery(sparql);
         Assert.assertEquals(expectedMeta, periodicStorage.getPeriodicQueryMetadata(id));
         Assert.assertEquals(Arrays.asList(nameFactory.makeTableName(RYA, id)), periodicStorage.listPeriodicTables());
         periodicStorage.deletePeriodicQuery(id);
     }
-    
-    
+
+
     @Test
     public void testAddListDelete() throws Exception {
-        
+
         String sparql = "select ?x where { ?x <urn:pred> ?y.}";
         String id = periodicStorage.createPeriodicQuery(sparql);
-        
+
         Set<BindingSet> expected = new HashSet<>();
         Set<VisibilityBindingSet> storageSet = new HashSet<>();
-        
+
         //add result matching user's visibility
         QueryBindingSet bs = new QueryBindingSet();
         bs.addBinding("periodicBinId", vf.createLiteral(1L));
         bs.addBinding("x",vf.createURI("uri:uri123"));
         expected.add(bs);
         storageSet.add(new VisibilityBindingSet(bs,"U"));
-        
+
         //add result with different visibility that is not expected
         bs = new QueryBindingSet();
         bs.addBinding("periodicBinId", vf.createLiteral(1L));
         bs.addBinding("x",vf.createURI("uri:uri456"));
         storageSet.add(new VisibilityBindingSet(bs,"V"));
-        
+
         periodicStorage.addPeriodicQueryResults(id, storageSet);
-        
+
         Set<BindingSet> actual = new HashSet<>();
         try(CloseableIterator<BindingSet> iter = periodicStorage.listResults(id, Optional.of(1L))) {
             iter.forEachRemaining(x -> actual.add(x));
         }
-        
+
         Assert.assertEquals(expected, actual);
-        
+
         periodicStorage.deletePeriodicQueryResults(id, 1L);
-        
+
         Set<BindingSet> actual2 = new HashSet<>();
         try(CloseableIterator<BindingSet> iter = periodicStorage.listResults(id, Optional.of(1L))) {
             iter.forEachRemaining(x -> actual2.add(x));
         }
-        
+
         Assert.assertEquals(new HashSet<>(), actual2);
         periodicStorage.deletePeriodicQuery(id);
-        
+
     }
-    
+
     @Test
     public void multiBinTest() throws PeriodicQueryStorageException, Exception {
-        
+
         String sparql = "prefix function: <http://org.apache.rya/function#> " //n
                 + "prefix time: <http://www.w3.org/2006/time#> " //n
                 + "select ?id (count(?obs) as ?total) where {" //n
                 + "Filter(function:periodic(?time, 2, .5, time:hours)) " //n
                 + "?obs <uri:hasTime> ?time. " //n
                 + "?obs <uri:hasId> ?id } group by ?id"; //n
-        
-        
+
+
         final ValueFactory vf = new ValueFactoryImpl();
         long currentTime = System.currentTimeMillis();
         String queryId = UUID.randomUUID().toString().replace("-", "");
-        
+
         // Create the expected results of the SPARQL query once the PCJ has been computed.
         final Set<BindingSet> expected1 = new HashSet<>();
         final Set<BindingSet> expected2 = new HashSet<>();
@@ -142,81 +142,81 @@ public class AccumuloPeriodicQueryResultStorageIT extends AccumuloITBase {
 
         long period = 1800000;
         long binId = (currentTime/period)*period;
-        
+
         MapBindingSet bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("2", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_1", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId));
         expected1.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
+
         bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("2", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_2", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId));
         expected1.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
+
         bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("1", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_3", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId));
         expected1.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
+
         bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("1", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_4", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId));
         expected1.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
+
         bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("1", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_1", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId + period));
         expected2.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
+
         bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("2", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_2", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId + period));
         expected2.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
+
         bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("1", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_3", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId + period));
         expected2.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
+
         bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("1", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_1", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId + 2*period));
         expected3.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
+
         bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("1", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_2", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId + 2*period));
         expected3.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
+
         bs = new MapBindingSet();
         bs.addBinding("total", vf.createLiteral("1", XMLSchema.INTEGER));
         bs.addBinding("id", vf.createLiteral("id_1", XMLSchema.STRING));
         bs.addBinding("periodicBinId", vf.createLiteral(binId + 3*period));
         expected4.add(bs);
         storageResults.add(new VisibilityBindingSet(bs));
-        
-        
+
+
         String id = periodicStorage.createPeriodicQuery(queryId, sparql);
         periodicStorage.addPeriodicQueryResults(queryId, storageResults);
-        
+
         try(CloseableIterator<BindingSet> iter = periodicStorage.listResults(queryId, Optional.of(binId))) {
             Set<BindingSet> actual1 = new HashSet<>();
             while(iter.hasNext()) {
@@ -224,7 +224,7 @@ public class AccumuloPeriodicQueryResultStorageIT extends AccumuloITBase {
             }
             Assert.assertEquals(expected1, actual1);
         }
-        
+
         periodicStorage.deletePeriodicQueryResults(queryId, binId);
         try(CloseableIterator<BindingSet> iter = periodicStorage.listResults(queryId, Optional.of(binId))) {
             Set<BindingSet> actual1 = new HashSet<>();
@@ -233,7 +233,7 @@ public class AccumuloPeriodicQueryResultStorageIT extends AccumuloITBase {
             }
             Assert.assertEquals(Collections.emptySet(), actual1);
         }
-        
+
         try(CloseableIterator<BindingSet> iter = periodicStorage.listResults(queryId, Optional.of(binId + period))) {
             Set<BindingSet> actual2 = new HashSet<>();
             while(iter.hasNext()) {
@@ -241,7 +241,7 @@ public class AccumuloPeriodicQueryResultStorageIT extends AccumuloITBase {
             }
             Assert.assertEquals(expected2, actual2);
         }
-        
+
         periodicStorage.deletePeriodicQueryResults(queryId, binId + period);
         try(CloseableIterator<BindingSet> iter = periodicStorage.listResults(queryId, Optional.of(binId + period))) {
             Set<BindingSet> actual2 = new HashSet<>();
@@ -250,7 +250,7 @@ public class AccumuloPeriodicQueryResultStorageIT extends AccumuloITBase {
             }
             Assert.assertEquals(Collections.emptySet(), actual2);
         }
-        
+
         try(CloseableIterator<BindingSet> iter = periodicStorage.listResults(queryId, Optional.of(binId + 2*period))) {
             Set<BindingSet> actual3 = new HashSet<>();
             while(iter.hasNext()) {
@@ -258,7 +258,7 @@ public class AccumuloPeriodicQueryResultStorageIT extends AccumuloITBase {
             }
             Assert.assertEquals(expected3, actual3);
         }
-        
+
         try(CloseableIterator<BindingSet> iter = periodicStorage.listResults(queryId, Optional.of(binId + 3*period))) {
             Set<BindingSet> actual4 = new HashSet<>();
             while(iter.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml b/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml
index f2e8cf9..5493a5f 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/pom.xml
@@ -62,7 +62,7 @@ under the License.
         
         <dependency>
             <groupId>org.apache.rya</groupId>
-            <artifactId>rya.api.function</artifactId>
+            <artifactId>rya.api.evaluation</artifactId>
         </dependency>
 
         <!-- 3rd Party Runtime Dependencies. -->

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.pcj.fluo/pcj.fluo.demo/src/main/java/org/apache/rya/indexing/pcj/fluo/demo/FluoAndHistoricPcjsDemo.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.demo/src/main/java/org/apache/rya/indexing/pcj/fluo/demo/FluoAndHistoricPcjsDemo.java b/extras/rya.pcj.fluo/pcj.fluo.demo/src/main/java/org/apache/rya/indexing/pcj/fluo/demo/FluoAndHistoricPcjsDemo.java
index 3fea6ed..181f322 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.demo/src/main/java/org/apache/rya/indexing/pcj/fluo/demo/FluoAndHistoricPcjsDemo.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.demo/src/main/java/org/apache/rya/indexing/pcj/fluo/demo/FluoAndHistoricPcjsDemo.java
@@ -33,12 +33,12 @@ import org.apache.rya.api.domain.RyaType;
 import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.api.persist.RyaDAOException;
 import org.apache.rya.api.resolver.RyaToRdfConversions;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.fluo.api.CreateFluoPcj;
 import org.apache.rya.indexing.pcj.fluo.api.InsertTriples;
 import org.apache.rya.indexing.pcj.fluo.app.query.UnsupportedQueryException;
 import org.apache.rya.indexing.pcj.storage.PcjException;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
 import org.apache.rya.rdftriplestore.RyaSailRepository;
 import org.openrdf.model.Statement;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/InputIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/InputIT.java b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/InputIT.java
index d623043..866d32b 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/InputIT.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/InputIT.java
@@ -29,10 +29,10 @@ import org.apache.fluo.api.client.FluoClient;
 import org.apache.fluo.api.client.FluoFactory;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.fluo.api.CreateFluoPcj;
 import org.apache.rya.indexing.pcj.fluo.api.InsertTriples;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
 import org.apache.rya.pcj.fluo.test.base.RyaExportITBase;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/QueryIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/QueryIT.java b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/QueryIT.java
index 3e72f1b..610f502 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/QueryIT.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/QueryIT.java
@@ -39,11 +39,11 @@ import org.apache.rya.api.client.RyaClient;
 import org.apache.rya.api.client.accumulo.AccumuloRyaClientFactory;
 import org.apache.rya.api.functions.DateTimeWithinPeriod;
 import org.apache.rya.api.functions.OWLTime;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.fluo.api.CreateFluoPcj;
 import org.apache.rya.indexing.pcj.fluo.app.query.UnsupportedQueryException;
 import org.apache.rya.indexing.pcj.storage.PeriodicQueryResultStorage;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPeriodicQueryResultStorage;
 import org.apache.rya.pcj.fluo.test.base.RyaExportITBase;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/RyaInputIncrementalUpdateIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/RyaInputIncrementalUpdateIT.java b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/RyaInputIncrementalUpdateIT.java
index 5cd3ab1..65083e8 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/RyaInputIncrementalUpdateIT.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/RyaInputIncrementalUpdateIT.java
@@ -27,10 +27,10 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.fluo.api.client.FluoClient;
 import org.apache.fluo.api.client.FluoFactory;
 import org.apache.rya.accumulo.AccumuloRyaDAO;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.external.PrecomputedJoinIndexer;
 import org.apache.rya.indexing.pcj.fluo.api.CreateFluoPcj;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
 import org.apache.rya.indexing.pcj.update.PrecomputedJoinUpdater;
 import org.apache.rya.pcj.fluo.test.base.RyaExportITBase;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/StreamingTestIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/StreamingTestIT.java b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/StreamingTestIT.java
index e83a894..6135920 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/StreamingTestIT.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/integration/StreamingTestIT.java
@@ -28,9 +28,9 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.fluo.api.client.FluoClient;
 import org.apache.fluo.api.client.FluoFactory;
 import org.apache.log4j.Logger;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.pcj.fluo.api.CreateFluoPcj;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
 import org.apache.rya.pcj.fluo.test.base.RyaExportITBase;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/visibility/PcjVisibilityIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/visibility/PcjVisibilityIT.java b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/visibility/PcjVisibilityIT.java
index 8529bd5..90ed01a 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/visibility/PcjVisibilityIT.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.integration/src/test/java/org/apache/rya/indexing/pcj/fluo/visibility/PcjVisibilityIT.java
@@ -47,12 +47,12 @@ import org.apache.rya.api.client.RyaClient;
 import org.apache.rya.api.client.accumulo.AccumuloRyaClientFactory;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.external.PrecomputedJoinIndexerConfig;
 import org.apache.rya.indexing.pcj.fluo.api.CreateFluoPcj;
 import org.apache.rya.indexing.pcj.fluo.api.InsertTriples;
 import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
 import org.apache.rya.indexing.pcj.storage.accumulo.AccumuloPcjStorage;
 import org.apache.rya.indexing.pcj.storage.accumulo.PcjTableNameFactory;
 import org.apache.rya.pcj.fluo.test.base.RyaExportITBase;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.streams/api/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/pom.xml b/extras/rya.streams/api/pom.xml
index 250028f..55c0e79 100644
--- a/extras/rya.streams/api/pom.xml
+++ b/extras/rya.streams/api/pom.xml
@@ -54,11 +54,6 @@ under the License.
             <artifactId>guava</artifactId>
         </dependency>
         
-        <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
-        </dependency>
-        
         <!-- Test dependences -->
         <dependency>
             <groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.streams/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/pom.xml b/extras/rya.streams/kafka/pom.xml
index 778630d..16b07b2 100644
--- a/extras/rya.streams/kafka/pom.xml
+++ b/extras/rya.streams/kafka/pom.xml
@@ -60,7 +60,7 @@ under the License.
         </dependency>
         <dependency>
             <groupId>org.apache.rya</groupId>
-            <artifactId>rya.api.function</artifactId>
+            <artifactId>rya.api.evaluation</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.rya</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
index 5f7a06b..124bc76 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
@@ -36,7 +36,7 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * {@link VisibilityBindingSet} because some downstream processors require more information about
  * which upstream processor is emitting the result in order to do their work.
  * </p>
- * Currently there are only two types processors:
+ * Currently there are only two types of processors:
  * <ul>
  *   <li>Unary Processor - A processor that only has a single upstream node feeding it input.</li>
  *   <li>Binary Processor - A processor that has two upstream nodes feeding it input.</li>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/CloseableIterator.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/CloseableIterator.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/CloseableIterator.java
deleted file mode 100644
index 9ea927d..0000000
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/CloseableIterator.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors.join;
-
-import java.util.Iterator;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-
-/**
- * An {@link Iterator} that is also {@link AutoCloseable}.
- *
- * @param <T> - The type of elements that will be iterated over.
- */
-@DefaultAnnotation(NonNull.class)
-public interface CloseableIterator<T> extends Iterator<T>, AutoCloseable { }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorSupplier.java
index 9ed2363..367ca6f 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorSupplier.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorSupplier.java
@@ -28,6 +28,7 @@ import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.rya.api.function.join.IterativeJoin;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.streams.kafka.processors.ProcessorResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
@@ -75,7 +76,7 @@ public class JoinProcessorSupplier extends RyaStreamsProcessorSupplier {
         this.allVars = requireNonNull(allVars);
 
         if(!allVars.subList(0, joinVars.size()).equals(joinVars)) {
-            throw new IllegalArgumentException("All vars must be lead by the join vars, but it did not. " +
+            throw new IllegalArgumentException("The allVars list must start with the joinVars list, but it did not. " +
                     "Join Vars: " + joinVars + ", All Vars: " + allVars);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
index 17a6ebb..2afc1d8 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/JoinStateStore.java
@@ -19,6 +19,7 @@
 package org.apache.rya.streams.kafka.processors.join;
 
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
index d12957a..254f226 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/join/KeyValueJoinStateStore.java
@@ -28,6 +28,7 @@ import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.state.KeyValueIterator;
 import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.utils.CloseableIterator;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
 import org.openrdf.query.impl.MapBindingSet;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/extras/rya.streams/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/pom.xml b/extras/rya.streams/pom.xml
index dd876a0..93b6b1c 100644
--- a/extras/rya.streams/pom.xml
+++ b/extras/rya.streams/pom.xml
@@ -38,7 +38,15 @@
         <module>kafka-test</module>
         <module>api</module>
         <module>client</module>
-        <module>geo</module>
         <module>integration</module>
     </modules>
+    
+    <profiles>
+        <profile>
+            <id>geoindexing</id>
+            <modules>
+                <module>geo</module>
+           </modules>
+        </profile>
+    </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/f0725df5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 31b17f8..99640f2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -217,7 +217,7 @@ under the License.
             </dependency>
             <dependency>
                 <groupId>org.apache.rya</groupId>
-                <artifactId>rya.api.function</artifactId>
+                <artifactId>rya.api.evaluation</artifactId>
                 <version>${project.version}</version>
             </dependency>
             <dependency>


[38/50] [abbrv] incubator-rya git commit: RYA-377 Update the Projection processor to handle Construct queries.

Posted by ca...@apache.org.
RYA-377 Update the Projection processor to handle Construct queries.


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

Branch: refs/heads/master
Commit: 538393fe2dc303f14fa87beea7e777b9d6be717b
Parents: 83d09f4
Author: kchilton2 <ke...@gmail.com>
Authored: Wed Nov 15 19:46:47 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../api/function/projection/BNodeIdFactory.java |  34 +++
 .../projection/MultiProjectionEvaluator.java    | 130 +++++++++++
 .../projection/ProjectionEvaluator.java         | 187 ++++++++++++++++
 .../function/projection/RandomUUIDFactory.java  |  36 +++
 .../MultiProjectionEvaluatorTest.java           | 193 ++++++++++++++++
 .../projection/ProjectionEvaluatorTest.java     | 221 +++++++++++++++++++
 .../rya/api/model/VisibilityStatement.java      |   5 +
 .../kafka/processors/ProcessorResult.java       |  48 ++++
 .../StatementPatternProcessorSupplier.java      |   2 +
 .../MultiProjectionProcessorSupplier.java       | 119 ++++++++++
 .../projection/ProjectionProcessorSupplier.java |  58 ++---
 .../apache/rya/streams/kafka/RdfTestUtil.java   |  46 ++++
 .../kafka/processors/ProjectionProcessorIT.java | 152 -------------
 .../projection/MultiProjectionProcessorIT.java  | 155 +++++++++++++
 .../MultiProjectionProcessorTest.java           | 119 ++++++++++
 .../projection/ProjectionProcessorIT.java       | 115 ++++++++++
 .../projection/ProjectionProcessorTest.java     |  82 +++++++
 17 files changed, 1516 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/BNodeIdFactory.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/BNodeIdFactory.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/BNodeIdFactory.java
new file mode 100644
index 0000000..99f545a
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/BNodeIdFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.projection;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Creates IDs for Blank Nodes.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface BNodeIdFactory {
+
+    /**
+     * @return The ID to use for the next Blank Node.
+     */
+    public String nextId();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/MultiProjectionEvaluator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/MultiProjectionEvaluator.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/MultiProjectionEvaluator.java
new file mode 100644
index 0000000..e2b7046
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/MultiProjectionEvaluator.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.projection;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.model.BNode;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.BNodeGenerator;
+import org.openrdf.query.algebra.Extension;
+import org.openrdf.query.algebra.ExtensionElem;
+import org.openrdf.query.algebra.MultiProjection;
+import org.openrdf.query.algebra.ProjectionElemList;
+import org.openrdf.query.algebra.TupleExpr;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Processes a {@link MultiProjection} node from a SPARQL query.
+ *
+ * @see ProjectionEvaluator
+ */
+@DefaultAnnotation(NonNull.class)
+public class MultiProjectionEvaluator {
+
+    private final ValueFactory vf = new ValueFactoryImpl();
+
+    private final Set<ProjectionEvaluator> projections;
+    private final Set<String> blankNodeSourceNames;
+    private final BNodeIdFactory bNodeIdFactory;
+
+    /**
+     * Constructs an instance of {@link MultiProjection}.
+     *
+     * @param projections - The {@link ProjectionEvaluators} that handle each projection within the multi. (not null)
+     * @param blankNodeSourceNames - If there are blank nodes in the projection, this is a set of their names
+     *   so that they may be re-label to have the same node IDs. (not null)
+     * @param bNodeIdFactory - Creates the IDs for Blank Nodes. (not null)
+     */
+    public MultiProjectionEvaluator(
+            final Set<ProjectionEvaluator> projections,
+            final Set<String> blankNodeSourceNames,
+            final BNodeIdFactory bnodeIdFactory) {
+        this.projections = requireNonNull(projections);
+        this.blankNodeSourceNames = requireNonNull(blankNodeSourceNames);
+        this.bNodeIdFactory = requireNonNull(bnodeIdFactory);
+    }
+
+    /**
+     * Make a {@link MultiProjectionEvaluator} that processes the logic of a {@link MultiProjection}.
+     *
+     * @param multiProjection - Defines the projections that will be processed. (not null)
+     * @param bNodeIdFactory - Creates the IDs for Blank Nodes. (not null)
+     * @return A {@link MultiProjectionEvaluator} for the provided {@link MultiProjection}.
+     */
+    public static MultiProjectionEvaluator make(final MultiProjection multiProjection, final BNodeIdFactory bNodeIdFactory) {
+        requireNonNull(multiProjection);
+
+        // Figure out if there are extensions.
+        final TupleExpr arg = multiProjection.getArg();
+        final Optional<Extension> extension = (arg instanceof Extension) ? Optional.of((Extension)arg): Optional.empty();
+
+        // If there are, iterate through them and find any blank node source names.
+        final Set<String> blankNodeSourceNames = new HashSet<>();
+        if(extension.isPresent()) {
+            for(final ExtensionElem elem : extension.get().getElements()) {
+                if(elem.getExpr() instanceof BNodeGenerator) {
+                    blankNodeSourceNames.add( elem.getName() );
+                }
+            }
+        }
+
+        // Create a ProjectionEvaluator for each projection that is part of the multi.
+        final Set<ProjectionEvaluator> projections = new HashSet<>();
+        for(final ProjectionElemList projectionElemList : multiProjection.getProjections()) {
+            projections.add( new ProjectionEvaluator(projectionElemList, extension) );
+        }
+
+        return new MultiProjectionEvaluator(projections, blankNodeSourceNames, bNodeIdFactory);
+    }
+
+    /**
+     * Apply the projections against a {@link VisibilityBindingSet}.
+     *
+     * @param bs - The value the projection will be applied to. (not null)
+     * @return A set of values that result from the projection.
+     */
+    public Set<VisibilityBindingSet> project(final VisibilityBindingSet bs) {
+        requireNonNull(bs);
+
+        // Generate an ID for each blank node that will appear in the results.
+        final Map<String, BNode> blankNodes = new HashMap<>();
+        for(final String blankNodeSourceName : blankNodeSourceNames) {
+            blankNodes.put(blankNodeSourceName, vf.createBNode(bNodeIdFactory.nextId()));
+        }
+
+        // Iterate through each of the projections and create the results from them.
+        final Set<VisibilityBindingSet> results = new HashSet<>();
+        for(final ProjectionEvaluator projection : projections) {
+            results.add( projection.project(bs, blankNodes) );
+        }
+
+        return results;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/ProjectionEvaluator.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/ProjectionEvaluator.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/ProjectionEvaluator.java
new file mode 100644
index 0000000..a0b59c1
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/ProjectionEvaluator.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.projection;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.model.BNode;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.BNodeGenerator;
+import org.openrdf.query.algebra.Extension;
+import org.openrdf.query.algebra.ExtensionElem;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.algebra.ProjectionElem;
+import org.openrdf.query.algebra.ProjectionElemList;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.algebra.ValueConstant;
+import org.openrdf.query.algebra.ValueExpr;
+import org.openrdf.query.impl.MapBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Processes a {@link Projection} node from a SPARQL query.
+ * </p>
+ * A projection is used transform the bindings that are in a Binding Set. It may do the following things:
+ * <ul>
+ *   <li>Change the binding name for a value.</li>
+ *   <li>Completely remove a binding from the Binding Set.</li>
+ *   <li>Insert a binding that has a constant value.</li>
+ *   <li>Insert a binding with a blank node that identifies some resource.</li>
+ * </ul>
+ * </p>
+ * If you do not supply ID values for the blank nodes that may result from a projection, then a random {@link UUID}
+ * is used.
+ */
+@DefaultAnnotation(NonNull.class)
+public class ProjectionEvaluator {
+
+    private final ValueFactory vf = new ValueFactoryImpl();
+
+    /**
+     * All off the projection elements that define what will appear in the resulting binding sets.
+     */
+    private final ProjectionElemList projectionElems;
+
+    /**
+     * Maps from a ProjectionElem's source name to the constant value that should be used for that name
+     * in resulting binding sets.
+     */
+    private final Map<String, Value> constantSources = new HashMap<>();
+
+    /**
+     * A set of names for the anonymous source names. These values will be blank node UUIDs.
+     */
+    private final Set<String> anonymousSources = new HashSet<>();
+
+    /**
+     * Constructs an instance of {@link ProjectionEvaluator}.
+     *
+     * @param projectionElems - Defines the structure of the resulting value. (not null)
+     * @param extensions - Extra information about the projection elements when there are anonymous constants or blank
+     *   nodes within the projection elements. (not null)
+     */
+    public ProjectionEvaluator(final ProjectionElemList projectionElems, final Optional<Extension> extensions) {
+        this.projectionElems = requireNonNull(projectionElems);
+        requireNonNull(extensions);
+
+        // Find all extensions that represent constant insertions.
+        if(extensions.isPresent()) {
+            for(final ExtensionElem extensionElem : extensions.get().getElements()) {
+                final ValueExpr valueExpr = extensionElem.getExpr();
+
+                // If the extension is a ValueConstant, store it so that they may be added to the binding sets.
+                if(valueExpr instanceof ValueConstant) {
+                    final String sourceName = extensionElem.getName();
+                    final Value targetValue = ((ValueConstant) valueExpr).getValue();
+                    constantSources.put(sourceName, targetValue);
+                }
+
+                // If the extension is a BNodeGenerator, keep track of the name so that we know we have to generate an ID for it.
+                else if(valueExpr instanceof BNodeGenerator) {
+                    final String sourceName = extensionElem.getName();
+                    anonymousSources.add( sourceName );
+                }
+            }
+        }
+    }
+
+    /**
+     * Make a {@link ProjectionEvaluator} that processes the logic of a {@link Projection}.
+     *
+     * @param projection - Defines the projection that will be processed. (not null)
+     * @return A {@link ProjectionEvaluator} for the provided {@link Projection}.
+     */
+    public static ProjectionEvaluator make(final Projection projection) {
+        requireNonNull(projection);
+
+        final ProjectionElemList projectionElems = projection.getProjectionElemList();
+
+        final TupleExpr arg = projection.getArg();
+        final Optional<Extension> extension = arg instanceof Extension ? Optional.of((Extension)arg) : Optional.empty();
+
+        return new ProjectionEvaluator(projectionElems, extension);
+    }
+
+    /**
+     * Applies the projection to a value. If the result has any blank nodes, those nodes will use random UUIDs.
+     * If you want to control what those IDs are, then use {@link #project(VisibilityBindingSet, Map)} instead.
+     *
+     * @param bs - The value the projection will be applied to. (not null)
+     * @return A new value that is the result of the projection.
+     */
+    public VisibilityBindingSet project(final VisibilityBindingSet bs) {
+        return project(bs, new HashMap<>());
+    }
+
+    /**
+     * Applies the projection to a value. If the result has a blank node whose ID is not mapped to a value in
+     * {@code blankNodes}, then a random UUID will be used.
+     *
+     * @param bs - The value the projection will be applied to. (not null)
+     * @param blankNodes - A map from node source names to the blank nodes that will be used for those names. (not null)
+     * @return A new value that is the result of the projection.
+     */
+    public VisibilityBindingSet project(final VisibilityBindingSet bs, final Map<String, BNode> blankNodes) {
+        requireNonNull(bs);
+        requireNonNull(blankNodes);
+
+        // Apply the projection elements against the original binding set.
+        final MapBindingSet result = new MapBindingSet();
+        for (final ProjectionElem elem : projectionElems.getElements()) {
+            final String sourceName = elem.getSourceName();
+
+            Value value = null;
+
+            // If the binding set already has the source name, then use the target name.
+            if (bs.hasBinding(sourceName)) {
+                value = bs.getValue(elem.getSourceName());
+            }
+
+            // If the source name represents a constant value, then use the constant.
+            else if(constantSources.containsKey(sourceName)) {
+                value = constantSources.get(sourceName);
+            }
+
+            // If the source name represents an anonymous value, then create a Blank Node.
+            else if(anonymousSources.contains(sourceName)) {
+                if(blankNodes.containsKey(sourceName)) {
+                    value = blankNodes.get(sourceName);
+                } else {
+                    value = vf.createBNode( UUID.randomUUID().toString() );
+                }
+            }
+
+            result.addBinding(elem.getTargetName(), value);
+        }
+
+        return new VisibilityBindingSet(result, bs.getVisibility());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/RandomUUIDFactory.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/RandomUUIDFactory.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/RandomUUIDFactory.java
new file mode 100644
index 0000000..59740b0
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/projection/RandomUUIDFactory.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.projection;
+
+import java.util.UUID;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * The Blank Node IDs are random {@link UUID}s.
+ */
+@DefaultAnnotation(NonNull.class)
+public class RandomUUIDFactory implements BNodeIdFactory {
+
+    @Override
+    public String nextId() {
+        return UUID.randomUUID().toString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/common/rya.api.function/src/test/java/org/apache/rya/api/function/projection/MultiProjectionEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/test/java/org/apache/rya/api/function/projection/MultiProjectionEvaluatorTest.java b/common/rya.api.function/src/test/java/org/apache/rya/api/function/projection/MultiProjectionEvaluatorTest.java
new file mode 100644
index 0000000..c02e87b
--- /dev/null
+++ b/common/rya.api.function/src/test/java/org/apache/rya/api/function/projection/MultiProjectionEvaluatorTest.java
@@ -0,0 +1,193 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.projection;
+
+import static java.util.Objects.requireNonNull;
+import static org.junit.Assert.assertEquals;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.junit.Test;
+import org.openrdf.model.BNode;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.model.vocabulary.XMLSchema;
+import org.openrdf.query.algebra.MultiProjection;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.impl.MapBindingSet;
+import org.openrdf.query.parser.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
+
+/**
+ * Unit tests the methods of {@link MultiProjectionEvaluator}.
+ */
+public class MultiProjectionEvaluatorTest {
+
+    @Test
+    public void singleBlankNode() throws Exception {
+        // Read the multi projection object from a SPARQL query.
+        final MultiProjection multiProjection = getMultiProjection(
+                "CONSTRUCT {" +
+                    "_:b a <urn:movementObservation> ; " +
+                    "<urn:location> ?location ; " +
+                    "<urn:direction> ?direction ; " +
+                "}" +
+                "WHERE {" +
+                    "?thing <urn:corner> ?location ." +
+                    "?thing <urn:compass> ?direction." +
+                "}");
+
+        // Create a Binding Set that contains the result of the WHERE clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("location", vf.createLiteral("South St and 5th St"));
+        bs.addBinding("direction", vf.createLiteral("NW"));
+        final VisibilityBindingSet original = new VisibilityBindingSet(bs, "a|b");
+
+        // Create the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final String blankNodeId = UUID.randomUUID().toString();
+        final BNode blankNode = vf.createBNode(blankNodeId);
+
+        bs = new MapBindingSet();
+        bs.addBinding("subject", blankNode);
+        bs.addBinding("predicate", RDF.TYPE);
+        bs.addBinding("object", vf.createURI("urn:movementObservation"));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("subject", blankNode);
+        bs.addBinding("predicate", vf.createURI("urn:location"));
+        bs.addBinding("object", vf.createLiteral("South St and 5th St"));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("subject", blankNode);
+        bs.addBinding("predicate", vf.createURI("urn:direction"));
+        bs.addBinding("object", vf.createLiteral("NW"));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        // Run the projection evaluator.
+        final Set<VisibilityBindingSet> results = MultiProjectionEvaluator.make(multiProjection, () -> blankNodeId).project(original);
+
+        // The expected binding sets.
+        assertEquals(expected, results);
+    }
+
+    @Test
+    public void multipleBlanknodes() throws Exception {
+        // Read the multi projection object from a SPARQL query.
+        final MultiProjection multiProjection = getMultiProjection(
+                "CONSTRUCT {" +
+                    "_:b a <urn:vehicle> . " +
+                    "_:b <urn:tiresCount> 4 ." +
+                    "_:c a <urn:pet> . " +
+                    "_:c <urn:isDead> false . " +
+                "}" +
+                "WHERE {" +
+                    "?vehicle <urn:owner> ?owner . " +
+                    "?vehicle <urn:plates> ?plates . " +
+                    "?pet <urn:owner> ?owner . " +
+                    "?pet <urn:isLiving> true . " +
+                "}");
+
+        // Create a Binding Set that contains the result of the WHERE clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("vehicle", vf.createLiteral("Alice's car"));
+        bs.addBinding("owner", vf.createURI("urn:Alice"));
+        bs.addBinding("plates", vf.createLiteral("XXXXXXX"));
+        bs.addBinding("pet", vf.createURI("urn:Kitty"));
+        final VisibilityBindingSet original = new VisibilityBindingSet(bs, "a|b");
+
+        // Run the projection evaluator.
+        final Set<VisibilityBindingSet> results = MultiProjectionEvaluator.make(multiProjection, new RandomUUIDFactory()).project(original);
+
+        // Figure out the blank nodes.
+        Value vehicalBNode = null;
+        Value petBNode = null;
+        for(final VisibilityBindingSet result : results) {
+            final Value object = result.getValue("object");
+            if(object.equals(vf.createURI("urn:vehicle"))) {
+                vehicalBNode = result.getValue("subject");
+            } else if(object.equals(vf.createURI("urn:pet"))) {
+                petBNode = result.getValue("subject");
+            }
+        }
+
+        // The expected binding sets.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        bs = new MapBindingSet();
+        bs.addBinding("subject", vehicalBNode);
+        bs.addBinding("predicate", RDF.TYPE);
+        bs.addBinding("object", vf.createURI("urn:vehicle"));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("subject", vehicalBNode);
+        bs.addBinding("predicate", vf.createURI("urn:tiresCount"));
+        bs.addBinding("object", vf.createLiteral("4", XMLSchema.INTEGER));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("subject", petBNode);
+        bs.addBinding("predicate", RDF.TYPE);
+        bs.addBinding("object", vf.createURI("urn:pet"));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("subject", petBNode);
+        bs.addBinding("predicate", vf.createURI("urn:isDead"));
+        bs.addBinding("object", vf.createLiteral(false));
+        expected.add( new VisibilityBindingSet(bs, "a|b") );
+
+        assertEquals(expected, results);
+    }
+
+    /**
+     * Get the first {@link MultiProjection} node from a SPARQL query.
+     *
+     * @param sparql - The query that contains a single Projection node.
+     * @return The first {@link MultiProjection} that is encountered.
+     * @throws Exception The query could not be parsed.
+     */
+    public static @Nullable MultiProjection getMultiProjection(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<MultiProjection> multiProjection = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final MultiProjection node) throws Exception {
+                multiProjection.set(node);
+            }
+        });
+
+        return multiProjection.get();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/common/rya.api.function/src/test/java/org/apache/rya/api/function/projection/ProjectionEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/test/java/org/apache/rya/api/function/projection/ProjectionEvaluatorTest.java b/common/rya.api.function/src/test/java/org/apache/rya/api/function/projection/ProjectionEvaluatorTest.java
new file mode 100644
index 0000000..e4a26a0
--- /dev/null
+++ b/common/rya.api.function/src/test/java/org/apache/rya/api/function/projection/ProjectionEvaluatorTest.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.projection;
+
+import static java.util.Objects.requireNonNull;
+import static org.junit.Assert.assertEquals;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.impl.MapBindingSet;
+import org.openrdf.query.parser.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
+
+/**
+ * Unit tests the methods of {@link ProjectionEvaluator}.
+ */
+public class ProjectionEvaluatorTest {
+
+    /**
+     * This Projection enumerates all of the variables that were in the query, none of them are anonymous, and
+     * none of them insert constants.
+     */
+    @Test
+    public void changesNothing() throws Exception {
+        // Read the projection object from a SPARQL query.
+        final Projection projection = getProjection(
+                "SELECT ?person ?employee ?business " +
+                "WHERE { " +
+                    "?person <urn:talksTo> ?employee . " +
+                    "?employee <urn:worksAt> ?business . " +
+                "}");
+
+        // Create a Binding Set that contains the result of the WHERE clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        final VisibilityBindingSet original = new VisibilityBindingSet(bs, "a|b");
+
+        // Execute the projection.
+        final VisibilityBindingSet result = ProjectionEvaluator.make(projection).project(original);
+        assertEquals(original, result);
+    }
+
+    /**
+     * This Projection replaces some of the variables names with different names.
+     */
+    @Test
+    public void renameBindings() throws Exception {
+        // Read the projection object from a SPARQL query.
+        final Projection projection = getProjection(
+                "SELECT (?person AS ?p) (?employee AS ?e) ?business " +
+                "WHERE { " +
+                    "?person <urn:talksTo> ?employee . " +
+                    "?employee <urn:worksAt> ?business . " +
+                "}");
+
+        // Create a Binding Set that contains the result of the WHERE clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        final VisibilityBindingSet original = new VisibilityBindingSet(bs, "a|b");
+
+        // The expected binding set changes the "person" binding name to "p" and "employee" to "e".
+        bs = new MapBindingSet();
+        bs.addBinding("p", vf.createURI("urn:Alice"));
+        bs.addBinding("e", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        final VisibilityBindingSet expected = new VisibilityBindingSet(bs, "a|b");
+
+        // Execute the projection.
+        final VisibilityBindingSet result = ProjectionEvaluator.make(projection).project(original);
+        assertEquals(expected, result);
+    }
+
+    /**
+     * This projection drops a binding from the original Binding Set.
+     */
+    @Test
+    public void dropsBinding() throws Exception {
+        // Read the projection object from a SPARQL query.
+        final Projection projection = getProjection(
+                "SELECT ?person " +
+                "WHERE { " +
+                    "?person <urn:talksTo> ?employee . " +
+                    "?employee <urn:worksAt> ?business . " +
+                "}");
+
+        // Create a Binding Set that contains the result of the WHERE clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        final VisibilityBindingSet original = new VisibilityBindingSet(bs, "a|b");
+
+        // The expected binding set only has the "person" binding.
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        final VisibilityBindingSet expected = new VisibilityBindingSet(bs, "a|b");
+
+        // Execute the projection.
+        final VisibilityBindingSet result = ProjectionEvaluator.make(projection).project(original);
+        assertEquals(expected, result);
+    }
+
+    /**
+     * This projection creates a Binding Set that represents a Statement and add a constant value to it.
+     */
+    @Test
+    public void addsConstantBinding() throws Exception {
+        // Read the projection object from a SPARQL query.
+        final Projection projection = getProjection(
+                "CONSTRUCT { ?person <urn:hasGrandchild> ?grandchild } " +
+                "WHERE {" +
+                    "?person <urn:hasChild> ?child ." +
+                    "?child <urn:hasChild> ?grandchild . " +
+                 "}");
+
+        // Create a Binding Set that contains the result of the WHERE clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("child", vf.createURI("urn:Bob"));
+        bs.addBinding("grandchild", vf.createURI("urn:Charlie"));
+        final VisibilityBindingSet original = new VisibilityBindingSet(bs, "a|b");
+
+        // The expected binding set represents a statement.
+        bs = new MapBindingSet();
+        bs.addBinding("subject", vf.createURI("urn:Alice"));
+        bs.addBinding("predicate", vf.createURI("urn:hasGrandchild"));
+        bs.addBinding("object", vf.createURI("urn:Charlie"));
+        final VisibilityBindingSet expected = new VisibilityBindingSet(bs, "a|b");
+
+        // Execute the projection.
+        final VisibilityBindingSet result = ProjectionEvaluator.make(projection).project(original);
+        assertEquals(expected, result);
+    }
+
+    /**
+     * This projection creates a Binding Set that represents a Statement that has a blank node added to it.
+     */
+    @Test
+    public void addsBlankNodeBinding() throws Exception {
+        // Read the projection object from a SPARQL query.
+        final Projection projection = getProjection(
+                "CONSTRUCT { ?person <urn:hasChild> _:b } " +
+                "WHERE {" +
+                    "?person <urn:hasGrandchild> ?grandchild ." +
+                 "}");
+
+        // Create a Binding Set that contains the result of the WHERE clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("hasGrandchild", vf.createURI("urn:Bob"));
+        final VisibilityBindingSet original = new VisibilityBindingSet(bs, "a|b");
+
+        // Execute the projection.
+        final VisibilityBindingSet result = ProjectionEvaluator.make(projection).project(original);
+
+        // The expected binding set represents a statement. We need to get the blank node's id from the
+        // result since that is different every time.
+        bs = new MapBindingSet();
+        bs.addBinding("subject", vf.createURI("urn:Alice"));
+        bs.addBinding("predicate", vf.createURI("urn:hasChild"));
+        bs.addBinding("object", result.getValue("object"));
+        final VisibilityBindingSet expected = new VisibilityBindingSet(bs, "a|b");
+
+        assertEquals(expected, result);
+    }
+
+    /**
+     * Get the first {@link Projection} node from a SPARQL query.
+     *
+     * @param sparql - The query that contains a single Projection node.
+     * @return The first {@link Projection} that is encountered.
+     * @throws Exception The query could not be parsed.
+     */
+    public static @Nullable Projection getProjection(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<Projection> projection = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final Projection node) throws Exception {
+                projection.set(node);
+            }
+        });
+
+        return projection.get();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/common/rya.api.model/src/main/java/org/apache/rya/api/model/VisibilityStatement.java
----------------------------------------------------------------------
diff --git a/common/rya.api.model/src/main/java/org/apache/rya/api/model/VisibilityStatement.java b/common/rya.api.model/src/main/java/org/apache/rya/api/model/VisibilityStatement.java
index 21e60d8..ffe95d7 100644
--- a/common/rya.api.model/src/main/java/org/apache/rya/api/model/VisibilityStatement.java
+++ b/common/rya.api.model/src/main/java/org/apache/rya/api/model/VisibilityStatement.java
@@ -84,4 +84,9 @@ public class VisibilityStatement extends StatementDecorator {
     public int hashCode() {
         return Objects.hash(visibility, super.getStatement());
     }
+
+    @Override
+    public String toString() {
+        return "Statement: " + super.getStatement().toString() + ", Visibility: " + visibility;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
index ac3d849..5f7a06b 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/ProcessorResult.java
@@ -21,6 +21,8 @@ package org.apache.rya.streams.kafka.processors;
 import static com.google.common.base.Preconditions.checkState;
 import static java.util.Objects.requireNonNull;
 
+import java.util.Objects;
+
 import org.apache.kafka.streams.processor.Processor;
 import org.apache.rya.api.model.VisibilityBindingSet;
 
@@ -92,6 +94,23 @@ public class ProcessorResult {
         return binary.get();
     }
 
+    @Override
+    public int hashCode() {
+        return Objects.hash(type, unary, binary);
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if(o instanceof ProcessorResult) {
+            final ProcessorResult other = (ProcessorResult) o;
+            return Objects.equals(type, other.type) &&
+                    Objects.equals(unary, other.unary) &&
+                    Objects.equals(binary, other.binary);
+        }
+        return false;
+    }
+
+
     /**
      * Creates a {@link ProcessorResult} using the supplied value.
      *
@@ -151,6 +170,20 @@ public class ProcessorResult {
         public VisibilityBindingSet getResult() {
             return result;
         }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(result);
+        }
+
+        @Override
+        public boolean equals(final Object o) {
+            if(o instanceof UnaryResult) {
+                final UnaryResult other = (UnaryResult) o;
+                return Objects.equals(result, other.result);
+            }
+            return false;
+        }
     }
 
     /**
@@ -186,6 +219,21 @@ public class ProcessorResult {
             return result;
         }
 
+        @Override
+        public int hashCode() {
+            return Objects.hash(side, result);
+        }
+
+        @Override
+        public boolean equals(final Object o) {
+            if(o instanceof BinaryResult) {
+                final BinaryResult other = (BinaryResult) o;
+                return Objects.equals(side, other.side) &&
+                        Objects.equals(result, other.result);
+            }
+            return false;
+        }
+
         /**
          * A label that is used to by the downstream binary prcoessor to distinguish which upstream processor
          * produced the {@link BinaryResult}.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
index bc99a7b..f7c2e5e 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorSupplier.java
@@ -97,6 +97,8 @@ public class StatementPatternProcessorSupplier implements ProcessorSupplier<Stri
 
         @Override
         public void process(final String key, final VisibilityStatement statement) {
+            log.debug("\nINPUT:\n{}\n", statement);
+
             // Check to see if the Statement matches the Statement Pattern.
             final Optional<BindingSet> bs = spMatcher.match(statement);
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorSupplier.java
new file mode 100644
index 0000000..68aec0a
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorSupplier.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.projection;
+
+import static java.util.Objects.requireNonNull;
+
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.function.projection.MultiProjectionEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.ResultType;
+import org.apache.rya.streams.kafka.processors.ProcessorResultFactory;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessor;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessorSupplier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Supplies {@link MultiProjectionProcessor} instances.
+ */
+@DefaultAnnotation(NonNull.class)
+public class MultiProjectionProcessorSupplier extends RyaStreamsProcessorSupplier {
+
+    private final MultiProjectionEvaluator multiProjection;
+
+    /**
+     * Constructs an instance of {@link MultiProjectionProcessorSupplier}.
+     *
+     * @param multiProjection - Defines the MultiProjection work that will be performed by supplied processors. (not null)
+     * @param resultFactory - The factory that the supplied processors will use to create results. (not null)
+     */
+    public MultiProjectionProcessorSupplier(
+            final MultiProjectionEvaluator multiProjection,
+            final ProcessorResultFactory resultFactory) {
+        super(resultFactory);
+        this.multiProjection = requireNonNull(multiProjection);
+    }
+
+    @Override
+    public Processor<Object, ProcessorResult> get() {
+        return new MultiProjectionProcessor(multiProjection, super.getResultFactory());
+    }
+
+    /**
+     * Evaluates {@link ProcessorResult}s against a {@link MultiProjectionEvaluator} and forwards its results
+     * to a downstream processor.
+     */
+    @DefaultAnnotation(NonNull.class)
+    public static final class MultiProjectionProcessor extends RyaStreamsProcessor {
+        private static final Logger log = LoggerFactory.getLogger(MultiProjectionProcessor.class);
+
+        private final MultiProjectionEvaluator multiProjection;
+
+        private ProcessorContext context;
+
+        /**
+         * Constructs an instance of {@link MultiProjectionProcessor}.
+         *
+         * @param multiProjection - Defines the MultiProjection work that will be performed by this processor. (not null)
+         * @param resultFactory - The factory that the supplied processors will use to create results. (not null)
+         */
+        public MultiProjectionProcessor(
+                final MultiProjectionEvaluator multiProjection,
+                final ProcessorResultFactory resultFactory) {
+            super(resultFactory);
+            this.multiProjection = requireNonNull(multiProjection);
+        }
+
+        @Override
+        public void init(final ProcessorContext context) {
+            this.context = context;
+        }
+
+        @Override
+        public void process(final Object key, final ProcessorResult value) {
+            // projections can only be unary
+            if (value.getType() != ResultType.UNARY) {
+                throw new RuntimeException("The ProcessorResult to be processed must be Unary.");
+            }
+
+            // Apply the projection to the binding set and forward the results.
+            final VisibilityBindingSet bs = value.getUnary().getResult();
+            for(final VisibilityBindingSet newVisBs : multiProjection.project(bs)) {
+                log.debug("\nOUTPUT:\n{}", newVisBs);
+                context.forward(key, super.getResultFactory().make(newVisBs));
+            }
+        }
+
+        @Override
+        public void punctuate(final long timestamp) {
+            // Do nothing.
+        }
+
+        @Override
+        public void close() {
+            // Do nothing.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorSupplier.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorSupplier.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorSupplier.java
index 67a777f..bc46d4f 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorSupplier.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorSupplier.java
@@ -22,16 +22,15 @@ import static java.util.Objects.requireNonNull;
 
 import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.function.projection.ProjectionEvaluator;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.streams.kafka.processors.ProcessorResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResult.ResultType;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
 import org.apache.rya.streams.kafka.processors.ProcessorResultFactory;
+import org.apache.rya.streams.kafka.processors.RyaStreamsProcessor;
 import org.apache.rya.streams.kafka.processors.RyaStreamsProcessorSupplier;
-import org.openrdf.query.algebra.Projection;
-import org.openrdf.query.algebra.ProjectionElem;
-import org.openrdf.query.algebra.ProjectionElemList;
-import org.openrdf.query.impl.MapBindingSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -42,49 +41,47 @@ import edu.umd.cs.findbugs.annotations.NonNull;
 @DefaultAnnotation(NonNull.class)
 public class ProjectionProcessorSupplier extends RyaStreamsProcessorSupplier {
 
-    private final ProjectionElemList projectionElems;
+    private final ProjectionEvaluator projection;
 
     /**
      * Constructs an instance of {@link ProjectionProcessorSupplier}.
      *
-     * @param projectionElems - The {@link ProjectionElemList} that defines which bindings get forwarded or changed. (not null)
+     * @param projection - Defines the projection work that will be performed by supplied processors. (not null)
      * @param resultFactory - The factory that the supplied processors will use to create results. (not null)
      */
     public ProjectionProcessorSupplier(
-            final ProjectionElemList projectionElems,
+            final ProjectionEvaluator projection,
             final ProcessorResultFactory resultFactory) {
         super(resultFactory);
-        this.projectionElems = requireNonNull(projectionElems);
+        this.projection = requireNonNull(projection);
     }
 
     @Override
     public Processor<Object, ProcessorResult> get() {
-        return new ProjectionProcessor(projectionElems, super.getResultFactory());
+        return new ProjectionProcessor(projection, super.getResultFactory());
     }
 
     /**
-     * Evaluates {@link ProcessorResult}s against a {@link Projection}.  Any results found in
-     * the {@link ProjectionElemList} will be modified and/or forwarded.  A {@link ProjectionElemList} defines
-     * a source and target name for a binding, so if a binding name appears in the source list of the {@link ProjectionElemList},
-     * then it will be renamed with the associated target name.
+     * Evaluates {@link ProcessorResult}s against a {@link ProjectionEvaluator} and forwards its result
+     * to a downstream processor.
      */
     @DefaultAnnotation(NonNull.class)
-    public static final class ProjectionProcessor implements Processor<Object, ProcessorResult> {
+    public static final class ProjectionProcessor extends RyaStreamsProcessor {
+        private static final Logger log = LoggerFactory.getLogger(ProjectionProcessor.class);
 
-        private final ProjectionElemList projectionElems;
-        private final ProcessorResultFactory resultFactory;
+        private final ProjectionEvaluator projection;
 
         private ProcessorContext context;
 
         /**
          * Constructs an instance of {@link ProjectionProcessor}.
          *
-         * @param projectionElems - The projection elems that will determine what to do with the bindings. (not null)
+         * @param projection - Defines the projection work that will be performed by this processor. (not null)
          * @param resultFactory - The factory that the processor will use to create results. (not null)
          */
-        public ProjectionProcessor(final ProjectionElemList projectionElems, final ProcessorResultFactory resultFactory) {
-            this.projectionElems = requireNonNull(projectionElems);
-            this.resultFactory = requireNonNull(resultFactory);
+        public ProjectionProcessor(final ProjectionEvaluator projection, final ProcessorResultFactory resultFactory) {
+            super(resultFactory);
+            this.projection = requireNonNull(projection);
         }
 
         @Override
@@ -99,20 +96,13 @@ public class ProjectionProcessorSupplier extends RyaStreamsProcessorSupplier {
                 throw new RuntimeException("The ProcessorResult to be processed must be Unary.");
             }
 
-            final UnaryResult unary = result.getUnary();
-            final VisibilityBindingSet bindingSet = unary.getResult();
+            // Apply the projection to the binding set.
+            final VisibilityBindingSet bs = result.getUnary().getResult();
+            final VisibilityBindingSet newVisBs = projection.project(bs);
 
-            final MapBindingSet newBindingSet = new MapBindingSet(bindingSet.size());
-            for (final ProjectionElem elem : projectionElems.getElements()) {
-                if (bindingSet.hasBinding(elem.getSourceName())) {
-                    newBindingSet.addBinding(elem.getTargetName(), bindingSet.getValue(elem.getSourceName()));
-                }
-            }
-
-            // wrap the new binding set with the original's visibility.
-            final VisibilityBindingSet newVisiSet = new VisibilityBindingSet(newBindingSet, bindingSet.getVisibility());
-            final ProcessorResult resultValue = resultFactory.make(newVisiSet);
-            context.forward(key, resultValue);
+            // Forward the result to the downstream processor.
+            log.debug("\nOUTPUT:\n{}", newVisBs);
+            context.forward(key, super.getResultFactory().make(newVisBs));
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
index 109e40d..190bad3 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RdfTestUtil.java
@@ -22,6 +22,8 @@ import static java.util.Objects.requireNonNull;
 
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.openrdf.query.algebra.MultiProjection;
+import org.openrdf.query.algebra.Projection;
 import org.openrdf.query.algebra.StatementPattern;
 import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
 import org.openrdf.query.parser.ParsedQuery;
@@ -59,4 +61,48 @@ public final class RdfTestUtil {
         });
         return statementPattern.get();
     }
+
+    /**
+     * Get the first {@link Projection} node from a SPARQL query.
+     *
+     * @param sparql - The query that contains a single Projection node.
+     * @return The first {@link Projection} that is encountered.
+     * @throws Exception The query could not be parsed.
+     */
+    public static @Nullable Projection getProjection(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<Projection> projection = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final Projection node) throws Exception {
+                projection.set(node);
+            }
+        });
+
+        return projection.get();
+    }
+
+    /**
+     * Get the first {@link MultiProjection} node from a SPARQL query.
+     *
+     * @param sparql - The query that contains a single Projection node.
+     * @return The first {@link MultiProjection} that is encountered.
+     * @throws Exception The query could not be parsed.
+     */
+    public static @Nullable MultiProjection getMultiProjection(final String sparql) throws Exception {
+        requireNonNull(sparql);
+
+        final AtomicReference<MultiProjection> multiProjection = new AtomicReference<>();
+        final ParsedQuery parsed = new SPARQLParser().parseQuery(sparql, null);
+        parsed.getTupleExpr().visit(new QueryModelVisitorBase<Exception>() {
+            @Override
+            public void meet(final MultiProjection node) throws Exception {
+                multiProjection.set(node);
+            }
+        });
+
+        return multiProjection.get();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
deleted file mode 100644
index f58387e..0000000
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/ProjectionProcessorIT.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.streams.kafka.processors;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.apache.kafka.streams.processor.TopologyBuilder;
-import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.api.model.VisibilityStatement;
-import org.apache.rya.streams.kafka.KafkaTestUtil;
-import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RdfTestUtil;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
-import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
-import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
-import org.apache.rya.streams.kafka.processors.projection.ProjectionProcessorSupplier;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
-import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
-import org.apache.rya.test.kafka.KafkaTestInstanceRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.algebra.ProjectionElem;
-import org.openrdf.query.algebra.ProjectionElemList;
-import org.openrdf.query.algebra.StatementPattern;
-import org.openrdf.query.algebra.evaluation.QueryBindingSet;
-
-import com.google.common.collect.Sets;
-
-/**
- * Integration tests the methods of {@link StatementPatternProcessor}.
- */
-public class ProjectionProcessorIT {
-
-    @Rule
-    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
-
-    @Test
-    public void projection_renameOne() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Get the StatementPattern object that will be evaluated.
-        final StatementPattern sp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
-
-        // Add a processor that handles the projection.
-        final ProjectionElemList elems = new ProjectionElemList();
-        elems.addElement(new ProjectionElem("otherPerson", "dog"));
-        elems.addElement(new ProjectionElem("person", "person"));
-        builder.addProcessor("P1", new ProjectionProcessorSupplier(elems, result -> ProcessorResult.make(new UnaryResult(result))), "SP1");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "P1");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
-
-        // Load some data into the input topic.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Sparky")), "a") );
-
-        // Show the correct binding set results from the job.
-        final QueryBindingSet bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("dog", vf.createURI("urn:Sparky"));
-        final VisibilityBindingSet binding = new VisibilityBindingSet(bs, "a");
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        expected.add(binding);
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected));
-    }
-
-    @Test
-    public void projection_keepOneDropOne() throws Exception {
-        // Enumerate some topics that will be re-used
-        final String ryaInstance = UUID.randomUUID().toString();
-        final UUID queryId = UUID.randomUUID();
-        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
-        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
-
-        // Get the StatementPattern object that will be evaluated.
-        final StatementPattern sp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
-
-        // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
-
-        // Add a processor that handles the projection.
-        final ProjectionElemList elems = new ProjectionElemList();
-        elems.addElement(new ProjectionElem("otherPerson", "otherPerson"));
-        builder.addProcessor("P1", new ProjectionProcessorSupplier(elems, result -> ProcessorResult.make(new UnaryResult(result))), "SP1");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "P1");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
-
-        // Load some data into the input topic.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
-
-        // Show the correct binding set results from the job.
-        final QueryBindingSet bs = new QueryBindingSet();
-        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
-        final VisibilityBindingSet binding = new VisibilityBindingSet(bs, "a");
-        final Set<VisibilityBindingSet> expected = new HashSet<>();
-        expected.add(binding);
-        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
new file mode 100644
index 0000000..ee0e55b
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.projection;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.processor.StateStoreSupplier;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.rya.api.function.join.NaturalJoin;
+import org.apache.rya.api.function.projection.MultiProjectionEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RdfTestUtil;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
+import org.apache.rya.streams.kafka.processors.projection.MultiProjectionProcessorSupplier.MultiProjectionProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerde;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.BNode;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.query.algebra.MultiProjection;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Integration tests the methods of {@link MultiProjectionProcessor}.
+ */
+public class MultiProjectionProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void showProcessorWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final StatementPattern sp1 = RdfTestUtil.getSp("SELECT * WHERE { ?thing <urn:corner> ?location . }");
+        final StatementPattern sp2 = RdfTestUtil.getSp("SELECT * WHERE { ?thing <urn:compass> ?direction . }");
+        final MultiProjection multiProjection = RdfTestUtil.getMultiProjection(
+                "CONSTRUCT {" +
+                    "_:b a <urn:movementObservation> ; " +
+                    "<urn:location> ?location ; " +
+                    "<urn:direction> ?direction ; " +
+                "}" +
+                "WHERE {" +
+                    "?thing <urn:corner> ?location ." +
+                    "?thing <urn:compass> ?direction." +
+                "}");
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyBuilder();
+        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
+        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp1,
+                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
+        builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2,
+                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
+
+        builder.addProcessor("NATURAL_JOIN", new JoinProcessorSupplier(
+                "NATURAL_JOIN",
+                new NaturalJoin(),
+                Lists.newArrayList("thing"),
+                Lists.newArrayList("thing", "location", "direction"),
+                result -> ProcessorResult.make( new UnaryResult(result) )), "SP1", "SP2");
+
+        final StateStoreSupplier joinStoreSupplier =
+                Stores.create( "NATURAL_JOIN" )
+                  .withStringKeys()
+                  .withValues(new VisibilityBindingSetSerde())
+                  .inMemory()
+                  .build();
+        builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
+
+        final String blankNodeId = UUID.randomUUID().toString();
+        builder.addProcessor("MULTIPROJECTION", new MultiProjectionProcessorSupplier(
+                MultiProjectionEvaluator.make(multiProjection, () -> blankNodeId),
+                result -> ProcessorResult.make(new UnaryResult(result))), "NATURAL_JOIN");
+
+        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "MULTIPROJECTION");
+        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:car1"), vf.createURI("urn:compass"), vf.createURI("urn:NW")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:car1"), vf.createURI("urn:corner"), vf.createURI("urn:corner1")), "a") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final BNode blankNode = vf.createBNode(blankNodeId);
+
+        MapBindingSet expectedBs = new MapBindingSet();
+        expectedBs.addBinding("subject", blankNode);
+        expectedBs.addBinding("predicate", RDF.TYPE);
+        expectedBs.addBinding("object", vf.createURI("urn:movementObservation"));
+
+        expectedBs = new MapBindingSet();
+        expectedBs.addBinding("subject", blankNode);
+        expectedBs.addBinding("predicate", vf.createURI("urn:direction"));
+        expectedBs.addBinding("object", vf.createURI("urn:NW"));
+
+
+        expectedBs = new MapBindingSet();
+        expectedBs.addBinding("subject", blankNode);
+        expectedBs.addBinding("predicate", vf.createURI("urn:location"));
+        expectedBs.addBinding("object", vf.createURI("urn:corner1"));
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/538393fe/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorTest.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorTest.java
new file mode 100644
index 0000000..d25db23
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorTest.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.projection;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.function.projection.MultiProjectionEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.RdfTestUtil;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.ResultType;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.projection.MultiProjectionProcessorSupplier.MultiProjectionProcessor;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.openrdf.model.BNode;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.query.algebra.MultiProjection;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Unit test the methods of {@link MultiProjectionProcessor}.
+ */
+public class MultiProjectionProcessorTest {
+
+    @Test
+    public void showProjectionFunctionIsCalled() throws Exception {
+        // The SPARQL that will define the projection.
+        final MultiProjection multiProjection = RdfTestUtil.getMultiProjection(
+                "CONSTRUCT {" +
+                    "_:b a <urn:movementObservation> ; " +
+                    "<urn:location> ?location ; " +
+                    "<urn:direction> ?direction ; " +
+                "}" +
+                "WHERE {" +
+                    "?thing <urn:corner> ?location ." +
+                    "?thing <urn:compass> ?direction." +
+                "}");
+
+        // Create a Binding Set that contains the result of the WHERE clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet inputBs = new MapBindingSet();
+        inputBs.addBinding("location", vf.createURI("urn:corner1"));
+        inputBs.addBinding("direction", vf.createURI("urn:NW"));
+        final VisibilityBindingSet inputVisBs = new VisibilityBindingSet(inputBs, "a|b");
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final String blankNodeId = UUID.randomUUID().toString();
+        final BNode blankNode = vf.createBNode(blankNodeId);
+
+        MapBindingSet expectedBs = new MapBindingSet();
+        expectedBs.addBinding("subject", blankNode);
+        expectedBs.addBinding("predicate", RDF.TYPE);
+        expectedBs.addBinding("object", vf.createURI("urn:movementObservation"));
+        expected.add(new VisibilityBindingSet(expectedBs, "a|b"));
+
+        expectedBs = new MapBindingSet();
+        expectedBs.addBinding("subject", blankNode);
+        expectedBs.addBinding("predicate", vf.createURI("urn:direction"));
+        expectedBs.addBinding("object", vf.createURI("urn:NW"));
+        expected.add(new VisibilityBindingSet(expectedBs, "a|b"));
+
+        expectedBs = new MapBindingSet();
+        expectedBs.addBinding("subject", blankNode);
+        expectedBs.addBinding("predicate", vf.createURI("urn:location"));
+        expectedBs.addBinding("object", vf.createURI("urn:corner1"));
+        expected.add(new VisibilityBindingSet(expectedBs, "a|b"));
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final MultiProjectionProcessor processor = new MultiProjectionProcessor(
+                MultiProjectionEvaluator.make(multiProjection, () -> blankNodeId),
+                result -> ProcessorResult.make(new UnaryResult(result)));
+        processor.init(context);
+        processor.process("key", ProcessorResult.make(new UnaryResult(inputVisBs)));
+
+        final ArgumentCaptor<ProcessorResult> results = ArgumentCaptor.forClass(ProcessorResult.class);
+        verify(context, times(3)).forward(any(), results.capture());
+
+        final Set<VisibilityBindingSet> resultBindingSets = results.getAllValues().stream()
+                .map(result -> {
+                    return (result.getType() == ResultType.UNARY) ? result.getUnary().getResult() : result.getBinary().getResult();
+                })
+                .collect(Collectors.toSet());
+
+        assertEquals(expected, resultBindingSets);
+    }
+}
\ No newline at end of file


[12/50] [abbrv] incubator-rya git commit: RYA-377 TopologyBuilder

Posted by ca...@apache.org.
RYA-377 TopologyBuilder

A factory for turning TupleExpr parsed from
SPARQL into TopologyBuilder objects used
by Rya Streams


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

Branch: refs/heads/master
Commit: 83d09f42c98f16150aab69c42465157b55c28a14
Parents: b8b0a12
Author: Andrew Smith <sm...@gmail.com>
Authored: Tue Nov 14 16:28:43 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../streams/kafka/topology/TopologyFactory.java | 414 +++++++++++++++++++
 .../processors/StatementPatternProcessorIT.java | 124 ++----
 .../kafka/processors/join/JoinProcessorIT.java  |  86 ++--
 .../kafka/topology/TopologyFactoryTest.java     | 106 +++++
 4 files changed, 573 insertions(+), 157 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/83d09f42/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
new file mode 100644
index 0000000..782a58b
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/topology/TopologyFactory.java
@@ -0,0 +1,414 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.topology;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorSupplier;
+import org.apache.kafka.streams.processor.StateStoreSupplier;
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.kafka.streams.state.Stores;
+import org.apache.rya.api.function.join.IterativeJoin;
+import org.apache.rya.api.function.join.LeftOuterJoin;
+import org.apache.rya.api.function.join.NaturalJoin;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.BinaryResult.Side;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier;
+import org.apache.rya.streams.kafka.processors.projection.ProjectionProcessorSupplier;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerde;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.algebra.BinaryTupleOperator;
+import org.openrdf.query.algebra.Extension;
+import org.openrdf.query.algebra.Join;
+import org.openrdf.query.algebra.LeftJoin;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.algebra.QueryModelNode;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.parser.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Factory for building {@link TopologyBuilder}s from a SPARQL query.
+ */
+@DefaultAnnotation(NonNull.class)
+public class TopologyFactory implements TopologyBuilderFactory {
+    private static final String SOURCE = "SOURCE";
+    private static final String STATEMENT_PATTERN_PREFIX = "SP_";
+    private static final String JOIN_PREFIX = "JOIN_";
+    private static final String PROJECTION_PREFIX = "PROJECTION_";
+    private static final String SINK = "SINK";
+
+    private List<ProcessorEntry> processorEntryList;
+
+    /**
+     * Builds a {@link TopologyBuilder} based on the provided sparql query.
+     *
+     * @param sparqlQuery - The SPARQL query to build a topology for. (not null)
+     * @param statementTopic - The topic for the source to read from. (not null)
+     * @param resultTopic - The topic for the sink to write to. (not null)
+     * @return - The created {@link TopologyBuilder}.
+     * @throws MalformedQueryException - The provided query is not a valid SPARQL query.
+     * @throws TopologyBuilderException - A problem occurred while constructing the topology.
+     */
+    @Override
+    public TopologyBuilder build(final String sparqlQuery, final String statementTopic, final String resultTopic)
+            throws MalformedQueryException, TopologyBuilderException {
+        requireNonNull(sparqlQuery);
+        requireNonNull(statementTopic);
+        requireNonNull(resultTopic);
+
+        final ParsedQuery parsedQuery = new SPARQLParser().parseQuery(sparqlQuery, null);
+        final TopologyBuilder builder = new TopologyBuilder();
+
+        final TupleExpr expr = parsedQuery.getTupleExpr();
+        final QueryVisitor visitor = new QueryVisitor();
+        expr.visit(visitor);
+
+        processorEntryList = visitor.getProcessorEntryList();
+        final Map<TupleExpr, String> idMap = visitor.getIDs();
+        // add source node
+        builder.addSource(SOURCE, new StringDeserializer(), new VisibilityStatementDeserializer(), statementTopic);
+
+        // processing the processor entry list in reverse order means we go from leaf
+        // nodes -> parent nodes.
+        // So, when the parent processing nodes get added, the upstream
+        // processing node will already exist.
+
+        ProcessorEntry entry = null;
+        for (int ii = processorEntryList.size() - 1; ii >= 0; ii--) {
+            entry = processorEntryList.get(ii);
+            //statement patterns need to be connected to the Source.
+            if(entry.getNode() instanceof StatementPattern) {
+                builder.addProcessor(entry.getID(), entry.getSupplier(), SOURCE);
+            } else {
+                final List<TupleExpr> parents = entry.getUpstreamNodes();
+                final String[] parentIDs = new String[parents.size()];
+                for (int id = 0; id < parents.size(); id++) {
+                    parentIDs[id] = idMap.get(parents.get(id));
+                }
+                builder.addProcessor(entry.getID(), entry.getSupplier(), parentIDs);
+            }
+
+            if (entry.getNode() instanceof Join || entry.getNode() instanceof LeftJoin) {
+                // Add a state store for the join processor.
+                final StateStoreSupplier joinStoreSupplier =
+                        Stores.create( entry.getID() )
+                        .withStringKeys()
+                        .withValues(new VisibilityBindingSetSerde())
+                        .persistent()
+                        .build();
+                builder.addStateStore(joinStoreSupplier, entry.getID());
+            }
+        }
+
+        // convert processing results to visibility binding sets
+        builder.addProcessor("OUTPUT_FORMATTER", new BindingSetOutputFormatterSupplier(), entry.getID());
+
+        // add sink
+        builder.addSink(SINK, resultTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "OUTPUT_FORMATTER");
+
+        return builder;
+    }
+
+    @VisibleForTesting
+    public List<ProcessorEntry> getProcessorEntry() {
+        return processorEntryList;
+    }
+
+    /**
+     * An entry to be added as a Processing node in kafka streams'
+     * TopologyBuilder.
+     */
+    final static class ProcessorEntry {
+        private final TupleExpr node;
+        private final String id;
+        private final Optional<Side> downstreamSide;
+        private final ProcessorSupplier<?, ?> supplier;
+        private final List<TupleExpr> upstreamNodes;
+
+        /**
+         * Creates a new {@link ProcessorEntry}.
+         *
+         * @param node - The RDF node to be added as a processor. (not null)
+         * @param id - The id for the {@link TupleExpr} node. (not null)
+         * @param downstreamSide - Which side the current node is on from its downstream processor. (not null)
+         * @param supplier - Supplies the {@link Processor} for this node. (not null)
+         * @param upstreamNodes - The RDF nodes that will become upstream processing nodes. (not null)
+         */
+        public ProcessorEntry(final TupleExpr node, final String id, final Optional<Side> downstreamSide, final ProcessorSupplier<?, ?> supplier, final List<TupleExpr> upstreamNodes) {
+            this.node = requireNonNull(node);
+            this.id = requireNonNull(id);
+            this.downstreamSide = requireNonNull(downstreamSide);
+            this.supplier = requireNonNull(supplier);
+            this.upstreamNodes = requireNonNull(upstreamNodes);
+        }
+
+        /**
+         * @return - The RDF node to added as a processor.
+         */
+        public TupleExpr getNode() {
+            return node;
+        }
+
+        /**
+         * @return - The side the node is on from its downstream processor.
+         */
+        public Optional<Side> getDownstreamSide() {
+            return downstreamSide;
+        }
+
+        /**
+         * @return - The upstream parents to this node. These parent nodes must
+         *         result in a {@link ProcessorEntry}
+         */
+        public List<TupleExpr> getUpstreamNodes() {
+            return upstreamNodes;
+        }
+
+        /**
+         * @return - The processor id of the node.
+         */
+        public String getID() {
+            return id;
+        }
+
+        /**
+         * @return - The {@link ProcessorSupplier} used to supply the
+         *         {@link Processor} for this node.
+         */
+        public ProcessorSupplier<?, ?> getSupplier() {
+            return supplier;
+        }
+
+        @Override
+        public boolean equals(final Object other) {
+            if (!(other instanceof ProcessorEntry)) {
+                return false;
+            }
+            final ProcessorEntry o = (ProcessorEntry) other;
+            return Objects.equals(node, o.node) &&
+                    Objects.equals(id, o.id) &&
+                    Objects.equals(downstreamSide, o.downstreamSide) &&
+                    Objects.equals(supplier, o.supplier) &&
+                    Objects.equals(upstreamNodes, o.upstreamNodes);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(node, downstreamSide, upstreamNodes, id, supplier);
+        }
+
+        @Override
+        public String toString() {
+            final StringBuilder sb = new StringBuilder();
+            sb.append("ID: " + id + "\n");
+            if (downstreamSide.isPresent()) {
+                sb.append("***********************************\n");
+                sb.append("SIDE: " + downstreamSide.get() + "\n");
+            }
+            sb.append("***********************************\n");
+            sb.append("PARENTS: ");
+            for (final TupleExpr expr : upstreamNodes) {
+                sb.append(expr.toString() + ",");
+            }
+            sb.append("\n***********************************\n");
+            sb.append("NODE: " + node.toString());
+            sb.append("\n");
+            return sb.toString();
+        }
+    }
+
+    /**
+     * Visits each node in a {@link TupleExpr} and creates a
+     * {@link ProcessorSupplier} and meta information needed for creating a
+     * {@link TopologyBuilder}.
+     */
+    final static class QueryVisitor extends QueryModelVisitorBase<TopologyBuilderException> {
+        // Each node needs a ProcessorEntry to be a processor node in the
+        // TopologyBuilder.
+        private final List<ProcessorEntry> entries = new ArrayList<>();
+        private final Map<TupleExpr, String> idMap = new HashMap<>();
+
+        /**
+         * @return The {@link ProcessorEntry}s used to create a Topology.
+         */
+        public List<ProcessorEntry> getProcessorEntryList() {
+            return entries;
+        }
+
+        /**
+         * @return The IDs created for each {@link TupleExpr} node in the query that resulted in a {@link ProcessorEntry}.
+         */
+        public Map<TupleExpr, String> getIDs() {
+            return idMap;
+        }
+
+        @Override
+        public void meet(final StatementPattern node) throws TopologyBuilderException {
+            // topology parent for Statement Patterns will always be a source
+            final String id = STATEMENT_PATTERN_PREFIX + UUID.randomUUID();
+            final Optional<Side> side = getSide(node);
+            final StatementPatternProcessorSupplier supplier = new StatementPatternProcessorSupplier(node, result -> getResult(side, result));
+            entries.add(new ProcessorEntry(node, id, side, supplier, Lists.newArrayList()));
+            idMap.put(node, id);
+            super.meet(node);
+        }
+
+        @Override
+        public void meet(final Projection node) throws TopologyBuilderException {
+            final String id = PROJECTION_PREFIX + UUID.randomUUID();
+            final Optional<Side> side = getSide(node);
+            TupleExpr arg = node.getArg();
+            // If the arg is an Extension, there are rebindings that need to be
+            // ignored since they do not have a processor node.
+            if (arg instanceof Extension) {
+                arg = ((Extension) arg).getArg();
+            }
+            final ProjectionProcessorSupplier supplier = new ProjectionProcessorSupplier(node.getProjectionElemList(), result -> getResult(side, result));
+            entries.add(new ProcessorEntry(node, id, side, supplier, Lists.newArrayList(arg)));
+            idMap.put(node, id);
+            super.meet(node);
+        }
+
+        @Override
+        public void meet(final Join node) throws TopologyBuilderException {
+            final String id = JOIN_PREFIX + UUID.randomUUID();
+            meetJoin(id, new NaturalJoin(), node);
+            super.meet(node);
+        }
+
+        @Override
+        public void meet(final LeftJoin node) throws TopologyBuilderException {
+            final String id = JOIN_PREFIX + UUID.randomUUID();
+            meetJoin(id, new LeftOuterJoin(), node);
+            super.meet(node);
+        }
+
+        /**
+         * Gets the {@link Side} the current node in the visitor is on relative to the provided node.
+         * @param node - The node used to determine the side of the current visitor node.
+         * @return The {@link Side} the current node is on.
+         */
+        private Optional<Side> getSide(final QueryModelNode node) {
+            // if query parent is a binary operator, need to determine if its left or right.
+            if (node.getParentNode() instanceof BinaryTupleOperator) {
+                final BinaryTupleOperator binary = (BinaryTupleOperator) node.getParentNode();
+                if (node.equals(binary.getLeftArg())) {
+                    return Optional.of(Side.LEFT);
+                } else {
+                    return Optional.of(Side.RIGHT);
+                }
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        /**
+         * Creates a join entry based on a provided {@link IterativeJoin} and the Join's
+         * {@link BinaryTupleOperator}.
+         *
+         * @param id - The ID of the join.
+         * @param joinFunction - The {@link IterativeJoin} function to perform during processing.
+         * @param node - The {@link BinaryTupleOperator} used to create the process.
+         */
+        private void meetJoin(final String id, final IterativeJoin joinFunction, final BinaryTupleOperator node) {
+            final Set<String> leftArgs = node.getLeftArg().getBindingNames();
+            final Set<String> rightArgs = node.getRightArg().getBindingNames();
+            final List<String> joinVars = Lists.newArrayList(Sets.intersection(leftArgs, rightArgs));
+
+            leftArgs.removeAll(joinVars);
+            rightArgs.removeAll(joinVars);
+
+            final List<String> otherVars = new ArrayList<>();
+            otherVars.addAll(leftArgs);
+            otherVars.addAll(rightArgs);
+
+            // the join variables need to be sorted so that when compared to all
+            // the variables, the start of the all variable list is congruent to
+            // the join var list.
+            joinVars.sort(Comparator.naturalOrder());
+            otherVars.sort(Comparator.naturalOrder());
+
+            final List<String> allVars = new ArrayList<>();
+            allVars.addAll(joinVars);
+            allVars.addAll(otherVars);
+
+            final Optional<Side> side = getSide(node);
+            final JoinProcessorSupplier supplier = new JoinProcessorSupplier(id, joinFunction, joinVars, allVars, result -> getResult(side, result));
+            entries.add(new ProcessorEntry(node, id, side, supplier, Lists.newArrayList(node.getLeftArg(), node.getRightArg())));
+            idMap.put(node, id);
+        }
+
+        /**
+         * Creates a {@link ProcessorResult} based on a side and result.
+         *
+         * @param side - If one is present, a {@link BinaryResult} is created.
+         * @param result - The result to wrap in a {@link ProcessorResult}.
+         * @return The {@link ProcessorResult} used by the {@link Processor}.
+         */
+        private ProcessorResult getResult(final Optional<Side> side, final VisibilityBindingSet result) {
+            if (side.isPresent()) {
+                return ProcessorResult.make(new BinaryResult(side.get(), result));
+            } else {
+                return ProcessorResult.make(new UnaryResult(result));
+            }
+        }
+    }
+
+    /**
+     * An Exception thrown when a problem occurs when constructing the processor
+     * topology in the {@link TopologyFactory}.
+     */
+    public class TopologyBuilderException extends Exception {
+        private static final long serialVersionUID = 1L;
+
+        public TopologyBuilderException(final String message, final Throwable cause) {
+            super(message, cause);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/83d09f42/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
index 0b2ff60..e55ec2e 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
@@ -24,25 +24,18 @@ import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.api.model.VisibilityStatement;
 import org.apache.rya.streams.kafka.KafkaTestUtil;
 import org.apache.rya.streams.kafka.KafkaTopics;
-import org.apache.rya.streams.kafka.RdfTestUtil;
-import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
 import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
-import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterSupplier.BindingSetOutputFormatter;
-import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
-import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.query.algebra.StatementPattern;
 import org.openrdf.query.algebra.evaluation.QueryBindingSet;
 
 /**
@@ -61,23 +54,10 @@ public class StatementPatternProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the StatementPattern object that will be evaluated.
-        final StatementPattern sp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
-
         // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "SP1");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String query = "SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
 
         // Create a statement that generate an SP result.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -104,23 +84,10 @@ public class StatementPatternProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the StatementPattern object that will be evaluated.
-        final StatementPattern sp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
-
         // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "SP1");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String query = "SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
 
         // Create some statements where some generates SP results and others do not.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -155,27 +122,13 @@ public class StatementPatternProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the StatementPattern object that will be evaluated.
-        final StatementPattern sp1 = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
-        final StatementPattern sp2 = RdfTestUtil.getSp("SELECT * WHERE { ?person ?action <urn:Bob> }");
-
         // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp1, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
-
-        // Add a processor that handles the second statement pattern.
-        builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "SP1", "SP2");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String query = "SELECT * WHERE { "
+                + "?person <urn:talksTo> ?otherPerson . "
+                + "?person ?action <urn:Bob>"
+                + "}";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
 
         // Create some statements where some generates SP results and others do not.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -185,14 +138,10 @@ public class StatementPatternProcessorIT {
         // Show the correct binding set results from the job.
         final Set<VisibilityBindingSet> expected = new HashSet<>();
 
-        QueryBindingSet bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        bs = new QueryBindingSet();
+        final QueryBindingSet bs = new QueryBindingSet();
         bs.addBinding("person", vf.createURI("urn:Alice"));
         bs.addBinding("action", vf.createURI("urn:talksTo"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
@@ -207,27 +156,13 @@ public class StatementPatternProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the StatementPattern object that will be evaluated.
-        final StatementPattern sp1 = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?otherPerson }");
-        final StatementPattern sp2 = RdfTestUtil.getSp("SELECT * WHERE { ?person ?action <urn:Bob> }");
-
         // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("SP1", new StatementPatternProcessorSupplier(sp1, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
-
-        // Add a processor that handles the second statement pattern.
-        builder.addProcessor("SP2", new StatementPatternProcessorSupplier(sp2, result -> ProcessorResult.make( new UnaryResult(result) )), "STATEMENTS");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "SP1", "SP2");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String query = "SELECT * WHERE { "
+                + "?person <urn:talksTo> ?otherPerson ."
+                + "?person ?action <urn:Bob>"
+                + "}";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
 
         // Create some statements where some generates SP results and others do not.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -240,24 +175,17 @@ public class StatementPatternProcessorIT {
         final Set<VisibilityBindingSet> expected = new HashSet<>();
 
         QueryBindingSet bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
-        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
         bs = new QueryBindingSet();
         bs.addBinding("person", vf.createURI("urn:Alice"));
         bs.addBinding("action", vf.createURI("urn:talksTo"));
-        expected.add( new VisibilityBindingSet(bs, "a") );
-
-        bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Alice"));
         bs.addBinding("otherPerson", vf.createURI("urn:Charlie"));
-        expected.add( new VisibilityBindingSet(bs, "a|b") );
+        expected.add(new VisibilityBindingSet(bs, "a&(a|b)"));
 
         bs = new QueryBindingSet();
-        bs.addBinding("person", vf.createURI("urn:Charlie"));
-        bs.addBinding("action", vf.createURI("urn:walksWith"));
-        expected.add( new VisibilityBindingSet(bs, "b") );
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("action", vf.createURI("urn:talksTo"));
+        bs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add(new VisibilityBindingSet(bs, "a"));
 
         // Run the test.
         KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/83d09f42/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
index 7051efa..dbad15c 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
@@ -46,6 +46,7 @@ import org.apache.rya.streams.kafka.processors.output.BindingSetOutputFormatterS
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerde;
 import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
 import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
 import org.apache.rya.test.kafka.KafkaTestInstanceRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -82,46 +83,13 @@ public class JoinProcessorIT {
         final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
         final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
 
-        // Get the StatementPatterns that will be evaluated.
-        final StatementPattern leftSp = RdfTestUtil.getSp("SELECT * WHERE { ?person <urn:talksTo> ?employee }");
-        final StatementPattern rightSp = RdfTestUtil.getSp("SELECT * WHERE { ?employee <urn:worksAt> ?business }");
-
         // Setup a topology.
-        final TopologyBuilder builder = new TopologyBuilder();
-
-        // The topic that Statements are written to is used as a source.
-        builder.addSource("STATEMENTS", new StringDeserializer(), new VisibilityStatementDeserializer(), statementsTopic);
-
-        // Add a processor that handles the first statement pattern.
-        builder.addProcessor("LEFT_SP", new StatementPatternProcessorSupplier(leftSp,
-                result -> ProcessorResult.make( new BinaryResult(Side.LEFT, result) )), "STATEMENTS");
-
-        // Add a processor that handles the second statement pattern.
-        builder.addProcessor("RIGHT_SP", new StatementPatternProcessorSupplier(rightSp,
-                result -> ProcessorResult.make( new BinaryResult(Side.RIGHT, result) )), "STATEMENTS");
-
-        // Add a processor that handles a natrual join over the SPs.
-        builder.addProcessor("NATURAL_JOIN", new JoinProcessorSupplier(
-                "NATURAL_JOIN",
-                new NaturalJoin(),
-                Lists.newArrayList("employee"),
-                Lists.newArrayList("employee", "person", "business"),
-                result -> ProcessorResult.make( new UnaryResult(result) )), "LEFT_SP", "RIGHT_SP");
-
-        // Add a state store for the join processor.
-        final StateStoreSupplier joinStoreSupplier =
-                Stores.create( "NATURAL_JOIN" )
-                  .withStringKeys()
-                  .withValues(new VisibilityBindingSetSerde())
-                  .inMemory()
-                  .build();
-        builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
-
-        // Add a processor that formats the VisibilityBindingSet for output.
-        builder.addProcessor("SINK_FORMATTER", BindingSetOutputFormatter::new, "NATURAL_JOIN");
-
-        // Add a sink that writes the data out to a new Kafka topic.
-        builder.addSink("QUERY_RESULTS", resultsTopic, new StringSerializer(), new VisibilityBindingSetSerializer(), "SINK_FORMATTER");
+        final String query = "SELECT * WHERE { "
+                + "?person <urn:talksTo> ?employee ."
+                + "?employee <urn:worksAt> ?business"
+                + " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic);
 
         // Create some statements that generate a bunch of right SP results.
         final ValueFactory vf = new ValueFactoryImpl();
@@ -194,10 +162,10 @@ public class JoinProcessorIT {
         // Add a state store for the join processor.
         final StateStoreSupplier joinStoreSupplier =
                 Stores.create( "NATURAL_JOIN" )
-                  .withStringKeys()
-                  .withValues(new VisibilityBindingSetSerde())
-                  .inMemory()
-                  .build();
+                .withStringKeys()
+                .withValues(new VisibilityBindingSetSerde())
+                .inMemory()
+                .build();
         builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
 
         // Add a processor that formats the VisibilityBindingSet for output.
@@ -277,10 +245,10 @@ public class JoinProcessorIT {
         // Add a state store for the join processor.
         final StateStoreSupplier joinStoreSupplier =
                 Stores.create( "NATURAL_JOIN" )
-                  .withStringKeys()
-                  .withValues(new VisibilityBindingSetSerde())
-                  .inMemory()
-                  .build();
+                .withStringKeys()
+                .withValues(new VisibilityBindingSetSerde())
+                .inMemory()
+                .build();
         builder.addStateStore(joinStoreSupplier, "NATURAL_JOIN");
 
         // Add a processor that formats the VisibilityBindingSet for output.
@@ -379,18 +347,18 @@ public class JoinProcessorIT {
         // Setup the join state suppliers.
         final StateStoreSupplier join1StoreSupplier =
                 Stores.create( "JOIN1" )
-                  .withStringKeys()
-                  .withValues(new VisibilityBindingSetSerde())
-                  .inMemory()
-                  .build();
+                .withStringKeys()
+                .withValues(new VisibilityBindingSetSerde())
+                .inMemory()
+                .build();
         builder.addStateStore(join1StoreSupplier, "JOIN1");
 
         final StateStoreSupplier join2StoreSupplier =
                 Stores.create( "JOIN2" )
-                  .withStringKeys()
-                  .withValues(new VisibilityBindingSetSerde())
-                  .inMemory()
-                  .build();
+                .withStringKeys()
+                .withValues(new VisibilityBindingSetSerde())
+                .inMemory()
+                .build();
         builder.addStateStore(join2StoreSupplier, "JOIN2");
 
         // Add a processor that formats the VisibilityBindingSet for output.
@@ -459,10 +427,10 @@ public class JoinProcessorIT {
         // Add a state store for the join processor.
         final StateStoreSupplier joinStoreSupplier =
                 Stores.create( "LEFT_JOIN" )
-                  .withStringKeys()
-                  .withValues(new VisibilityBindingSetSerde())
-                  .inMemory()
-                  .build();
+                .withStringKeys()
+                .withValues(new VisibilityBindingSetSerde())
+                .inMemory()
+                .build();
         builder.addStateStore(joinStoreSupplier, "LEFT_JOIN");
 
         // Add a processor that formats the VisibilityBindingSet for output.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/83d09f42/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/topology/TopologyFactoryTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/topology/TopologyFactoryTest.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/topology/TopologyFactoryTest.java
new file mode 100644
index 0000000..eda4c89
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/topology/TopologyFactoryTest.java
@@ -0,0 +1,106 @@
+package org.apache.rya.streams.kafka.topology;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.rya.streams.kafka.topology.TopologyFactory.ProcessorEntry;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.Join;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.Var;
+
+public class TopologyFactoryTest {
+    private static TopologyFactory FACTORY;
+
+    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
+    private static final Var TALKS_TO = new Var("-const-urn:talksTo", VF.createURI("urn:talksTo"));
+    private static final Var CHEWS = new Var("-const-urn:chews", VF.createURI("urn:chews"));
+
+    static {
+        TALKS_TO.setAnonymous(true);
+        TALKS_TO.setConstant(true);
+        CHEWS.setAnonymous(true);
+        CHEWS.setConstant(true);
+    }
+
+    @Before
+    public void setup() {
+        FACTORY = new TopologyFactory();
+    }
+
+    @Test
+    public void projectionStatementPattern() throws Exception {
+        final String query = "SELECT * WHERE { "
+                + "?person <urn:talksTo> ?otherPerson . "
+                + "}";
+
+        FACTORY.build(query, "source", "sink");
+        final List<ProcessorEntry> entries = FACTORY.getProcessorEntry();
+
+        assertTrue(entries.get(0).getNode() instanceof Projection);
+        assertTrue(entries.get(1).getNode() instanceof StatementPattern);
+
+        final StatementPattern expected = new StatementPattern(new Var("person"), TALKS_TO, new Var("otherPerson"));
+        assertEquals(expected, entries.get(1).getNode());
+    }
+
+    @Test
+    public void projectionJoinStatementPattern() throws Exception {
+        final String query = "SELECT * WHERE { "
+                + "?person <urn:talksTo> ?otherPerson . "
+                + "?otherPerson <urn:talksTo> ?dog . "
+                + "}";
+
+        FACTORY.build(query, "source", "sink");
+        final List<ProcessorEntry> entries = FACTORY.getProcessorEntry();
+
+        assertTrue(entries.get(0).getNode() instanceof Projection);
+        assertTrue(entries.get(1).getNode() instanceof Join);
+        StatementPattern expected = new StatementPattern(new Var("person"), TALKS_TO, new Var("otherPerson"));
+        assertEquals(expected, entries.get(2).getNode());
+        expected = new StatementPattern(new Var("otherPerson"), TALKS_TO, new Var("dog"));
+        assertEquals(expected, entries.get(3).getNode());
+    }
+
+    @Test
+    public void projectionJoinJoinStatementPattern() throws Exception {
+        final String query = "SELECT * WHERE { "
+                + "?person <urn:talksTo> ?otherPerson . "
+                + "?otherPerson <urn:talksTo> ?dog . "
+                + "?dog <urn:chews> ?toy . "
+                + "}";
+
+        FACTORY.build(query, "source", "sink");
+        final List<ProcessorEntry> entries = FACTORY.getProcessorEntry();
+
+        assertTrue(entries.get(0).getNode() instanceof Projection);
+        assertTrue(entries.get(1).getNode() instanceof Join);
+        assertTrue(entries.get(2).getNode() instanceof Join);
+        StatementPattern expected = new StatementPattern(new Var("person"), TALKS_TO, new Var("otherPerson"));
+        assertEquals(expected, entries.get(3).getNode());
+        expected = new StatementPattern(new Var("otherPerson"), TALKS_TO, new Var("dog"));
+        assertEquals(expected, entries.get(4).getNode());
+        expected = new StatementPattern(new Var("dog"), CHEWS, new Var("toy"));
+        assertEquals(expected, entries.get(5).getNode());
+    }
+
+    @Test
+    public void projectionStatementPattern_rebind() throws Exception {
+        final String query = "CONSTRUCT { ?person <urn:mightKnow> ?otherPerson } WHERE { "
+                + "?person <urn:talksTo> ?otherPerson . "
+            + "}";
+
+        FACTORY.build(query, "source", "sink");
+        final List<ProcessorEntry> entries = FACTORY.getProcessorEntry();
+
+        assertTrue(entries.get(0).getNode() instanceof Projection);
+        final StatementPattern expected = new StatementPattern(new Var("person"), TALKS_TO, new Var("otherPerson"));
+        assertEquals(expected, entries.get(1).getNode());
+    }
+}



[46/50] [abbrv] incubator-rya git commit: RYA-377 Before Instant Function

Posted by ca...@apache.org.
RYA-377 Before Instant Function


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

Branch: refs/heads/master
Commit: 2957c75a930396f86b606be8f7373dd3adcaf46c
Parents: bc92591
Author: Andrew Smith <sm...@gmail.com>
Authored: Fri Dec 1 12:40:27 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../temporal/BeforeTemporalInstant.java         | 46 ++++++++++++
 .../api/function/temporal/EqualsTemporal.java   |  3 +
 ...f.query.algebra.evaluation.function.Function |  5 +-
 .../temporal/BeforeTemporalFunctionsTest.java   | 75 ++++++++++++++++++++
 .../temporal/EqualsTemporalFunctionsTest.java   | 75 ++++++++++++++++++++
 .../temporal/TemporalFunctionsTest.java         | 75 --------------------
 .../processors/filter/TemporalFilterIT.java     | 32 +++++----
 7 files changed, 220 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2957c75a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/BeforeTemporalInstant.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/BeforeTemporalInstant.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/BeforeTemporalInstant.java
new file mode 100644
index 0000000..c9f31c3
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/BeforeTemporalInstant.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.rya.api.function.temporal;
+
+import java.time.ZonedDateTime;
+import java.util.Objects;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Filter function in a SPARQL query used to filter when a point of time is before another.
+ */
+@DefaultAnnotation(NonNull.class)
+public class BeforeTemporalInstant extends TemporalRelationFunction {
+    public static final String URI = BASE_URI + "before";
+
+    @Override
+    public String getURI() {
+        return URI;
+    }
+
+    @Override
+    protected boolean relation(final ZonedDateTime d1, final ZonedDateTime d2) {
+        Objects.requireNonNull(d1);
+        Objects.requireNonNull(d2);
+        return d1.isBefore(d2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2957c75a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
index c8a6041..07391c8 100644
--- a/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/temporal/EqualsTemporal.java
@@ -25,6 +25,9 @@ import java.util.Objects;
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
 
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
 /**
  * Filter function in a SPARQL query used to filter equality over time.
  */

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2957c75a/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function b/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
index 475b9dd..3cb1c56 100644
--- a/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
+++ b/common/rya.api.function/src/main/resources/META-INF/services/org.openrdf.query.algebra.evaluation.function.Function
@@ -1,3 +1,4 @@
+#
 # 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
@@ -14,4 +15,6 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-org.apache.rya.api.function.temporal.EqualsTemporal
\ No newline at end of file
+#
+org.apache.rya.api.function.temporal.EqualsTemporal
+org.apache.rya.api.function.temporal.BeforeTemporalInstant

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2957c75a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/BeforeTemporalFunctionsTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/BeforeTemporalFunctionsTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/BeforeTemporalFunctionsTest.java
new file mode 100644
index 0000000..1b15035
--- /dev/null
+++ b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/BeforeTemporalFunctionsTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.temporal;
+
+import static org.junit.Assert.assertEquals;
+
+import java.time.ZonedDateTime;
+
+import org.junit.Test;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+public class BeforeTemporalFunctionsTest {
+    private static final ZonedDateTime TIME = ZonedDateTime.parse("2015-12-30T12:00:00Z");
+    private static final ZonedDateTime TIME_10 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
+    private static final ZonedDateTime TIME_20 = ZonedDateTime.parse("2015-12-30T12:00:20Z");
+
+    final ValueFactory VF = ValueFactoryImpl.getInstance();
+
+    @Test
+    public void testBefore_same() throws Exception {
+        final BeforeTemporalInstant function = new BeforeTemporalInstant();
+
+        // 2 times equal
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+
+    @Test
+    public void testBefore_before() throws Exception {
+        final BeforeTemporalInstant function = new BeforeTemporalInstant();
+
+        // 2 times equal
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME_10.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(true), rez);
+    }
+
+    @Test
+    public void testBefore_after() throws Exception {
+        final BeforeTemporalInstant function = new BeforeTemporalInstant();
+
+        // 2 times equal
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME_20.toString());
+        args[1] = VF.createLiteral(TIME.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2957c75a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/EqualsTemporalFunctionsTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/EqualsTemporalFunctionsTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/EqualsTemporalFunctionsTest.java
new file mode 100644
index 0000000..f32904a
--- /dev/null
+++ b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/EqualsTemporalFunctionsTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.temporal;
+
+import static org.junit.Assert.assertEquals;
+
+import java.time.ZonedDateTime;
+
+import org.junit.Test;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+public class EqualsTemporalFunctionsTest {
+    private static final ZonedDateTime TIME = ZonedDateTime.parse("2015-12-30T12:00:00Z");
+    private static final ZonedDateTime TIME_10 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
+    private static final ZonedDateTime TIME_20 = ZonedDateTime.parse("2015-12-30T12:00:20Z");
+
+    final ValueFactory VF = ValueFactoryImpl.getInstance();
+
+    @Test
+    public void testEquals_equal() throws Exception {
+        final EqualsTemporal function = new EqualsTemporal();
+
+        // 2 times equal
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(true), rez);
+    }
+
+    @Test
+    public void testEquals_before() throws Exception {
+        final EqualsTemporal function = new EqualsTemporal();
+
+        // first time is before
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME.toString());
+        args[1] = VF.createLiteral(TIME_10.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+
+    @Test
+    public void testEquals_after() throws Exception {
+        final EqualsTemporal function = new EqualsTemporal();
+
+        // first time is after
+        final Value[] args = new Value[2];
+        args[0] = VF.createLiteral(TIME_20.toString());
+        args[1] = VF.createLiteral(TIME_10.toString());
+        final Value rez = function.evaluate(VF, args);
+
+        assertEquals(VF.createLiteral(false), rez);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2957c75a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/TemporalFunctionsTest.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/TemporalFunctionsTest.java b/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/TemporalFunctionsTest.java
deleted file mode 100644
index e0dabe1..0000000
--- a/common/rya.api.function/src/main/test/org/apache/rya/api/function/temporal/TemporalFunctionsTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.rya.api.function.temporal;
-
-import static org.junit.Assert.assertEquals;
-
-import java.time.ZonedDateTime;
-
-import org.junit.Test;
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-
-public class TemporalFunctionsTest {
-    private static final ZonedDateTime TIME = ZonedDateTime.parse("2015-12-30T12:00:00Z");
-    private static final ZonedDateTime TIME_10 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
-    private static final ZonedDateTime TIME_20 = ZonedDateTime.parse("2015-12-30T12:00:20Z");
-
-    final ValueFactory VF = ValueFactoryImpl.getInstance();
-
-    @Test
-    public void testEquals_equal() throws Exception {
-        final EqualsTemporal function = new EqualsTemporal();
-
-        // 2 times equal
-        final Value[] args = new Value[2];
-        args[0] = VF.createLiteral(TIME.toString());
-        args[1] = VF.createLiteral(TIME.toString());
-        final Value rez = function.evaluate(VF, args);
-
-        assertEquals(VF.createLiteral(true), rez);
-    }
-
-    @Test
-    public void testEquals_before() throws Exception {
-        final EqualsTemporal function = new EqualsTemporal();
-
-        // first time is before
-        final Value[] args = new Value[2];
-        args[0] = VF.createLiteral(TIME.toString());
-        args[1] = VF.createLiteral(TIME_10.toString());
-        final Value rez = function.evaluate(VF, args);
-
-        assertEquals(VF.createLiteral(false), rez);
-    }
-
-    @Test
-    public void testEquals_after() throws Exception {
-        final EqualsTemporal function = new EqualsTemporal();
-
-        // first time is after
-        final Value[] args = new Value[2];
-        args[0] = VF.createLiteral(TIME_20.toString());
-        args[1] = VF.createLiteral(TIME_10.toString());
-        final Value rez = function.evaluate(VF, args);
-
-        assertEquals(VF.createLiteral(false), rez);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/2957c75a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
index 2bc98ca..837b57b 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
@@ -57,8 +57,9 @@ import org.openrdf.query.impl.MapBindingSet;
 public class TemporalFilterIT {
     private static final ValueFactory vf = new ValueFactoryImpl();
     private static final String TEMPORAL = "http://rya.apache.org/ns/temporal";
-    private static final ZonedDateTime time1 = ZonedDateTime.parse("2015-12-30T12:00:00Z");
-    private static final ZonedDateTime time2 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
+    private static final ZonedDateTime TIME = ZonedDateTime.parse("2015-12-30T12:00:00Z");
+    private static final ZonedDateTime TIME_10 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
+    private static final ZonedDateTime TIME_20 = ZonedDateTime.parse("2015-12-30T12:00:20Z");
 
     @Rule
     public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(false);
@@ -73,12 +74,12 @@ public class TemporalFilterIT {
             }
         }
 
-        // There are 1 temporal functions registered, ensure that there are 1.
-        assertEquals(1, count);
+        // There are 2 temporal functions registered, ensure that there are 2.
+        assertEquals(2, count);
     }
 
     @Test
-    public void showProcessorWorks() throws Exception {
+    public void showEqualsWorks() throws Exception {
         // Enumerate some topics that will be re-used
         final String ryaInstance = UUID.randomUUID().toString();
         final UUID queryId = UUID.randomUUID();
@@ -88,12 +89,12 @@ public class TemporalFilterIT {
         // Get the RDF model objects that will be used to build the query.
         final String sparql =
                 "PREFIX time: <http://www.w3.org/2006/time/> \n"
-                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
-                        + "SELECT * \n"
-                        + "WHERE { \n"
-                        + "  <urn:time> time:atTime ?date .\n"
-                        + " FILTER(tempf:equals(?date, \"" + time1.toString() + "\")) "
-                        + "}";
+            + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+            + "SELECT * \n"
+            + "WHERE { \n"
+            + "  <urn:time> time:atTime ?date .\n"
+            + " FILTER(tempf:equals(?date, \"" + TIME.toString() + "\")) "
+            + "}";
         // Setup a topology.
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
@@ -104,7 +105,7 @@ public class TemporalFilterIT {
         // Make the expected results.
         final Set<VisibilityBindingSet> expected = new HashSet<>();
         final MapBindingSet bs = new MapBindingSet();
-        bs.addBinding("date", vf.createLiteral(time1.toString()));
+        bs.addBinding("date", vf.createLiteral(TIME.toString()));
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
@@ -113,8 +114,9 @@ public class TemporalFilterIT {
 
     private List<VisibilityStatement> getStatements() throws Exception {
         final List<VisibilityStatement> statements = new ArrayList<>();
-        statements.add(new VisibilityStatement(statement(time1), "a"));
-        statements.add(new VisibilityStatement(statement(time2), "a"));
+        statements.add(new VisibilityStatement(statement(TIME), "a"));
+        statements.add(new VisibilityStatement(statement(TIME_10), "a"));
+        statements.add(new VisibilityStatement(statement(TIME_20), "a"));
         return statements;
     }
 
@@ -124,4 +126,4 @@ public class TemporalFilterIT {
         final Value object = vf.createLiteral(time.toString());
         return new StatementImpl(subject, predicate, object);
     }
-}
\ No newline at end of file
+}


[25/50] [abbrv] incubator-rya git commit: RYA-377 Fixed a bug where the Rya Streams client couldn't use some of the included Statement parsers.

Posted by ca...@apache.org.
RYA-377 Fixed a bug where the Rya Streams client couldn't use some of the included Statement parsers.


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

Branch: refs/heads/master
Commit: a51765e07af7faf9d0c84a0147b7a31ee1e4c660
Parents: 5e5d7c8
Author: kchilton2 <ke...@gmail.com>
Authored: Fri Dec 8 13:27:03 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 extras/rya.streams/client/pom.xml               | 48 ++++++++++++++------
 .../client/command/LoadStatementsCommand.java   |  1 +
 .../kafka/interactor/KafkaLoadStatements.java   |  8 +++-
 pom.xml                                         | 40 ++++++++--------
 4 files changed, 62 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a51765e0/extras/rya.streams/client/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/pom.xml b/extras/rya.streams/client/pom.xml
index 2e49952..6febd64 100644
--- a/extras/rya.streams/client/pom.xml
+++ b/extras/rya.streams/client/pom.xml
@@ -48,6 +48,28 @@ under the License.
             <artifactId>rya.api</artifactId>
         </dependency>
 
+        <!-- Statement formats we support for loading. -->
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-rio-nquads</artifactId>
+        </dependency>
+       <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-rio-ntriples</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-rio-rdfxml</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-rio-trig</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.openrdf.sesame</groupId>
+            <artifactId>sesame-rio-turtle</artifactId>
+        </dependency>
+            
         <!-- Third Party dependencies -->
         <dependency>
             <groupId>com.beust</groupId>
@@ -79,27 +101,25 @@ under the License.
     <build>
         <plugins>
             <plugin>
-                <artifactId>maven-assembly-plugin</artifactId>
-                <configuration>
-                    <descriptorRefs>
-                        <descriptorRef>jar-with-dependencies</descriptorRef>
-                    </descriptorRefs>
-                    <archive>
-                        <manifest>
-                            <mainClass>org.apache.rya.streams.client.CLIDriver</mainClass>
-                        </manifest>
-                    </archive>
-                </configuration>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
                 <executions>
                     <execution>
-                        <id>make-assembly</id>
                         <phase>package</phase>
                         <goals>
-                            <goal>single</goal>
+                          <goal>shade</goal>
                         </goals>
+                        <configuration>
+                            <transformers>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                    <mainClass>org.apache.rya.streams.client.CLIDriver</mainClass>
+                                </transformer>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                            </transformers>
+                        </configuration>
                     </execution>
                 </executions>
             </plugin>
         </plugins>
      </build>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a51765e0/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
index 42020b3..bb31830 100644
--- a/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
+++ b/extras/rya.streams/client/src/main/java/org/apache/rya/streams/client/command/LoadStatementsCommand.java
@@ -131,6 +131,7 @@ public class LoadStatementsCommand implements RyaStreamsCommand {
             statements.fromFile(statementsPath, params.visibilities);
         } catch (final Exception e) {
             System.err.println("Unable to parse statements file: " + statementsPath.toString());
+            e.printStackTrace();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a51765e0/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
index d3ec650..cf10f6a 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaLoadStatements.java
@@ -69,8 +69,14 @@ public class KafkaLoadStatements implements LoadStatements {
         requireNonNull(statementsPath);
         requireNonNull(visibilities);
 
+        if(!statementsPath.toFile().exists()) {
+            throw new RyaStreamsException("Could not load statements at path '" + statementsPath + "' because that " +
+                    "does not exist. Make sure you've entered the correct path.");
+        }
+
         // Create an RDF Parser whose format is derived from the statementPath's file extension.
-        final RDFParser parser = Rio.createParser(RDFFormat.forFileName(statementsPath.getFileName().toString()));
+        final RDFFormat format = RDFFormat.forFileName(statementsPath.getFileName().toString());
+        final RDFParser parser = Rio.createParser(format);
 
         // Set a handler that writes the statements to the specified kafka topic.
         parser.setRDFHandler(new RDFHandlerBase() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a51765e0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6b2ad61..58cc7c5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -412,57 +412,57 @@ under the License.
             </dependency>
             <dependency>
                 <groupId>org.openrdf.sesame</groupId>
-                <artifactId>sesame-rio-rdfxml</artifactId>
+                <artifactId>sesame-rio-nquads</artifactId>
                 <version>${openrdf.sesame.version}</version>
             </dependency>
-            <dependency>
+           <dependency>
                 <groupId>org.openrdf.sesame</groupId>
-                <artifactId>sesame-queryrender</artifactId>
+                <artifactId>sesame-rio-ntriples</artifactId>
                 <version>${openrdf.sesame.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.openrdf.sesame</groupId>
-                <artifactId>sesame-runtime-osgi</artifactId>
+                <artifactId>sesame-rio-rdfxml</artifactId>
                 <version>${openrdf.sesame.version}</version>
-                <exclusions>
-                    <exclusion>
-                        <groupId>org.openrdf.sesame</groupId>
-                        <artifactId>sesame-http-client</artifactId>
-                    </exclusion>
-                    <exclusion>
-                        <groupId>org.openrdf.sesame</groupId>
-                        <artifactId>sesame-http-server-spring</artifactId>
-                    </exclusion>
-                </exclusions>
             </dependency>
             <dependency>
                 <groupId>org.openrdf.sesame</groupId>
-                <artifactId>sesame-rio-ntriples</artifactId>
+                <artifactId>sesame-rio-trig</artifactId>
                 <version>${openrdf.sesame.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.openrdf.sesame</groupId>
-                <artifactId>sesame-rio-nquads</artifactId>
+                <artifactId>sesame-rio-turtle</artifactId>
                 <version>${openrdf.sesame.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.openrdf.sesame</groupId>
-                <artifactId>sesame-rio-trig</artifactId>
+                <artifactId>sesame-queryrender</artifactId>
                 <version>${openrdf.sesame.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.openrdf.sesame</groupId>
-                <artifactId>sesame-queryalgebra-evaluation</artifactId>
+                <artifactId>sesame-runtime-osgi</artifactId>
                 <version>${openrdf.sesame.version}</version>
+                <exclusions>
+                    <exclusion>
+                        <groupId>org.openrdf.sesame</groupId>
+                        <artifactId>sesame-http-client</artifactId>
+                    </exclusion>
+                    <exclusion>
+                        <groupId>org.openrdf.sesame</groupId>
+                        <artifactId>sesame-http-server-spring</artifactId>
+                    </exclusion>
+                </exclusions>
             </dependency>
             <dependency>
                 <groupId>org.openrdf.sesame</groupId>
-                <artifactId>sesame-queryresultio-sparqljson</artifactId>
+                <artifactId>sesame-queryalgebra-evaluation</artifactId>
                 <version>${openrdf.sesame.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.openrdf.sesame</groupId>
-                <artifactId>sesame-rio-turtle</artifactId>
+                <artifactId>sesame-queryresultio-sparqljson</artifactId>
                 <version>${openrdf.sesame.version}</version>
             </dependency>
             <dependency>


[26/50] [abbrv] incubator-rya git commit: RYA-377 Geo Filter support

Posted by ca...@apache.org.
RYA-377 Geo Filter support

Added geo filter support to rya streams.


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

Branch: refs/heads/master
Commit: 9cd0c568d14f104a1604c2b5425a7fe9d154fc24
Parents: 9442322
Author: Andrew Smith <sm...@gmail.com>
Authored: Tue Nov 28 13:12:57 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../rya.pcj.fluo/rya.pcj.functions.geo/pom.xml  |  34 +----
 extras/rya.streams/kafka/pom.xml                |  22 ++-
 .../kafka/processors/filter/GeoFilterIT.java    | 139 +++++++++++++++++++
 3 files changed, 162 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9cd0c568/extras/rya.pcj.fluo/rya.pcj.functions.geo/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/rya.pcj.functions.geo/pom.xml b/extras/rya.pcj.fluo/rya.pcj.functions.geo/pom.xml
index b9b2143..373d869 100644
--- a/extras/rya.pcj.fluo/rya.pcj.functions.geo/pom.xml
+++ b/extras/rya.pcj.fluo/rya.pcj.functions.geo/pom.xml
@@ -40,35 +40,15 @@ under the License.
             <artifactId>rya.api</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.rya</groupId>
-            <artifactId>rya.pcj.fluo.api</artifactId>
-        </dependency>
-        <dependency>
             <groupId>com.vividsolutions</groupId>
             <artifactId>jts</artifactId>
             <version>1.13</version>
         </dependency>
         <dependency>
-            <groupId>org.apache.rya</groupId>
-            <artifactId>rya.pcj.fluo.client</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.rya</groupId>
-            <artifactId>rya.indexing</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.rya</groupId>
-            <artifactId>rya.pcj.fluo.test.base</artifactId>
-        </dependency>
-        <dependency>
             <groupId>org.eclipse.rdf4j</groupId>
             <artifactId>rdf4j-queryalgebra-geosparql</artifactId>
             <version>2.2</version>
         </dependency>
-         <dependency>
-            <groupId>org.apache.fluo</groupId>
-            <artifactId>fluo-api</artifactId>
-        </dependency>
         <dependency>
             <groupId>org.eclipse.rdf4j</groupId>
             <artifactId>rdf4j-queryalgebra-evaluation</artifactId>
@@ -76,18 +56,8 @@ under the License.
         </dependency>
         <!-- Testing dependencies. -->
         <dependency>
-            <groupId>org.apache.fluo</groupId>
-            <artifactId>fluo-mini</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-             <groupId>org.apache.fluo</groupId>
-            <artifactId>fluo-recipes-test</artifactId>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.pcj.fluo.test.base</artifactId>
             <scope>test</scope>
         </dependency>
     </dependencies>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9cd0c568/extras/rya.streams/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/pom.xml b/extras/rya.streams/kafka/pom.xml
index 0ccbb6e..2d33f32 100644
--- a/extras/rya.streams/kafka/pom.xml
+++ b/extras/rya.streams/kafka/pom.xml
@@ -33,6 +33,26 @@ under the License.
         This module contains the Rya Streams components that integrate with Kafka.
     </description>
 
+    <profiles>
+        <profile>
+            <id>geoindexing</id>
+                <dependencies>
+                    <!-- Rya dependencies -->
+                    <dependency>
+                        <groupId>org.apache.rya</groupId>
+                        <artifactId>rya.pcj.functions.geo</artifactId>
+                        <version>3.2.12-incubating-SNAPSHOT</version>
+                    </dependency>
+                    <dependency>
+                        <groupId>org.apache.rya</groupId>
+                        <artifactId>rya.geo.common</artifactId>
+                        <version>${project.version}</version>
+                    </dependency>
+                </dependencies>
+        </profile>
+    </profiles>
+
+
     <dependencies>
         <!-- Rya dependencies -->
         <dependency>
@@ -95,4 +115,4 @@ under the License.
             <scope>test</scope>
         </dependency>
     </dependencies>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9cd0c568/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
new file mode 100644
index 0000000..4057e18
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.filter;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.indexing.GeoConstants;
+import org.apache.rya.streams.kafka.KafkaTestUtil;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.evaluation.function.Function;
+import org.openrdf.query.algebra.evaluation.function.FunctionRegistry;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.Geometry;
+import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.io.WKTWriter;
+
+/**
+ * Integration tests the methods of {@link FilterProcessor}.
+ */
+public class GeoFilterIT {
+    private static final String GEO = "http://www.opengis.net/def/function/geosparql/";
+    private static final GeometryFactory GF = new GeometryFactory();
+    private static final Geometry ZERO = GF.createPoint(new Coordinate(0, 0));
+    private static final Geometry ONE = GF.createPoint(new Coordinate(1, 1));
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void showGeoFunctionsRegistered() {
+        int count = 0;
+        final Collection<Function> funcs = FunctionRegistry.getInstance().getAll();
+        for (final Function fun : funcs) {
+            if (fun.getURI().startsWith(GEO)) {
+                count++;
+            }
+        }
+
+        // There are 30 geo functions registered, ensure that there are 30.
+        assertEquals(30, count);
+    }
+
+    @Test
+    public void showProcessorWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX time: <http://www.w3.org/2006/time#> \n"
+                        + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
+                        + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                        + "PREFIX geof: <" + GEO + ">\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:event1> geo:asWKT ?point .\n"
+                        + " FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+                        + "}";
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        final WKTWriter w = new WKTWriter();
+        bs.addBinding("point", vf.createLiteral(w.write(ZERO), GeoConstants.XMLSCHEMA_OGC_WKT));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        KafkaTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    private List<VisibilityStatement> getStatements() throws Exception {
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        // geo 2x2 points
+        statements.add(new VisibilityStatement(statement(ZERO), "a"));
+        statements.add(new VisibilityStatement(statement(ONE), "a"));
+        return statements;
+    }
+
+    private static Statement statement(final Geometry geo) {
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Resource subject = vf.createURI("urn:event1");
+        final URI predicate = GeoConstants.GEO_AS_WKT;
+        final WKTWriter w = new WKTWriter();
+        final Value object = vf.createLiteral(w.write(geo), GeoConstants.XMLSCHEMA_OGC_WKT);
+        return new StatementImpl(subject, predicate, object);
+    }
+}
\ No newline at end of file


[30/50] [abbrv] incubator-rya git commit: RYA-377 Fixing broken integration tests.

Posted by ca...@apache.org.
RYA-377 Fixing broken integration tests.


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

Branch: refs/heads/master
Commit: cf4336073a3365d2407a43ed779aa135bdc48091
Parents: 9e02a54
Author: kchilton2 <ke...@gmail.com>
Authored: Wed Dec 6 16:24:11 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../rya/streams/client/command/RunQueryCommandIT.java |  3 ++-
 .../apache/rya/streams/kafka/RyaStreamsTestUtil.java  |  4 +---
 .../kafka/processors/StatementPatternProcessorIT.java |  8 ++++----
 .../aggregation/AggregationProcessorIT.java           | 14 +++++++-------
 .../kafka/processors/filter/FilterProcessorIT.java    |  2 +-
 .../streams/kafka/processors/filter/GeoFilterIT.java  |  2 +-
 .../kafka/processors/filter/TemporalFilterIT.java     | 10 +++++-----
 .../kafka/processors/join/JoinProcessorIT.java        | 10 +++++-----
 .../projection/MultiProjectionProcessorIT.java        |  2 +-
 .../processors/projection/ProjectionProcessorIT.java  |  2 +-
 10 files changed, 28 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/RunQueryCommandIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/RunQueryCommandIT.java b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/RunQueryCommandIT.java
index 788b41f..f2100e8 100644
--- a/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/RunQueryCommandIT.java
+++ b/extras/rya.streams/client/src/test/java/org/apache/rya/streams/client/command/RunQueryCommandIT.java
@@ -102,7 +102,8 @@ public class RunQueryCommandIT {
                 "--ryaInstance", "" + ryaInstance,
                 "--kafkaHostname", kafka.getKafkaHostname(),
                 "--kafkaPort", kafka.getKafkaPort(),
-                "--queryID", UUID.randomUUID().toString()
+                "--queryID", UUID.randomUUID().toString(),
+                "--zookeepers", kafka.getZookeeperServers()
         };
 
         // Run the test. This will throw an exception.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
index b7e2be2..ee25f8c 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/RyaStreamsTestUtil.java
@@ -60,7 +60,6 @@ public class RyaStreamsTestUtil {
      * @param statementsTopic - The topic statements will be written to. (not null)
      * @param resultsTopic - The topic results will be read from. (not null)
      * @param builder - The streams topology that will be executed. (not null)
-     * @param startupMs - How long to wait for the topology to start before writing the statements.
      * @param statements - The statements that will be loaded into the topic. (not null)
      * @param expected - The expected results. (not null)
      * @param expectedDeserializerClass - The class of the deserializer that will be used when reading
@@ -72,7 +71,6 @@ public class RyaStreamsTestUtil {
             final String statementsTopic,
             final String resultsTopic,
             final TopologyBuilder builder,
-            final int startupMs,
             final List<VisibilityStatement> statements,
             final Set<T> expected,
             final Class<? extends Deserializer<T>> expectedDeserializerClass) throws Exception {
@@ -98,7 +96,7 @@ public class RyaStreamsTestUtil {
             streams.start();
 
             // Wait for the streams application to start. Streams only see data after their consumers are connected.
-            Thread.sleep(startupMs);
+            Thread.sleep(6000);
 
             // Load the statements into the input topic.
             try(Producer<String, VisibilityStatement> producer = KafkaTestUtil.makeProducer(

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
index 80b6e42..33dc945 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/StatementPatternProcessorIT.java
@@ -75,7 +75,7 @@ public class StatementPatternProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -113,7 +113,7 @@ public class StatementPatternProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a|b") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -147,7 +147,7 @@ public class StatementPatternProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -190,6 +190,6 @@ public class StatementPatternProcessorIT {
         expected.add(new VisibilityBindingSet(bs, "a"));
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
index ccf5c0c..072469a 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
@@ -95,7 +95,7 @@ public class AggregationProcessorIT {
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -142,7 +142,7 @@ public class AggregationProcessorIT {
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -188,7 +188,7 @@ public class AggregationProcessorIT {
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -238,7 +238,7 @@ public class AggregationProcessorIT {
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -288,7 +288,7 @@ public class AggregationProcessorIT {
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -389,7 +389,7 @@ public class AggregationProcessorIT {
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 4000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -452,6 +452,6 @@ public class AggregationProcessorIT {
         final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
index fb5305f..aaa67ea 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
@@ -81,6 +81,6 @@ public class FilterProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
index 642ecbc..c090afa 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
@@ -115,7 +115,7 @@ public class GeoFilterIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     private List<VisibilityStatement> getStatements() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
index cac9559..0a2185d 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
@@ -109,9 +109,9 @@ public class TemporalFilterIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
-    
+
     @Test
     public void showBeforeWorks() throws Exception {
         // Enumerate some topics that will be re-used
@@ -143,9 +143,9 @@ public class TemporalFilterIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
-    
+
     @Test
     public void showAfterWorks() throws Exception {
         // Enumerate some topics that will be re-used
@@ -177,7 +177,7 @@ public class TemporalFilterIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     private List<VisibilityStatement> getStatements() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
index 51bb0ae..bdb9be6 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
@@ -111,7 +111,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -162,7 +162,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -219,7 +219,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a&c") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -260,7 +260,7 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "a") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 3000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 
     @Test
@@ -311,6 +311,6 @@ public class JoinProcessorIT {
         expected.add( new VisibilityBindingSet(bs, "c") );
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
index c96919c..a8de401 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
@@ -87,6 +87,6 @@ public class MultiProjectionProcessorIT {
         expected.add(new VisibilityStatement(vf.createStatement(blankNode, vf.createURI("urn:location"), vf.createURI("urn:corner1")), "a"));
 
         // Run the test.
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, expected, VisibilityStatementDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityStatementDeserializer.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/cf433607/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
index 63c2cc7..2af3a49 100644
--- a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
@@ -80,6 +80,6 @@ public class ProjectionProcessorIT {
         expectedBs.addBinding("otherPerson", vf.createURI("urn:Bob"));
         expected.add(new VisibilityBindingSet(expectedBs, "a"));
 
-        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, 2000, statements, Sets.newHashSet(expected), VisibilityBindingSetDeserializer.class);
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, Sets.newHashSet(expected), VisibilityBindingSetDeserializer.class);
     }
 }
\ No newline at end of file


[42/50] [abbrv] incubator-rya git commit: RYA-377 Repackaged the common Aggregation code into the rya.api.functions project.

Posted by ca...@apache.org.
RYA-377 Repackaged the common Aggregation code into the rya.api.functions project.


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

Branch: refs/heads/master
Commit: 8363724b4d684c47fe806a4f364e40134964e6cb
Parents: 1535b46
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Nov 21 16:29:30 2017 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 .../aggregation/AggregationElement.java         | 105 +++++
 .../aggregation/AggregationFunction.java        |  41 ++
 .../function/aggregation/AggregationState.java  | 120 ++++++
 .../function/aggregation/AggregationType.java   |  66 +++
 .../function/aggregation/AverageFunction.java   |  96 +++++
 .../api/function/aggregation/AverageState.java  |  93 +++++
 .../api/function/aggregation/CountFunction.java |  61 +++
 .../api/function/aggregation/MaxFunction.java   |  63 +++
 .../api/function/aggregation/MinFunction.java   |  63 +++
 .../api/function/aggregation/SumFunction.java   |  85 ++++
 .../pcj/fluo/app/AggregationResultUpdater.java  | 410 +------------------
 .../fluo/app/observers/AggregationObserver.java |   4 +-
 .../pcj/fluo/app/query/AggregationMetadata.java | 130 +-----
 .../pcj/fluo/app/query/FluoQueryColumns.java    |   2 +-
 .../fluo/app/query/FluoQueryMetadataDAO.java    |  11 +-
 .../fluo/app/query/SparqlFluoQueryBuilder.java  | 255 ++++++------
 .../fluo/app/query/FluoQueryMetadataDAOIT.java  |  66 +--
 17 files changed, 994 insertions(+), 677 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationElement.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationElement.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationElement.java
new file mode 100644
index 0000000..3112059
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationElement.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Represents all of the metadata require to perform an Aggregation that is part of a SPARQL query.
+ * </p>
+ * For example, if you have the following in SPARQL:
+ * <pre>
+ * SELECT (avg(?price) as ?avgPrice) {
+ *     ...
+ * }
+ * </pre>
+ * You would construct an instance of this object like so:
+ * <pre>
+ * new AggregationElement(AggregationType.AVERAGE, "price", "avgPrice");
+ * </pre>
+ */
+@DefaultAnnotation(NonNull.class)
+public final class AggregationElement implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final AggregationType aggregationType;
+    private final String aggregatedBindingName;
+    private final String resultBindingName;
+
+    /**
+     * Constructs an instance of {@link AggregationElement}.
+     *
+     * @param aggregationType - Defines how the binding values will be aggregated. (not null)
+     * @param aggregatedBindingName - The name of the binding whose values is aggregated. This binding must
+     *   appear within the child node's emitted binding sets. (not null)
+     * @param resultBindingName - The name of the binding this aggregation's results are written to. This binding
+     *   must appeared within the AggregationMetadata's variable order. (not null)
+     */
+    public AggregationElement(
+            final AggregationType aggregationType,
+            final String aggregatedBindingName,
+            final String resultBindingName) {
+        this.aggregationType = requireNonNull(aggregationType);
+        this.aggregatedBindingName = requireNonNull(aggregatedBindingName);
+        this.resultBindingName = requireNonNull(resultBindingName);
+    }
+
+    /**
+     * @return Defines how the binding values will be aggregated.
+     */
+    public AggregationType getAggregationType() {
+        return aggregationType;
+    }
+
+    /**
+     * @return The name of the binding whose values is aggregated. This binding must appear within the child node's emitted binding sets.
+     */
+    public String getAggregatedBindingName() {
+        return aggregatedBindingName;
+    }
+
+    /**
+     * @return The name of the binding this aggregation's results are written to. This binding must appeared within the AggregationMetadata's variable order.
+     */
+    public String getResultBindingName() {
+        return resultBindingName;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(aggregationType, aggregatedBindingName, resultBindingName);
+    }
+
+    @Override
+    public boolean equals(final Object o ) {
+        if(o instanceof AggregationElement) {
+            final AggregationElement agg = (AggregationElement) o;
+            return Objects.equals(aggregationType, agg.aggregationType) &&
+                    Objects.equals(aggregatedBindingName, agg.aggregatedBindingName) &&
+                    Objects.equals(resultBindingName, agg.resultBindingName);
+        }
+        return false;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationFunction.java
new file mode 100644
index 0000000..e8c49e7
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationFunction.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A function that updates an {@link AggregationState}.
+ */
+@DefaultAnnotation(NonNull.class)
+public interface AggregationFunction {
+
+    /**
+     * Updates an {@link AggregationState} based on the values of a child Binding Set.
+     *
+     * @param aggregation - Defines which function needs to be performed as well as any details required
+     *   to do the aggregation work. (not null)
+     * @param state - The state that will be updated. (not null)
+     * @param childBindingSet - The Binding Set whose values will be used to update the state.
+     */
+    public void update(AggregationElement aggregation, AggregationState state, VisibilityBindingSet childBindingSet);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationState.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationState.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationState.java
new file mode 100644
index 0000000..2551696
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationState.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.openrdf.query.impl.MapBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Keeps track information required to update and build the resulting Binding Set for a set of Group By values.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class AggregationState implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    // The visibility equation that encompasses all data the aggregation state is derived from.
+    private String visibility;
+
+    // A binding set that holds the current state of the aggregations.
+    private final MapBindingSet bindingSet;
+
+    // A map from result binding name to the state that derived that binding's value.
+    private final Map<String, AverageState> avgStates;
+
+    /**
+     * Constructs an instance of {@link AggregationState}.
+     */
+    public AggregationState() {
+        this.visibility = "";
+        this.bindingSet = new MapBindingSet();
+        this.avgStates = new HashMap<>();
+    }
+
+    /**
+     * Constructs an instance of {@link AggregationState}.
+     *
+     * @param visibility - The visibility equation associated with the resulting binding set. (not null)
+     * @param bindingSet - The Binding Set whose values are being updated. It holds the result for a set of
+     *   Group By values. (not null)
+     * @param avgStates - If the aggregation is doing an Average, this is a map from result binding name to
+     *   average state for that binding.
+     */
+    public AggregationState(
+            final String visibility,
+            final MapBindingSet bindingSet,
+            final Map<String, AverageState> avgStates) {
+        this.visibility = requireNonNull(visibility);
+        this.bindingSet = requireNonNull(bindingSet);
+        this.avgStates = requireNonNull(avgStates);
+    }
+
+    /**
+     * @return The visibility equation associated with the resulting binding set.
+     */
+    public String getVisibility() {
+        return visibility;
+    }
+
+    /**
+     * @param visibility - The visibility equation associated with the resulting binding set.
+     */
+    public void setVisibility(final String visibility) {
+        this.visibility = requireNonNull(visibility);
+    }
+
+    /**
+     * @return The Binding Set whose values are being updated. It holds the result for a set of Group By values.
+     */
+    public MapBindingSet getBindingSet() {
+        return bindingSet;
+    }
+
+    /**
+     * @return If the aggregation is doing an Average, this is a map from result binding name to
+     *   average state for that binding.
+     */
+    public Map<String, AverageState> getAverageStates() {
+        return avgStates;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(visibility, bindingSet, avgStates);
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if(o instanceof AggregationState) {
+            final AggregationState state = (AggregationState) o;
+            return Objects.equals(visibility, state.visibility) &&
+                    Objects.equals(bindingSet, state.bindingSet) &&
+                    Objects.equals(avgStates, state.avgStates);
+        }
+        return false;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationType.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationType.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationType.java
new file mode 100644
index 0000000..5383da1
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AggregationType.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Optional;
+
+import org.openrdf.query.algebra.AggregateOperator;
+import org.openrdf.query.algebra.Avg;
+import org.openrdf.query.algebra.Count;
+import org.openrdf.query.algebra.Max;
+import org.openrdf.query.algebra.Min;
+import org.openrdf.query.algebra.Sum;
+
+import com.google.common.collect.ImmutableMap;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * The different types of Aggregation functions that an aggregate node may perform.
+ */
+@DefaultAnnotation(NonNull.class)
+public enum AggregationType {
+    MIN(Min.class),
+    MAX(Max.class),
+    COUNT(Count.class),
+    SUM(Sum.class),
+    AVERAGE(Avg.class);
+
+    private final Class<? extends AggregateOperator> operatorClass;
+
+    private AggregationType(final Class<? extends AggregateOperator> operatorClass) {
+        this.operatorClass = requireNonNull(operatorClass);
+    }
+
+    private static final ImmutableMap<Class<? extends AggregateOperator>, AggregationType> byOperatorClass;
+    static {
+        final ImmutableMap.Builder<Class<? extends AggregateOperator>, AggregationType> builder = ImmutableMap.builder();
+        for(final AggregationType type : AggregationType.values()) {
+            builder.put(type.operatorClass, type);
+        }
+        byOperatorClass = builder.build();
+    }
+
+    public static Optional<AggregationType> byOperatorClass(final Class<? extends AggregateOperator> operatorClass) {
+        return Optional.ofNullable( byOperatorClass.get(operatorClass) );
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageFunction.java
new file mode 100644
index 0000000..a73d5ac
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageFunction.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Map;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.model.Literal;
+import org.openrdf.model.Value;
+import org.openrdf.model.datatypes.XMLDatatypeUtil;
+import org.openrdf.model.impl.DecimalLiteralImpl;
+import org.openrdf.model.impl.IntegerLiteralImpl;
+import org.openrdf.query.algebra.MathExpr.MathOp;
+import org.openrdf.query.algebra.evaluation.ValueExprEvaluationException;
+import org.openrdf.query.algebra.evaluation.util.MathUtil;
+import org.openrdf.query.impl.MapBindingSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Update the {@link AggregationState}'s average if the child Binding Set contains the binding name
+ * that is being averaged by the {@link AggregationElement}.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class AverageFunction implements AggregationFunction {
+    private static final Logger log = LoggerFactory.getLogger(AverageFunction.class);
+
+    @Override
+    public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
+        checkArgument(aggregation.getAggregationType() == AggregationType.AVERAGE, "The AverageFunction only accepts AVERAGE AggregationElements.");
+
+        // Only update the average if the child contains the binding that we are averaging.
+        final String aggregatedName = aggregation.getAggregatedBindingName();
+        if(childBindingSet.hasBinding(aggregatedName)) {
+            final MapBindingSet result = state.getBindingSet();
+            final String resultName = aggregation.getResultBindingName();
+            final boolean newBinding = !result.hasBinding(resultName);
+
+            // Get the state of the average.
+            final Map<String, AverageState> averageStates = state.getAverageStates();
+            AverageState averageState = newBinding ? new AverageState() : averageStates.get(resultName);
+
+            // Update the state of the average.
+            final Value childValue = childBindingSet.getValue(aggregatedName);
+            if(childValue instanceof Literal) {
+                final Literal childLiteral = (Literal) childValue;
+                if (childLiteral.getDatatype() != null && XMLDatatypeUtil.isNumericDatatype(childLiteral.getDatatype())) {
+                    try {
+                        // Update the sum.
+                        final Literal oldSum = new DecimalLiteralImpl(averageState.getSum());
+                        final BigDecimal sum = MathUtil.compute(oldSum, childLiteral, MathOp.PLUS).decimalValue();
+
+                        // Update the count.
+                        final BigInteger count = averageState.getCount().add( BigInteger.ONE );
+
+                        // Update the BindingSet to include the new average.
+                        final Literal sumLiteral = new DecimalLiteralImpl(sum);
+                        final Literal countLiteral = new IntegerLiteralImpl(count);
+                        final Literal average = MathUtil.compute(sumLiteral, countLiteral, MathOp.DIVIDE);
+                        result.addBinding(resultName, average);
+
+                        // Update the average state that is stored.
+                        averageState = new AverageState(sum, count);
+                        averageStates.put(resultName, averageState);
+                    } catch (final ValueExprEvaluationException e) {
+                        log.error("A problem was encountered while updating an Average Aggregation. This binding set will be ignored: " + childBindingSet);
+                        return;
+                    }
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageState.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageState.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageState.java
new file mode 100644
index 0000000..8917751
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/AverageState.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Objects;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * The Sum and Count of the values that are being averaged. The average itself is derived from these values.
+ */
+@DefaultAnnotation(NonNull.class)
+public class AverageState implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final BigDecimal sum;
+    private final BigInteger count;
+
+    /**
+     * Constructs an instance of {@link AverageState} where the count and sum start at 0.
+     */
+    public AverageState() {
+        sum = BigDecimal.ZERO;
+        count = BigInteger.ZERO;
+    }
+
+    /**
+     * Constructs an instance of {@link AverageState}.
+     *
+     * @param sum - The sum of the values that are averaged. (not null)
+     * @param count - The number of values that are averaged. (not null)
+     */
+    public AverageState(final BigDecimal sum, final BigInteger count) {
+        this.sum = requireNonNull(sum);
+        this.count = requireNonNull(count);
+    }
+
+    /**
+     * @return The sum of the values that are averaged.
+     */
+    public BigDecimal getSum() {
+        return sum;
+    }
+
+    /**
+     * @return The number of values that are averaged.
+     */
+    public BigInteger getCount() {
+        return count;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(sum, count);
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if(o instanceof AverageState) {
+            final AverageState state = (AverageState) o;
+            return Objects.equals(sum, state.sum) &&
+                    Objects.equals(count, state.count);
+        }
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return "Sum: " + sum + " Count: " + count;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/CountFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/CountFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/CountFunction.java
new file mode 100644
index 0000000..7dd5b21
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/CountFunction.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.math.BigInteger;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.model.Literal;
+import org.openrdf.model.impl.IntegerLiteralImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Increments the {@link AggregationState}'s count if the child Binding Set contains the binding name
+ * that is being counted by the {@link AggregationElement}.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class CountFunction implements AggregationFunction {
+    @Override
+    public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
+        checkArgument(aggregation.getAggregationType() == AggregationType.COUNT, "The CountFunction only accepts COUNT AggregationElements.");
+
+        // Only add one to the count if the child contains the binding that we are counting.
+        final String aggregatedName = aggregation.getAggregatedBindingName();
+        if(childBindingSet.hasBinding(aggregatedName)) {
+            final MapBindingSet result = state.getBindingSet();
+            final String resultName = aggregation.getResultBindingName();
+            final boolean newBinding = !result.hasBinding(resultName);
+
+            if(newBinding) {
+                // Initialize the binding.
+                result.addBinding(resultName, new IntegerLiteralImpl(BigInteger.ONE));
+            } else {
+                // Update the existing binding.
+                final Literal count = (Literal) result.getValue(resultName);
+                final BigInteger updatedCount = count.integerValue().add( BigInteger.ONE );
+                result.addBinding(resultName, new IntegerLiteralImpl(updatedCount));
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MaxFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MaxFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MaxFunction.java
new file mode 100644
index 0000000..3295fbb
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MaxFunction.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.model.Value;
+import org.openrdf.query.algebra.evaluation.util.ValueComparator;
+import org.openrdf.query.impl.MapBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Update the {@link AggregationState}'s max if the child binding Set contains the binding name that is being
+ * maxed by the {@link AggregationElement}.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class MaxFunction implements AggregationFunction {
+
+    private final ValueComparator compare = new ValueComparator();
+
+    @Override
+    public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
+        checkArgument(aggregation.getAggregationType() == AggregationType.MAX, "The MaxFunction only accepts MAX AggregationElements.");
+
+        // Only update the max if the child contains the binding that we are finding the max value for.
+        final String aggregatedName = aggregation.getAggregatedBindingName();
+        if(childBindingSet.hasBinding(aggregatedName)) {
+            final MapBindingSet result = state.getBindingSet();
+            final String resultName = aggregation.getResultBindingName();
+            final boolean newBinding = !result.hasBinding(resultName);
+
+            Value max;
+            if(newBinding) {
+                max = childBindingSet.getValue(aggregatedName);
+            } else {
+                final Value oldMax = result.getValue(resultName);
+                final Value childMax = childBindingSet.getValue(aggregatedName);
+                max = compare.compare(childMax, oldMax) > 0 ? childMax : oldMax;
+            }
+
+            result.addBinding(resultName, max);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MinFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MinFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MinFunction.java
new file mode 100644
index 0000000..d6bf751
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/MinFunction.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.model.Value;
+import org.openrdf.query.algebra.evaluation.util.ValueComparator;
+import org.openrdf.query.impl.MapBindingSet;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Update the {@link AggregationState}'s min if the child binding Set contains the binding name that is being
+ * mined by the {@link AggregationElement}.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class MinFunction implements AggregationFunction {
+
+    private final ValueComparator compare = new ValueComparator();
+
+    @Override
+    public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
+        checkArgument(aggregation.getAggregationType() == AggregationType.MIN, "The MinFunction only accepts MIN AggregationElements.");
+
+        // Only update the min if the child contains the binding that we are finding the min value for.
+        final String aggregatedName = aggregation.getAggregatedBindingName();
+        if(childBindingSet.hasBinding(aggregatedName)) {
+            final MapBindingSet result = state.getBindingSet();
+            final String resultName = aggregation.getResultBindingName();
+            final boolean newBinding = !result.hasBinding(resultName);
+
+            Value min;
+            if(newBinding) {
+                min = childBindingSet.getValue(aggregatedName);
+            } else {
+                final Value oldMin = result.getValue(resultName);
+                final Value chidlMin = childBindingSet.getValue(aggregatedName);
+                min = compare.compare(chidlMin, oldMin) < 0 ? chidlMin : oldMin;
+            }
+
+            result.addBinding(resultName, min);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/SumFunction.java
----------------------------------------------------------------------
diff --git a/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/SumFunction.java b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/SumFunction.java
new file mode 100644
index 0000000..97735f2
--- /dev/null
+++ b/common/rya.api.function/src/main/java/org/apache/rya/api/function/aggregation/SumFunction.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.api.function.aggregation;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.math.BigInteger;
+
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.openrdf.model.Literal;
+import org.openrdf.model.Value;
+import org.openrdf.model.datatypes.XMLDatatypeUtil;
+import org.openrdf.model.impl.IntegerLiteralImpl;
+import org.openrdf.query.algebra.MathExpr.MathOp;
+import org.openrdf.query.algebra.evaluation.ValueExprEvaluationException;
+import org.openrdf.query.algebra.evaluation.util.MathUtil;
+import org.openrdf.query.impl.MapBindingSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * Add to the {@link AggregationState}'s sum if the child Binding Set contains the binding name
+ * that is being summed by the {@link AggregationElement}.
+ */
+@DefaultAnnotation(NonNull.class)
+public final class SumFunction implements AggregationFunction {
+    private static final Logger log = LoggerFactory.getLogger(SumFunction.class);
+
+    @Override
+    public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
+        checkArgument(aggregation.getAggregationType() == AggregationType.SUM, "The SumFunction only accepts SUM AggregationElements.");
+
+        // Only add values to the sum if the child contains the binding that we are summing.
+        final String aggregatedName = aggregation.getAggregatedBindingName();
+        if(childBindingSet.hasBinding(aggregatedName)) {
+            final MapBindingSet result = state.getBindingSet();
+            final String resultName = aggregation.getResultBindingName();
+            final boolean newBinding = !result.hasBinding(resultName);
+
+            // Get the starting number for the sum.
+            Literal sum;
+            if(newBinding) {
+                sum = new IntegerLiteralImpl(BigInteger.ZERO);
+            } else {
+                sum = (Literal) state.getBindingSet().getValue(resultName);
+            }
+
+            // Add the child binding set's value if it is a numeric literal.
+            final Value childValue = childBindingSet.getValue(aggregatedName);
+            if(childValue instanceof Literal) {
+                final Literal childLiteral = (Literal) childValue;
+                if (childLiteral.getDatatype() != null && XMLDatatypeUtil.isNumericDatatype(childLiteral.getDatatype())) {
+                    try {
+                        sum = MathUtil.compute(sum, childLiteral, MathOp.PLUS);
+                    } catch (final ValueExprEvaluationException e) {
+                        log.error("A problem was encountered while updating a Sum Aggregation. This binding set will be ignored: " + childBindingSet);
+                        return;
+                    }
+                }
+            }
+
+            // Update the state to include the new sum.
+            result.addBinding(resultName, sum);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AggregationResultUpdater.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AggregationResultUpdater.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AggregationResultUpdater.java
index bb96a6a..4fbaad9 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AggregationResultUpdater.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/AggregationResultUpdater.java
@@ -18,19 +18,12 @@
  */
 package org.apache.rya.indexing.pcj.fluo.app;
 
-import static com.google.common.base.Preconditions.checkArgument;
 import static java.util.Objects.requireNonNull;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.ObjectOutputStream;
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
 import java.util.Optional;
 
 import org.apache.commons.io.serialization.ValidatingObjectInputStream;
@@ -38,22 +31,21 @@ import org.apache.fluo.api.client.TransactionBase;
 import org.apache.fluo.api.data.Bytes;
 import org.apache.log4j.Logger;
 import org.apache.rya.accumulo.utils.VisibilitySimplifier;
+import org.apache.rya.api.function.aggregation.AggregationElement;
+import org.apache.rya.api.function.aggregation.AggregationFunction;
+import org.apache.rya.api.function.aggregation.AggregationState;
+import org.apache.rya.api.function.aggregation.AggregationType;
+import org.apache.rya.api.function.aggregation.AverageFunction;
+import org.apache.rya.api.function.aggregation.AverageState;
+import org.apache.rya.api.function.aggregation.CountFunction;
+import org.apache.rya.api.function.aggregation.MaxFunction;
+import org.apache.rya.api.function.aggregation.MinFunction;
+import org.apache.rya.api.function.aggregation.SumFunction;
 import org.apache.rya.api.log.LogUtils;
 import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata;
-import org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata.AggregationElement;
-import org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata.AggregationType;
 import org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns;
 import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
-import org.openrdf.model.Literal;
-import org.openrdf.model.Value;
-import org.openrdf.model.datatypes.XMLDatatypeUtil;
-import org.openrdf.model.impl.DecimalLiteralImpl;
-import org.openrdf.model.impl.IntegerLiteralImpl;
-import org.openrdf.query.algebra.MathExpr.MathOp;
-import org.openrdf.query.algebra.evaluation.ValueExprEvaluationException;
-import org.openrdf.query.algebra.evaluation.util.MathUtil;
-import org.openrdf.query.algebra.evaluation.util.ValueComparator;
 import org.openrdf.query.impl.MapBindingSet;
 
 import com.google.common.collect.ImmutableMap;
@@ -154,213 +146,6 @@ public class AggregationResultUpdater extends AbstractNodeUpdater {
     }
 
     /**
-     * A function that updates an {@link AggregationState}.
-     */
-    public static interface AggregationFunction {
-
-        /**
-         * Updates an {@link AggregationState} based on the values of a child Binding Set.
-         *
-         * @param aggregation - Defines which function needs to be performed as well as any details required
-         *   to do the aggregation work. (not null)
-         * @param state - The state that will be updated. (not null)
-         * @param childBindingSet - The Binding Set whose values will be used to update the state.
-         */
-        public void update(AggregationElement aggregation, AggregationState state, VisibilityBindingSet childBindingSet);
-    }
-
-    /**
-     * Increments the {@link AggregationState}'s count if the child Binding Set contains the binding name
-     * that is being counted by the {@link AggregationElement}.
-     */
-    public static final class CountFunction implements AggregationFunction {
-        @Override
-        public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
-            checkArgument(aggregation.getAggregationType() == AggregationType.COUNT, "The CountFunction only accepts COUNT AggregationElements.");
-
-            // Only add one to the count if the child contains the binding that we are counting.
-            final String aggregatedName = aggregation.getAggregatedBindingName();
-            if(childBindingSet.hasBinding(aggregatedName)) {
-                final MapBindingSet result = state.getBindingSet();
-                final String resultName = aggregation.getResultBindingName();
-                final boolean newBinding = !result.hasBinding(resultName);
-
-                if(newBinding) {
-                    // Initialize the binding.
-                    result.addBinding(resultName, new IntegerLiteralImpl(BigInteger.ONE));
-                } else {
-                    // Update the existing binding.
-                    final Literal count = (Literal) result.getValue(resultName);
-                    final BigInteger updatedCount = count.integerValue().add( BigInteger.ONE );
-                    result.addBinding(resultName, new IntegerLiteralImpl(updatedCount));
-                }
-            }
-        }
-    }
-
-    /**
-     * Add to the {@link AggregationState}'s sum if the child Binding Set contains the binding name
-     * that is being summed by the {@link AggregationElement}.
-     */
-    public static final class SumFunction implements AggregationFunction {
-        @Override
-        public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
-            checkArgument(aggregation.getAggregationType() == AggregationType.SUM, "The SumFunction only accepts SUM AggregationElements.");
-
-            // Only add values to the sum if the child contains the binding that we are summing.
-            final String aggregatedName = aggregation.getAggregatedBindingName();
-            if(childBindingSet.hasBinding(aggregatedName)) {
-                final MapBindingSet result = state.getBindingSet();
-                final String resultName = aggregation.getResultBindingName();
-                final boolean newBinding = !result.hasBinding(resultName);
-
-                // Get the starting number for the sum.
-                Literal sum;
-                if(newBinding) {
-                    sum = new IntegerLiteralImpl(BigInteger.ZERO);
-                } else {
-                    sum = (Literal) state.getBindingSet().getValue(resultName);
-                }
-
-                // Add the child binding set's value if it is a numeric literal.
-                final Value childValue = childBindingSet.getValue(aggregatedName);
-                if(childValue instanceof Literal) {
-                    final Literal childLiteral = (Literal) childValue;
-                    if (childLiteral.getDatatype() != null && XMLDatatypeUtil.isNumericDatatype(childLiteral.getDatatype())) {
-                        try {
-                            sum = MathUtil.compute(sum, childLiteral, MathOp.PLUS);
-                        } catch (final ValueExprEvaluationException e) {
-                            log.error("A problem was encountered while updating a Sum Aggregation. This binding set will be ignored: " + childBindingSet);
-                            return;
-                        }
-                    }
-                }
-
-                // Update the state to include the new sum.
-                result.addBinding(resultName, sum);
-            }
-        }
-    }
-
-    /**
-     * Update the {@link AggregationState}'s average if the child Binding Set contains the binding name
-     * that is being averaged by the {@link AggregationElement}.
-     */
-    public static final class AverageFunction implements AggregationFunction {
-        @Override
-        public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
-            checkArgument(aggregation.getAggregationType() == AggregationType.AVERAGE, "The AverageFunction only accepts AVERAGE AggregationElements.");
-
-            // Only update the average if the child contains the binding that we are averaging.
-            final String aggregatedName = aggregation.getAggregatedBindingName();
-            if(childBindingSet.hasBinding(aggregatedName)) {
-                final MapBindingSet result = state.getBindingSet();
-                final String resultName = aggregation.getResultBindingName();
-                final boolean newBinding = !result.hasBinding(resultName);
-
-                // Get the state of the average.
-                final Map<String, AverageState> averageStates = state.getAverageStates();
-                AverageState averageState = newBinding ? new AverageState() : averageStates.get(resultName);
-
-                // Update the state of the average.
-                final Value childValue = childBindingSet.getValue(aggregatedName);
-                if(childValue instanceof Literal) {
-                    final Literal childLiteral = (Literal) childValue;
-                    if (childLiteral.getDatatype() != null && XMLDatatypeUtil.isNumericDatatype(childLiteral.getDatatype())) {
-                        try {
-                            // Update the sum.
-                            final Literal oldSum = new DecimalLiteralImpl(averageState.getSum());
-                            final BigDecimal sum = MathUtil.compute(oldSum, childLiteral, MathOp.PLUS).decimalValue();
-
-                            // Update the count.
-                            final BigInteger count = averageState.getCount().add( BigInteger.ONE );
-
-                            // Update the BindingSet to include the new average.
-                            final Literal sumLiteral = new DecimalLiteralImpl(sum);
-                            final Literal countLiteral = new IntegerLiteralImpl(count);
-                            final Literal average = MathUtil.compute(sumLiteral, countLiteral, MathOp.DIVIDE);
-                            result.addBinding(resultName, average);
-
-                            // Update the average state that is stored.
-                            averageState = new AverageState(sum, count);
-                            averageStates.put(resultName, averageState);
-                        } catch (final ValueExprEvaluationException e) {
-                            log.error("A problem was encountered while updating an Average Aggregation. This binding set will be ignored: " + childBindingSet);
-                            return;
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Update the {@link AggregationState}'s max if the child binding Set contains the binding name that is being
-     * maxed by the {@link AggregationElement}.
-     */
-    public static final class MaxFunction implements AggregationFunction {
-
-        private final ValueComparator compare = new ValueComparator();
-
-        @Override
-        public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
-            checkArgument(aggregation.getAggregationType() == AggregationType.MAX, "The MaxFunction only accepts MAX AggregationElements.");
-
-            // Only update the max if the child contains the binding that we are finding the max value for.
-            final String aggregatedName = aggregation.getAggregatedBindingName();
-            if(childBindingSet.hasBinding(aggregatedName)) {
-                final MapBindingSet result = state.getBindingSet();
-                final String resultName = aggregation.getResultBindingName();
-                final boolean newBinding = !result.hasBinding(resultName);
-
-                Value max;
-                if(newBinding) {
-                    max = childBindingSet.getValue(aggregatedName);
-                } else {
-                    final Value oldMax = result.getValue(resultName);
-                    final Value childMax = childBindingSet.getValue(aggregatedName);
-                    max = compare.compare(childMax, oldMax) > 0 ? childMax : oldMax;
-                }
-
-                result.addBinding(resultName, max);
-            }
-        }
-    }
-
-    /**
-     * Update the {@link AggregationState}'s min if the child binding Set contains the binding name that is being
-     * mined by the {@link AggregationElement}.
-     */
-    public static final class MinFunction implements AggregationFunction {
-
-        private final ValueComparator compare = new ValueComparator();
-
-        @Override
-        public void update(final AggregationElement aggregation, final AggregationState state, final VisibilityBindingSet childBindingSet) {
-            checkArgument(aggregation.getAggregationType() == AggregationType.MIN, "The MinFunction only accepts MIN AggregationElements.");
-
-            // Only update the min if the child contains the binding that we are finding the min value for.
-            final String aggregatedName = aggregation.getAggregatedBindingName();
-            if(childBindingSet.hasBinding(aggregatedName)) {
-                final MapBindingSet result = state.getBindingSet();
-                final String resultName = aggregation.getResultBindingName();
-                final boolean newBinding = !result.hasBinding(resultName);
-
-                Value min;
-                if(newBinding) {
-                    min = childBindingSet.getValue(aggregatedName);
-                } else {
-                    final Value oldMin = result.getValue(resultName);
-                    final Value chidlMin = childBindingSet.getValue(aggregatedName);
-                    min = compare.compare(chidlMin, oldMin) < 0 ? chidlMin : oldMin;
-                }
-
-                result.addBinding(resultName, min);
-            }
-        }
-    }
-
-    /**
      * Reads/Writes instances of {@link AggregationState} to/from bytes.
      */
     public static interface AggregationStateSerDe {
@@ -410,18 +195,18 @@ public class AggregationResultUpdater extends AbstractNodeUpdater {
             // System.out.println("vois.accept(" + className + ".class, ");};};
                         ) {
                 // These classes are allowed to be deserialized. Others throw InvalidClassException.
-                vois.accept(org.apache.rya.indexing.pcj.fluo.app.AggregationResultUpdater.AggregationState.class, //
-                                org.apache.rya.indexing.pcj.fluo.app.AggregationResultUpdater.AverageState.class, //
-                                java.util.HashMap.class, //
-                                java.math.BigInteger.class, //
-                                java.lang.Number.class, //
-                                java.math.BigDecimal.class, //
-                                org.openrdf.query.impl.MapBindingSet.class, //
-                                java.util.LinkedHashMap.class, //
-                                org.openrdf.query.impl.BindingImpl.class, //
-                                org.openrdf.model.impl.URIImpl.class, //
-                                org.openrdf.model.impl.LiteralImpl.class, //
-                                org.openrdf.model.impl.DecimalLiteralImpl.class, //
+                vois.accept(AggregationState.class,
+                                AverageState.class,
+                                java.util.HashMap.class,
+                                java.math.BigInteger.class,
+                                java.lang.Number.class,
+                                java.math.BigDecimal.class,
+                                org.openrdf.query.impl.MapBindingSet.class,
+                                java.util.LinkedHashMap.class,
+                                org.openrdf.query.impl.BindingImpl.class,
+                                org.openrdf.model.impl.URIImpl.class,
+                                org.openrdf.model.impl.LiteralImpl.class,
+                                org.openrdf.model.impl.DecimalLiteralImpl.class,
                                 org.openrdf.model.impl.IntegerLiteralImpl.class);
                 vois.accept("[B"); // Array of Bytes
                 final Object o = vois.readObject();
@@ -437,155 +222,4 @@ public class AggregationResultUpdater extends AbstractNodeUpdater {
             return state;
         }
     }
-
-    /**
-     * Keeps track information required to update and build the resulting Binding Set for a set of Group By values.
-     */
-    public static final class AggregationState implements Serializable {
-        private static final long serialVersionUID = 1L;
-
-        // The visibility equation that encompasses all data the aggregation state is derived from.
-        private String visibility;
-
-        // A binding set that holds the current state of the aggregations.
-        private final MapBindingSet bindingSet;
-
-        // A map from result binding name to the state that derived that binding's value.
-        private final Map<String, AverageState> avgStates;
-
-        /**
-         * Constructs an instance of {@link AggregationState}.
-         */
-        public AggregationState() {
-            this.visibility = "";
-            this.bindingSet = new MapBindingSet();
-            this.avgStates = new HashMap<>();
-        }
-
-        /**
-         * Constructs an instance of {@link AggregationState}.
-         *
-         * @param visibility - The visibility equation associated with the resulting binding set. (not null)
-         * @param bindingSet - The Binding Set whose values are being updated. It holds the result for a set of
-         *   Group By values. (not null)
-         * @param avgStates - If the aggregation is doing an Average, this is a map from result binding name to
-         *   average state for that binding.
-         */
-        public AggregationState(
-                final String visibility,
-                final MapBindingSet bindingSet,
-                final Map<String, AverageState> avgStates) {
-            this.visibility = requireNonNull(visibility);
-            this.bindingSet = requireNonNull(bindingSet);
-            this.avgStates = requireNonNull(avgStates);
-        }
-
-        /**
-         * @return The visibility equation associated with the resulting binding set.
-         */
-        public String getVisibility() {
-            return visibility;
-        }
-
-        /**
-         * @param visibility - The visibility equation associated with the resulting binding set.
-         */
-        public void setVisibility(final String visibility) {
-            this.visibility = requireNonNull(visibility);
-        }
-
-        /**
-         * @return The Binding Set whose values are being updated. It holds the result for a set of Group By values.
-         */
-        public MapBindingSet getBindingSet() {
-            return bindingSet;
-        }
-
-        /**
-         * @return If the aggregation is doing an Average, this is a map from result binding name to
-         *   average state for that binding.
-         */
-        public Map<String, AverageState> getAverageStates() {
-            return avgStates;
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(visibility, bindingSet, avgStates);
-        }
-
-        @Override
-        public boolean equals(final Object o) {
-            if(o instanceof AggregationState) {
-                final AggregationState state = (AggregationState) o;
-                return Objects.equals(visibility, state.visibility) &&
-                        Objects.equals(bindingSet, state.bindingSet) &&
-                        Objects.equals(avgStates, state.avgStates);
-            }
-            return false;
-        }
-    }
-
-    /**
-     * The Sum and Count of the values that are being averaged. The average itself is derived from these values.
-     */
-    public static class AverageState implements Serializable {
-        private static final long serialVersionUID = 1L;
-
-        private final BigDecimal sum;
-        private final BigInteger count;
-
-        /**
-         * Constructs an instance of {@link AverageState} where the count and sum start at 0.
-         */
-        public AverageState() {
-            sum = BigDecimal.ZERO;
-            count = BigInteger.ZERO;
-        }
-
-        /**
-         * Constructs an instance of {@link AverageState}.
-         *
-         * @param sum - The sum of the values that are averaged. (not null)
-         * @param count - The number of values that are averaged. (not null)
-         */
-        public AverageState(final BigDecimal sum, final BigInteger count) {
-            this.sum = requireNonNull(sum);
-            this.count = requireNonNull(count);
-        }
-
-        /**
-         * @return The sum of the values that are averaged.
-         */
-        public BigDecimal getSum() {
-            return sum;
-        }
-
-        /**
-         * @return The number of values that are averaged.
-         */
-        public BigInteger getCount() {
-            return count;
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(sum, count);
-        }
-
-        @Override
-        public boolean equals(final Object o) {
-            if(o instanceof AverageState) {
-                final AverageState state = (AverageState) o;
-                return Objects.equals(sum, state.sum) &&
-                        Objects.equals(count, state.count);
-            }
-            return false;
-        }
-
-        @Override
-        public String toString() {
-            return "Sum: " + sum + " Count: " + count;
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/AggregationObserver.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/AggregationObserver.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/AggregationObserver.java
index 0271519..e806b15 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/AggregationObserver.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/observers/AggregationObserver.java
@@ -23,10 +23,10 @@ import static org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants.AG
 
 import org.apache.fluo.api.client.TransactionBase;
 import org.apache.fluo.api.data.Bytes;
-import org.apache.rya.indexing.pcj.fluo.app.AggregationResultUpdater.AggregationState;
+import org.apache.rya.api.function.aggregation.AggregationState;
+import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.indexing.pcj.fluo.app.AggregationResultUpdater.AggregationStateSerDe;
 import org.apache.rya.indexing.pcj.fluo.app.AggregationResultUpdater.ObjectSerializationAggregationStateSerDe;
-import org.apache.rya.api.model.VisibilityBindingSet;
 import org.apache.rya.indexing.pcj.fluo.app.BindingSetRow;
 import org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata;
 import org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryColumns;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/AggregationMetadata.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/AggregationMetadata.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/AggregationMetadata.java
index eaa072f..a839645 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/AggregationMetadata.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/AggregationMetadata.java
@@ -20,23 +20,14 @@ package org.apache.rya.indexing.pcj.fluo.app.query;
 
 import static java.util.Objects.requireNonNull;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
-import java.util.Optional;
 
+import org.apache.rya.api.function.aggregation.AggregationElement;
 import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
-import org.openrdf.query.algebra.AggregateOperator;
-import org.openrdf.query.algebra.Avg;
-import org.openrdf.query.algebra.Count;
-import org.openrdf.query.algebra.Max;
-import org.openrdf.query.algebra.Min;
-import org.openrdf.query.algebra.Sum;
-
-import com.google.common.collect.ImmutableMap;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
@@ -50,115 +41,6 @@ import net.jcip.annotations.Immutable;
 @DefaultAnnotation(NonNull.class)
 public class AggregationMetadata extends CommonNodeMetadata {
 
-    /**
-     * The different types of Aggregation functions that an aggregate node may perform.
-     */
-    public static enum AggregationType {
-        MIN(Min.class),
-        MAX(Max.class),
-        COUNT(Count.class),
-        SUM(Sum.class),
-        AVERAGE(Avg.class);
-
-        private final Class<? extends AggregateOperator> operatorClass;
-
-        private AggregationType(final Class<? extends AggregateOperator> operatorClass) {
-            this.operatorClass = requireNonNull(operatorClass);
-        }
-
-        private static final ImmutableMap<Class<? extends AggregateOperator>, AggregationType> byOperatorClass;
-        static {
-            final ImmutableMap.Builder<Class<? extends AggregateOperator>, AggregationType> builder = ImmutableMap.builder();
-            for(final AggregationType type : AggregationType.values()) {
-                builder.put(type.operatorClass, type);
-            }
-            byOperatorClass = builder.build();
-        }
-
-        public static Optional<AggregationType> byOperatorClass(final Class<? extends AggregateOperator> operatorClass) {
-            return Optional.ofNullable( byOperatorClass.get(operatorClass) );
-        }
-    }
-
-    /**
-     * Represents all of the metadata require to perform an Aggregation that is part of a SPARQL query.
-     * </p>
-     * For example, if you have the following in SPARQL:
-     * <pre>
-     * SELECT (avg(?price) as ?avgPrice) {
-     *     ...
-     * }
-     * </pre>
-     * You would construct an instance of this object like so:
-     * <pre>
-     * new AggregationElement(AggregationType.AVERAGE, "price", "avgPrice");
-     * </pre>
-     */
-    @Immutable
-    @DefaultAnnotation(NonNull.class)
-    public static final class AggregationElement implements Serializable {
-        private static final long serialVersionUID = 1L;
-
-        private final AggregationType aggregationType;
-        private final String aggregatedBindingName;
-        private final String resultBindingName;
-
-        /**
-         * Constructs an instance of {@link AggregationElement}.
-         *
-         * @param aggregationType - Defines how the binding values will be aggregated. (not null)
-         * @param aggregatedBindingName - The name of the binding whose values is aggregated. This binding must
-         *   appear within the child node's emitted binding sets. (not null)
-         * @param resultBindingName - The name of the binding this aggregation's results are written to. This binding
-         *   must appeared within the AggregationMetadata's variable order. (not null)
-         */
-        public AggregationElement(
-                final AggregationType aggregationType,
-                final String aggregatedBindingName,
-                final String resultBindingName) {
-            this.aggregationType = requireNonNull(aggregationType);
-            this.aggregatedBindingName = requireNonNull(aggregatedBindingName);
-            this.resultBindingName = requireNonNull(resultBindingName);
-        }
-
-        /**
-         * @return Defines how the binding values will be aggregated.
-         */
-        public AggregationType getAggregationType() {
-            return aggregationType;
-        }
-
-        /**
-         * @return The name of the binding whose values is aggregated. This binding must appear within the child node's emitted binding sets.
-         */
-        public String getAggregatedBindingName() {
-            return aggregatedBindingName;
-        }
-
-        /**
-         * @return The name of the binding this aggregation's results are written to. This binding must appeared within the AggregationMetadata's variable order.
-         */
-        public String getResultBindingName() {
-            return resultBindingName;
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(aggregationType, aggregatedBindingName, resultBindingName);
-        }
-
-        @Override
-        public boolean equals(final Object o ) {
-            if(o instanceof AggregationElement) {
-                final AggregationElement agg = (AggregationElement) o;
-                return Objects.equals(aggregationType, agg.aggregationType) &&
-                        Objects.equals(aggregatedBindingName, agg.aggregatedBindingName) &&
-                        Objects.equals(resultBindingName, agg.resultBindingName);
-            }
-            return false;
-        }
-    }
-
     private final String parentNodeId;
     private final String childNodeId;
     private final Collection<AggregationElement> aggregations;
@@ -308,6 +190,7 @@ public class AggregationMetadata extends CommonNodeMetadata {
         /**
          * @return This node's Node ID.
          */
+        @Override
         public String getNodeId() {
             return nodeId;
         }
@@ -321,10 +204,11 @@ public class AggregationMetadata extends CommonNodeMetadata {
             this.varOrder = varOrder;
             return this;
         }
-        
+
         /**
          * @return the variable order of binding sets that are emitted by this node.
          */
+        @Override
         public VariableOrder getVariableOrder() {
             return varOrder;
         }
@@ -337,7 +221,7 @@ public class AggregationMetadata extends CommonNodeMetadata {
             this.parentNodeId = parentNodeId;
             return this;
         }
-       
+
         public String getParentNodeId() {
             return parentNodeId;
         }
@@ -350,7 +234,7 @@ public class AggregationMetadata extends CommonNodeMetadata {
             this.childNodeId = childNodeId;
             return this;
         }
-        
+
         public String getChildNodeId() {
             return childNodeId;
         }
@@ -375,7 +259,7 @@ public class AggregationMetadata extends CommonNodeMetadata {
             this.groupByVariables = groupByVariables;
             return this;
         }
-        
+
         /**
          * @return variable order that defines how data is grouped for the aggregation function
          */

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryColumns.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryColumns.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryColumns.java
index 9d1c4fc..e043671 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryColumns.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryColumns.java
@@ -25,8 +25,8 @@ import java.util.List;
 
 import org.apache.fluo.api.data.Column;
 import org.apache.rya.api.client.CreatePCJ.QueryType;
+import org.apache.rya.api.function.aggregation.AggregationState;
 import org.apache.rya.api.model.VisibilityBindingSet;
-import org.apache.rya.indexing.pcj.fluo.app.AggregationResultUpdater.AggregationState;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/8363724b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAO.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAO.java b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAO.java
index ba75a56..c193ef7 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAO.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.app/src/main/java/org/apache/rya/indexing/pcj/fluo/app/query/FluoQueryMetadataDAO.java
@@ -37,11 +37,12 @@ import org.apache.fluo.api.data.Bytes;
 import org.apache.fluo.api.data.Column;
 import org.apache.rya.api.client.CreatePCJ.ExportStrategy;
 import org.apache.rya.api.client.CreatePCJ.QueryType;
+import org.apache.rya.api.function.aggregation.AggregationElement;
+import org.apache.rya.api.function.aggregation.AggregationType;
 import org.apache.rya.indexing.pcj.fluo.app.ConstructGraph;
 import org.apache.rya.indexing.pcj.fluo.app.ConstructGraphSerializer;
 import org.apache.rya.indexing.pcj.fluo.app.IncrementalUpdateConstants;
 import org.apache.rya.indexing.pcj.fluo.app.NodeType;
-import org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata.AggregationElement;
 import org.apache.rya.indexing.pcj.fluo.app.query.JoinMetadata.JoinType;
 import org.apache.rya.indexing.pcj.storage.accumulo.VariableOrder;
 
@@ -567,10 +568,10 @@ public class FluoQueryMetadataDAO {
         // System.out.println("vois.accept(" + className + ".class, ");};};
         ) {
             // These classes are allowed to be deserialized. Others throw InvalidClassException.
-            vois.accept(java.util.ArrayList.class, //
-                            java.lang.Enum.class, //
-                            org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata.AggregationElement.class, //
-                            org.apache.rya.indexing.pcj.fluo.app.query.AggregationMetadata.AggregationType.class);
+            vois.accept(java.util.ArrayList.class,
+                            java.lang.Enum.class,
+                            AggregationElement.class,
+                            AggregationType.class);
             final Object object = vois.readObject();
             if (!(object instanceof Collection<?>)) {
                 throw new InvalidClassException("Object read was not of type Collection. It was: " + object.getClass());


[33/50] [abbrv] incubator-rya git commit: RYA-377 Fixed the project layout so that it builds even when the geoindexing profile is not enabled.

Posted by ca...@apache.org.
RYA-377 Fixed the project layout so that it builds even when the geoindexing profile is not enabled.


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

Branch: refs/heads/master
Commit: 92c85ee11030712289df48faed4710359b1b0601
Parents: 923448f
Author: kchilton2 <ke...@gmail.com>
Authored: Wed Jan 3 16:41:08 2018 -0500
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:01 2018 -0500

----------------------------------------------------------------------
 extras/rya.streams/geo/pom.xml                  |  71 +++
 .../kafka/processors/filter/GeoFilterIT.java    | 137 ++++++
 extras/rya.streams/integration/pom.xml          |  69 +++
 .../aggregation/AggregationProcessorIT.java     | 457 +++++++++++++++++++
 .../processors/filter/FilterProcessorIT.java    |  86 ++++
 .../processors/filter/FilterProcessorTest.java  |  75 +++
 .../processors/filter/TemporalFilterIT.java     | 231 ++++++++++
 .../kafka/processors/join/JoinProcessorIT.java  | 316 +++++++++++++
 .../projection/MultiProjectionProcessorIT.java  |  93 ++++
 .../projection/ProjectionProcessorIT.java       |  86 ++++
 .../sp/StatementPatternProcessorIT.java         | 196 ++++++++
 extras/rya.streams/kafka-test/pom.xml           |  51 +++
 .../rya/streams/kafka/RyaStreamsTestUtil.java   | 122 +++++
 extras/rya.streams/kafka/pom.xml                |  32 +-
 .../apache/rya/streams/kafka/RdfTestUtil.java   | 131 ------
 .../rya/streams/kafka/RyaStreamsTestUtil.java   | 122 -----
 .../processors/StatementPatternProcessorIT.java | 195 --------
 .../aggregation/AggregationProcessorIT.java     | 457 -------------------
 .../processors/filter/FilterProcessorIT.java    |  86 ----
 .../processors/filter/FilterProcessorTest.java  |  75 ---
 .../kafka/processors/filter/GeoFilterIT.java    | 137 ------
 .../processors/filter/TemporalFilterIT.java     | 231 ----------
 .../kafka/processors/join/JoinProcessorIT.java  | 316 -------------
 .../projection/MultiProjectionProcessorIT.java  |  92 ----
 .../projection/ProjectionProcessorIT.java       |  85 ----
 extras/rya.streams/pom.xml                      |   3 +
 pom.xml                                         |  10 +
 test/pom.xml                                    |   1 +
 test/rdf/pom.xml                                |  59 +++
 .../apache/rya/streams/kafka/RdfTestUtil.java   | 131 ++++++
 30 files changed, 2212 insertions(+), 1941 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/geo/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/geo/pom.xml b/extras/rya.streams/geo/pom.xml
new file mode 100644
index 0000000..2f179d0
--- /dev/null
+++ b/extras/rya.streams/geo/pom.xml
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+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">
+
+    <parent>
+        <groupId>org.apache.rya</groupId>
+        <artifactId>rya.streams.parent</artifactId>
+        <version>3.2.12-incubating-SNAPSHOT</version>
+    </parent>
+    
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>rya.streams.geo-test</artifactId>
+    
+    <name>Apache Rya Streams Geo Test</name>
+    <description>
+        A module that contains Geo integration tests with Rya Streams.
+    </description>
+
+    <dependencies>
+        <!-- Rya dependencies -->
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.pcj.functions.geo</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.geo.common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <!-- Test dependencies -->
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.test.kafka</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.streams.kafka-test</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/geo/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/geo/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java b/extras/rya.streams/geo/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
new file mode 100644
index 0000000..c090afa
--- /dev/null
+++ b/extras/rya.streams/geo/src/test/java/org/apache/rya/streams/kafka/processors/filter/GeoFilterIT.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.filter;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.indexing.GeoConstants;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.evaluation.function.Function;
+import org.openrdf.query.algebra.evaluation.function.FunctionRegistry;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.Geometry;
+import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.io.WKTWriter;
+
+/**
+ * Integration tests the geo methods of {@link FilterProcessor}.
+ */
+public class GeoFilterIT {
+    private static final String GEO = "http://www.opengis.net/def/function/geosparql/";
+    private static final GeometryFactory GF = new GeometryFactory();
+    private static final Geometry ZERO = GF.createPoint(new Coordinate(0, 0));
+    private static final Geometry ONE = GF.createPoint(new Coordinate(1, 1));
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void showGeoFunctionsRegistered() {
+        int count = 0;
+        final Collection<Function> funcs = FunctionRegistry.getInstance().getAll();
+        for (final Function fun : funcs) {
+            if (fun.getURI().startsWith(GEO)) {
+                count++;
+            }
+        }
+
+        // There are 30 geo functions registered, ensure that there are 30.
+        assertEquals(30, count);
+    }
+
+    @Test
+    public void showProcessorWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                        + "PREFIX geof: <" + GEO + ">\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:event1> geo:asWKT ?point .\n"
+                        + " FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+                        + "}";
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        final WKTWriter w = new WKTWriter();
+        bs.addBinding("point", vf.createLiteral(w.write(ZERO), GeoConstants.XMLSCHEMA_OGC_WKT));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    private List<VisibilityStatement> getStatements() throws Exception {
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        // geo 2x2 points
+        statements.add(new VisibilityStatement(statement(ZERO), "a"));
+        statements.add(new VisibilityStatement(statement(ONE), "a"));
+        return statements;
+    }
+
+    private static Statement statement(final Geometry geo) {
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Resource subject = vf.createURI("urn:event1");
+        final URI predicate = GeoConstants.GEO_AS_WKT;
+        final WKTWriter w = new WKTWriter();
+        final Value object = vf.createLiteral(w.write(geo), GeoConstants.XMLSCHEMA_OGC_WKT);
+        return new StatementImpl(subject, predicate, object);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/integration/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/pom.xml b/extras/rya.streams/integration/pom.xml
new file mode 100644
index 0000000..26ec9f7
--- /dev/null
+++ b/extras/rya.streams/integration/pom.xml
@@ -0,0 +1,69 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+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">
+
+    <parent>
+        <groupId>org.apache.rya</groupId>
+        <artifactId>rya.streams.parent</artifactId>
+        <version>3.2.12-incubating-SNAPSHOT</version>
+    </parent>
+    
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>rya.streams.integration</artifactId>
+    
+    <name>Apache Rya Streams Kafka Integration Tests</name>
+    <description>
+        A module that contains Kafka Integration tests for Rya Streams.
+    </description>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.streams.kafka</artifactId>
+        </dependency>
+        
+        <!-- Test dependencies -->
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.test.rdf</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.test.kafka</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>rya.streams.kafka-test</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
new file mode 100644
index 0000000..072469a
--- /dev/null
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/aggregation/AggregationProcessorIT.java
@@ -0,0 +1,457 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.aggregation;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.aggregation.AggregationProcessorSupplier.AggregationProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.XMLSchema;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Integration tests {@link AggregationProcessor}.
+ */
+public class AggregationProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(false);
+
+    @Test
+    public void count() throws Exception {
+        // A query that figures out how many books each person has.
+        final String sparql =
+                "SELECT ?person (count(?book) as ?bookCount) " +
+                "WHERE { " +
+                    "?person <urn:hasBook> ?book " +
+                "} GROUP BY ?person";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasBook"), vf.createLiteral("Book 1")), "a"));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasBook"), vf.createLiteral("Book 1")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasBook"), vf.createLiteral("Book 2")), "b"));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("bookCount", vf.createLiteral("1", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, "a"));
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("bookCount", vf.createLiteral("1", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("bookCount", vf.createLiteral("2", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, "a&b"));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void sum() throws Exception {
+        // A query that figures out how much food each person has.
+        final String sparql =
+                "SELECT ?person (sum(?foodCount) as ?totalFood) " +
+                "WHERE { " +
+                    "?person <urn:hasFoodType> ?food . " +
+                    "?food <urn:count> ?foodCount . " +
+                "} GROUP BY ?person";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasFoodType"), vf.createURI("urn:corn")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasFoodType"), vf.createURI("urn:apple")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:corn"), vf.createURI("urn:count"), vf.createLiteral(4)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:apple"), vf.createURI("urn:count"), vf.createLiteral(3)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("totalFood", vf.createLiteral("4", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("totalFood", vf.createLiteral("7", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void average() throws Exception {
+        // A query that figures out the average age across all people.
+        final String sparql =
+                "SELECT (avg(?age) as ?avgAge) " +
+                "WHERE { " +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(3)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(2)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("avgAge", vf.createLiteral("3", XMLSchema.DECIMAL));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("avgAge", vf.createLiteral("5", XMLSchema.DECIMAL));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("avgAge", vf.createLiteral("4", XMLSchema.DECIMAL));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void min() throws Exception {
+        // A query that figures out what the youngest age is across all people.
+        final String sparql =
+                "SELECT (min(?age) as ?youngest) " +
+                "WHERE { " +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(13)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(14)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:age"), vf.createLiteral(5)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:age"), vf.createLiteral(25)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(13));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(7));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(5));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void max() throws Exception {
+        // A query that figures out what the oldest age is across all people.
+        final String sparql =
+                "SELECT (max(?age) as ?oldest) " +
+                "WHERE { " +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(13)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(14)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:age"), vf.createLiteral(5)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:age"), vf.createLiteral(25)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("oldest", vf.createLiteral(13));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("oldest", vf.createLiteral(14));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("oldest", vf.createLiteral(25));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void multipleGroupByVars() throws Exception {
+        // A query that contains more than one group by variable.
+        final String sparql =
+                "SELECT ?business ?employee (sum(?hours) AS ?totalHours) " +
+                "WHERE {" +
+                    "?employee <urn:worksAt> ?business . " +
+                    "?business <urn:hasTimecardId> ?timecardId . " +
+                    "?employee <urn:hasTimecardId> ?timecardId . " +
+                    "?timecardId <urn:hours> ?hours . " +
+                "} GROUP BY ?business ?employee";
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoJoint")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard1")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard1")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard1"), vf.createURI("urn:hours"), vf.createLiteral(40)), ""));
+
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard2")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard2")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard2"), vf.createURI("urn:hours"), vf.createLiteral(25)), ""));
+
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoJoint")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard3")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard3")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard3"), vf.createURI("urn:hours"), vf.createLiteral(28)), ""));
+
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:TacoJoint"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard4")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard4")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard4"), vf.createURI("urn:hours"), vf.createLiteral(28)), ""));
+
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:CoffeeShop"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard5")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasTimecardId"), vf.createURI("urn:timecard5")), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:timecard5"), vf.createURI("urn:hours"), vf.createLiteral(12)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        bs.addBinding("employee", vf.createURI("urn:Alice"));
+        bs.addBinding("totalHours", vf.createLiteral("40", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        bs.addBinding("employee", vf.createURI("urn:Alice"));
+        bs.addBinding("totalHours", vf.createLiteral("65", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("totalHours", vf.createLiteral("28", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:TacoJoint"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("totalHours", vf.createLiteral("56", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("business", vf.createURI("urn:CoffeeShop"));
+        bs.addBinding("employee", vf.createURI("urn:Alice"));
+        bs.addBinding("totalHours", vf.createLiteral("12", XMLSchema.INTEGER));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void multipleAggregations() throws Exception {
+        // A query that figures out what the youngest and oldest ages are across all people.
+        final String sparql =
+                "SELECT (min(?age) as ?youngest) (max(?age) as ?oldest) " +
+                "WHERE { " +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Create the statements that will be input into the query..
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(13)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(14)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:age"), vf.createLiteral(7)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:age"), vf.createLiteral(5)), ""));
+        statements.add(new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:age"), vf.createLiteral(25)), ""));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(13));
+        bs.addBinding("oldest", vf.createLiteral(13));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(13));
+        bs.addBinding("oldest", vf.createLiteral(14));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(7));
+        bs.addBinding("oldest", vf.createLiteral(14));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(5));
+        bs.addBinding("oldest", vf.createLiteral(14));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        bs = new MapBindingSet();
+        bs.addBinding("youngest", vf.createLiteral(5));
+        bs.addBinding("oldest", vf.createLiteral(25));
+        expected.add(new VisibilityBindingSet(bs, ""));
+
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
new file mode 100644
index 0000000..aaa67ea
--- /dev/null
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorIT.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.filter;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Integration tests the methods of {@link FilterProcessor}.
+ */
+public class FilterProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void showProcessorWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "SELECT * " +
+                "WHERE { " +
+                    "FILTER(?age < 10)" +
+                    "?person <urn:age> ?age " +
+                "}";
+
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:age"), vf.createLiteral(11)), "a"));
+        statements.add(new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:age"), vf.createLiteral(9)), "a"));
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("age", vf.createLiteral(9));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
new file mode 100644
index 0000000..3ff8e8d
--- /dev/null
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/FilterProcessorTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.filter;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.rya.api.function.filter.FilterEvaluator;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.kafka.RdfTestUtil;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.Filter;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Unit tests the methods of {@link FilterProcessor}.
+ */
+public class FilterProcessorTest {
+
+    @Test
+    public void showFilterFunctionIsCalled() throws Exception {
+        // Read the filter object from a SPARQL query.
+        final Filter filter = RdfTestUtil.getFilter(
+                "SELECT * " +
+                "WHERE { " +
+                    "FILTER(?age < 10)" +
+                    "?person <urn:age> ?age " +
+                "}");
+
+        // Create a Binding Set that will be passed into the Filter function based on the where clause.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("age", vf.createLiteral(9));
+        final VisibilityBindingSet inputVisBs = new VisibilityBindingSet(bs, "a");
+
+        // Mock the processor context that will be invoked.
+        final ProcessorContext context = mock(ProcessorContext.class);
+
+        // Run the test.
+        final FilterProcessor processor = new FilterProcessor(
+                FilterEvaluator.make(filter),
+                result -> ProcessorResult.make(new UnaryResult(result)));
+        processor.init(context);
+        processor.process("key", ProcessorResult.make(new UnaryResult(inputVisBs)));
+
+        // Verify the binding set was passed through.
+        verify(context, times(1)).forward(eq("key"), eq(ProcessorResult.make(new UnaryResult(inputVisBs))));
+
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
new file mode 100644
index 0000000..22a883b
--- /dev/null
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/filter/TemporalFilterIT.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.filter;
+
+import static org.junit.Assert.assertEquals;
+
+import java.time.ZonedDateTime;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.filter.FilterProcessorSupplier.FilterProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.algebra.evaluation.function.Function;
+import org.openrdf.query.algebra.evaluation.function.FunctionRegistry;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Integration tests the temporal methods of {@link FilterProcessor}.
+ */
+public class TemporalFilterIT {
+    private static final ValueFactory vf = new ValueFactoryImpl();
+    private static final String TEMPORAL = "http://rya.apache.org/ns/temporal";
+    private static final ZonedDateTime TIME = ZonedDateTime.parse("2015-12-30T12:00:00Z");
+    private static final ZonedDateTime TIME_10 = ZonedDateTime.parse("2015-12-30T12:00:10Z");
+    private static final ZonedDateTime TIME_20 = ZonedDateTime.parse("2015-12-30T12:00:20Z");
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(false);
+
+    @Test
+    public void temporalFunctionsRegistered() {
+        int count = 0;
+        final Collection<Function> funcs = FunctionRegistry.getInstance().getAll();
+        for (final Function fun : funcs) {
+            if (fun.getURI().startsWith(TEMPORAL)) {
+                count++;
+            }
+        }
+
+        // There are 4 temporal functions registered, ensure that there are 4.
+        assertEquals(4, count);
+    }
+
+    @Test
+    public void showEqualsWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX time: <http://www.w3.org/2006/time/> \n"
+                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:time> time:atTime ?date .\n"
+                        + " FILTER(tempf:equals(?date, \"" + TIME.toString() + "\")) "
+                        + "}";
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("date", vf.createLiteral(TIME.toString()));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void showBeforeWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX time: <http://www.w3.org/2006/time/> \n"
+                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:time> time:atTime ?date .\n"
+                        + " FILTER(tempf:before(?date, \"" + TIME_10.toString() + "\")) "
+                        + "}";
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("date", vf.createLiteral(TIME.toString()));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void showAfterWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX time: <http://www.w3.org/2006/time/> \n"
+                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:time> time:atTime ?date .\n"
+                        + " FILTER(tempf:after(?date, \"" + TIME_10.toString() + "\")) "
+                        + "}";
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("date", vf.createLiteral(TIME_20.toString()));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void showWithinWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Get the RDF model objects that will be used to build the query.
+        final String sparql =
+                "PREFIX time: <http://www.w3.org/2006/time/> \n"
+                        + "PREFIX tempf: <http://rya.apache.org/ns/temporal/>\n"
+                        + "SELECT * \n"
+                        + "WHERE { \n"
+                        + "  <urn:time> time:atTime ?date .\n"
+                        + " FILTER(tempf:within(?date, \"" + TIME.toString() + "/" + TIME_20.toString() + "\")) "
+                        + "}";
+        // Setup a topology.
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = getStatements();
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("date", vf.createLiteral(TIME_10.toString()));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    private List<VisibilityStatement> getStatements() throws Exception {
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add(new VisibilityStatement(statement(TIME), "a"));
+        statements.add(new VisibilityStatement(statement(TIME_10), "a"));
+        statements.add(new VisibilityStatement(statement(TIME_20), "a"));
+        return statements;
+    }
+
+    private static Statement statement(final ZonedDateTime time) {
+        final Resource subject = vf.createURI("urn:time");
+        final URI predicate = vf.createURI("http://www.w3.org/2006/time/atTime");
+        final Value object = vf.createLiteral(time.toString());
+        return new StatementImpl(subject, predicate, object);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
new file mode 100644
index 0000000..bdb9be6
--- /dev/null
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/join/JoinProcessorIT.java
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.join;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.join.NaturalJoin;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.ProcessorResult;
+import org.apache.rya.streams.kafka.processors.ProcessorResult.UnaryResult;
+import org.apache.rya.streams.kafka.processors.join.JoinProcessorSupplier.JoinProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Integration tests the methods of {@link JoinProcessor}.
+ */
+public class JoinProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test(expected = IllegalArgumentException.class)
+    public void badAllVars() throws IllegalArgumentException {
+        new JoinProcessorSupplier(
+                "NATURAL_JOIN",
+                new NaturalJoin(),
+                Lists.newArrayList("employee"),
+                Lists.newArrayList("person", "employee", "business"),
+                result -> ProcessorResult.make( new UnaryResult(result) ));
+    }
+
+    @Test
+    public void newLeftResult() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "?employee <urn:worksAt> ?business" +
+                " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create some statements that generate a bunch of right SP results.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "a&b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), "b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "b|c") );
+
+        // Add a statement that will generate a left result that joins with some of those right results.
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "c") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
+        expected.add( new VisibilityBindingSet(bs, "a&b&c") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void newRightResult() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "?employee <urn:worksAt> ?business" +
+                " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create some statements that generate a bunch of right SP results.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "a&b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), "b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "b|c") );
+
+        // Add a statement that will generate a left result that joins with some of those right results.
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "c") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
+        expected.add( new VisibilityBindingSet(bs, "a&b&c") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void newResultsBothSides() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "?employee <urn:worksAt> ?business" +
+                " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create some statements that generate a bunch of right SP results.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "a&b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "c") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:worksAt"), vf.createURI("urn:CoffeeShop")), "b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "b|c") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:talksTo"), vf.createURI("urn:Charlie")), "c") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
+        expected.add( new VisibilityBindingSet(bs, "a&b&c") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add( new VisibilityBindingSet(bs, "c&(b|c)") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("employee", vf.createURI("urn:Charlie"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        expected.add( new VisibilityBindingSet(bs, "a&c") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void manyJoins() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "?employee <urn:worksAt> ?business ." +
+                    "?employee <urn:hourlyWage> ?wage ." +
+                " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create some statements that generate a bunch of right SP results.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hourlyWage"), vf.createLiteral(7.25)), "a") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        final MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:BurgerJoint"));
+        bs.addBinding("wage", vf.createLiteral(7.25));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+
+    @Test
+    public void leftJoin() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Setup a topology.
+        final String query =
+                "SELECT * WHERE { " +
+                    "?person <urn:talksTo> ?employee ." +
+                    "OPTIONAL{ ?employee <urn:worksAt> ?business } " +
+                " }";
+        final TopologyFactory factory = new TopologyFactory();
+        final TopologyBuilder builder = factory.build(query, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Create some statements that generate a result that includes the optional value as well as one that does not.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:worksAt"), vf.createURI("urn:TacoPlace")), "b") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:talksTo"), vf.createURI("urn:Charlie")), "c") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:worksAt"), vf.createURI("urn:BurgerJoint")), "d") );
+
+        // Make the expected results.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        expected.add( new VisibilityBindingSet(bs, "a") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Alice"));
+        bs.addBinding("employee", vf.createURI("urn:Bob"));
+        bs.addBinding("business", vf.createURI("urn:TacoPlace"));
+        expected.add( new VisibilityBindingSet(bs, "a&b") );
+
+        bs = new MapBindingSet();
+        bs.addBinding("person", vf.createURI("urn:Bob"));
+        bs.addBinding("employee", vf.createURI("urn:Charlie"));
+        expected.add( new VisibilityBindingSet(bs, "c") );
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityBindingSetDeserializer.class);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
new file mode 100644
index 0000000..a560294
--- /dev/null
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/projection/MultiProjectionProcessorIT.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.projection;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.projection.MultiProjectionProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.projection.MultiProjectionProcessorSupplier.MultiProjectionProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityStatementDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.BNode;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.RDF;
+
+/**
+ * Integration tests the methods of {@link MultiProjectionProcessor}.
+ */
+public class MultiProjectionProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void showProcessorWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Create a topology for the Query that will be tested.
+        final String sparql =
+                "CONSTRUCT {" +
+                    "_:b a <urn:movementObservation> ; " +
+                    "<urn:location> ?location ; " +
+                    "<urn:direction> ?direction ; " +
+                "}" +
+                "WHERE {" +
+                    "?thing <urn:corner> ?location ." +
+                    "?thing <urn:compass> ?direction." +
+                "}";
+
+        final String bNodeId = UUID.randomUUID().toString();
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, () -> bNodeId);
+
+        // Create the statements that will be input into the query.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:car1"), vf.createURI("urn:compass"), vf.createURI("urn:NW")), "a") );
+        statements.add( new VisibilityStatement(
+                vf.createStatement(vf.createURI("urn:car1"), vf.createURI("urn:corner"), vf.createURI("urn:corner1")), "a") );
+
+        // Make the expected results.
+        final Set<VisibilityStatement> expected = new HashSet<>();
+        final BNode blankNode = vf.createBNode(bNodeId);
+
+        expected.add(new VisibilityStatement(vf.createStatement(blankNode, RDF.TYPE, vf.createURI("urn:movementObservation")), "a"));
+        expected.add(new VisibilityStatement(vf.createStatement(blankNode, vf.createURI("urn:direction"), vf.createURI("urn:NW")), "a"));
+        expected.add(new VisibilityStatement(vf.createStatement(blankNode, vf.createURI("urn:location"), vf.createURI("urn:corner1")), "a"));
+
+        // Run the test.
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, expected, VisibilityStatementDeserializer.class);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/92c85ee1/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
new file mode 100644
index 0000000..322bba9
--- /dev/null
+++ b/extras/rya.streams/integration/src/test/java/org/apache/rya/streams/kafka/processors/projection/ProjectionProcessorIT.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.processors.projection;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.kafka.streams.processor.TopologyBuilder;
+import org.apache.rya.api.function.projection.RandomUUIDFactory;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.api.model.VisibilityStatement;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.RyaStreamsTestUtil;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier;
+import org.apache.rya.streams.kafka.processors.StatementPatternProcessorSupplier.StatementPatternProcessor;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+import org.apache.rya.streams.kafka.topology.TopologyFactory;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Sets;
+
+/**
+ * Integration tests the methods of {@link StatementPatternProcessor}.
+ */
+public class ProjectionProcessorIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    @Test
+    public void showProcessorWorks() throws Exception {
+        // Enumerate some topics that will be re-used
+        final String ryaInstance = UUID.randomUUID().toString();
+        final UUID queryId = UUID.randomUUID();
+        final String statementsTopic = KafkaTopics.statementsTopic(ryaInstance);
+        final String resultsTopic = KafkaTopics.queryResultsTopic(queryId);
+
+        // Create a topology for the Query that will be tested.
+        final String sparql =
+                "SELECT (?person AS ?p) ?otherPerson " +
+                "WHERE { " +
+                    "?person <urn:talksTo> ?otherPerson . " +
+                "}";
+
+        final TopologyBuilder builder = new TopologyFactory().build(sparql, statementsTopic, resultsTopic, new RandomUUIDFactory());
+
+        // Load some data into the input topic.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final List<VisibilityStatement> statements = new ArrayList<>();
+        statements.add( new VisibilityStatement(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:talksTo"), vf.createURI("urn:Bob")), "a") );
+
+        // Show the correct binding set results from the job.
+        final Set<VisibilityBindingSet> expected = new HashSet<>();
+
+        final MapBindingSet expectedBs = new MapBindingSet();
+        expectedBs.addBinding("p", vf.createURI("urn:Alice"));
+        expectedBs.addBinding("otherPerson", vf.createURI("urn:Bob"));
+        expected.add(new VisibilityBindingSet(expectedBs, "a"));
+
+        RyaStreamsTestUtil.runStreamProcessingTest(kafka, statementsTopic, resultsTopic, builder, statements, Sets.newHashSet(expected), VisibilityBindingSetDeserializer.class);
+    }
+}
\ No newline at end of file



[18/50] [abbrv] incubator-rya git commit: RYA-377 Implemented a Kafka version of GetQueryResultStream.

Posted by ca...@apache.org.
RYA-377 Implemented a Kafka version of GetQueryResultStream.


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

Branch: refs/heads/master
Commit: fc9775e2639b8efa263c4f54f32f233da7e0b397
Parents: 6056528
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Oct 31 18:24:36 2017 -0400
Committer: caleb <ca...@parsons.com>
Committed: Tue Jan 9 15:13:00 2018 -0500

----------------------------------------------------------------------
 .../streams/api/entity/QueryResultStream.java   |  24 ++-
 .../apache/rya/streams/kafka/KafkaTopics.java   |  21 +-
 .../kafka/entity/KafkaQueryResultStream.java    |  99 ++++++++++
 .../interactor/KafkaGetQueryResultStream.java   | 108 ++++++++++
 .../interactor/KafkaGetQueryResultStreamIT.java | 195 +++++++++++++++++++
 .../rya/test/kafka/KafkaTestInstanceRule.java   |  14 ++
 6 files changed, 455 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/fc9775e2/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
index fdd62df..aa5dcfd 100644
--- a/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
+++ b/extras/rya.streams/api/src/main/java/org/apache/rya/streams/api/entity/QueryResultStream.java
@@ -18,7 +18,8 @@
  */
 package org.apache.rya.streams.api.entity;
 
-import java.util.Collection;
+import static java.util.Objects.requireNonNull;
+
 import java.util.UUID;
 
 import org.apache.rya.api.model.VisibilityBindingSet;
@@ -31,12 +32,26 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  * An infinite stream of {@link VisibilityBindingSet}s that are the results of a query within Rya Streams.
  */
 @DefaultAnnotation(NonNull.class)
-public interface QueryResultStream extends AutoCloseable {
+public abstract class QueryResultStream implements AutoCloseable {
+
+    private final UUID queryId;
+
+    /**
+     * Constructs an instance of {@link QueryResultStream}.
+     *
+     * @param queryId - The query whose results this stream iterates over. (not null)
+     */
+    public QueryResultStream(final UUID queryId) {
+        this.queryId = requireNonNull(queryId);
+    }
+
 
     /**
      * @return Identifies which query in Rya Streams this result stream is over.
      */
-    public UUID getQueryId();
+    public UUID getQueryId() {
+        return queryId;
+    }
 
     /**
      * Wait at most {@code timeoutMs} milliseconds for the next collection of results.
@@ -44,7 +59,8 @@ public interface QueryResultStream extends AutoCloseable {
      * @param timeoutMs - The number of milliseconds to at most wait for the next collection of results. (not null)
      * @return The next collection of {@link VisibilityBindingSet}s that are the result of the query. Empty if
      *   there where no new results within the timout period.
+     * @throws IllegalStateException If the stream has been closed.
      * @throws RyaStreamsException Could not fetch the next set of results.
      */
-    public Collection<VisibilityBindingSet> poll(long timeoutMs) throws RyaStreamsException;
+    public abstract Iterable<VisibilityBindingSet> poll(long timeoutMs) throws IllegalStateException, RyaStreamsException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/fc9775e2/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
index dfc4c9d..a8fbf23 100644
--- a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/KafkaTopics.java
@@ -18,6 +18,10 @@
  */
 package org.apache.rya.streams.kafka;
 
+import static java.util.Objects.requireNonNull;
+
+import java.util.UUID;
+
 import org.apache.rya.streams.api.queries.QueryChangeLog;
 
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
@@ -30,23 +34,36 @@ import edu.umd.cs.findbugs.annotations.NonNull;
 public class KafkaTopics {
 
     /**
-     * Creates the Kafka topic that will be used for a specific instance of Rya's {@link QueryChangeLog}.
+     * Creates the Kafka topic name that is used for a specific instance of Rya's {@link QueryChangeLog}.
      *
      * @param ryaInstance - The Rya instance the change log is for. (not null)
      * @return The name of the Kafka topic.
      */
     public static String queryChangeLogTopic(final String ryaInstance) {
+        requireNonNull(ryaInstance);
         return ryaInstance + "-QueryChangeLog";
     }
 
     /**
-     * Creates the Kafka topic that will be used to load statements into the Rya Streams system for a specific
+     * Creates the Kafka topic name that is used to load statements into the Rya Streams system for a specific
      * instance of Rya.
      *
      * @param ryaInstance - The Rya instance the statements are for. (not null)
      * @return The name of the Kafka topic.
      */
     public static String statementsTopic(final String ryaInstance) {
+        requireNonNull(ryaInstance);
         return ryaInstance + "-Statements";
     }
+
+    /**
+     * Creates the Kafka topic name that is used for a specific query that is managed within Rya Streams.
+     *
+     * @param queryId - The id of the query the topic is for.
+     * @return The name of the Kafka topic.
+     */
+    public static String queryResultsTopic(final UUID queryId) {
+        requireNonNull(queryId);
+        return "QueryResults-" + queryId.toString();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/fc9775e2/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/entity/KafkaQueryResultStream.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/entity/KafkaQueryResultStream.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/entity/KafkaQueryResultStream.java
new file mode 100644
index 0000000..360aaa2
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/entity/KafkaQueryResultStream.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.entity;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Iterator;
+import java.util.UUID;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.api.entity.QueryResultStream;
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A Kafka implementation of {@link QueryResultStream}. It delegates the {@link #poll(long)} method to
+ * a {@link Consumer}. As a result, the starting point of this stream is whatever position the consumer
+ * starts at within the Kafka topic.
+ */
+@DefaultAnnotation(NonNull.class)
+public class KafkaQueryResultStream extends QueryResultStream {
+
+    private final Consumer<?, VisibilityBindingSet> consumer;
+
+    /**
+     * Constructs an instance of {@link KafkaQueryResultStream}.
+     *
+     * @param queryId - The query the results are for. (not null)
+     * @param consumer - The consumer that will be polled by this class. (not null)
+     */
+    public KafkaQueryResultStream(final UUID queryId, final Consumer<?, VisibilityBindingSet> consumer) {
+        super(queryId);
+        this.consumer = requireNonNull(consumer);
+    }
+
+    @Override
+    public Iterable<VisibilityBindingSet> poll(final long timeoutMs) throws RyaStreamsException {
+        return new RecordEntryIterable<>( consumer.poll(timeoutMs) );
+    }
+
+    /**
+     * An {@link Iterable} that creates {@link Iterator}s over a {@link ConsumerRecords}' values.
+     * This is useful for when you don't care about the key portion of a record.
+     *
+     * @param <K> - The type of the record's key.
+     * @param <T> - The type of the record's value.
+     */
+    private final class RecordEntryIterable<K, T> implements Iterable<T> {
+
+        private final ConsumerRecords<K, T> records;
+
+        public RecordEntryIterable(final ConsumerRecords<K, T> records) {
+            this.records = requireNonNull(records);
+        }
+
+        @Override
+        public Iterator<T> iterator() {
+            return new Iterator<T>() {
+                private final Iterator<ConsumerRecord<K, T>> it = records.iterator();
+
+                @Override
+                public boolean hasNext() {
+                    return it.hasNext();
+                }
+
+                @Override
+                public T next() {
+                    return it.next().value();
+                }
+            };
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        consumer.close();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/fc9775e2/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStream.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStream.java b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStream.java
new file mode 100644
index 0000000..b3c3fea
--- /dev/null
+++ b/extras/rya.streams/kafka/src/main/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStream.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.interactor;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Arrays;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.api.entity.QueryResultStream;
+import org.apache.rya.streams.api.exception.RyaStreamsException;
+import org.apache.rya.streams.api.interactor.GetQueryResultStream;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.entity.KafkaQueryResultStream;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetDeserializer;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A Kafka topic implementation of {@link GetQueryResultStream}.
+ */
+@DefaultAnnotation(NonNull.class)
+public class KafkaGetQueryResultStream implements GetQueryResultStream {
+
+    private final String bootstrapServers;
+
+    /**
+     * Constructs an instance of {@link KafkaGetQueryResultStream}.
+     *
+     * @param kafkaHostname - The hostname of the Kafka Broker to connect to. (not null)
+     * @param kafkaPort - The port of the Kafka Broker to connect to. (not null)
+     */
+    public KafkaGetQueryResultStream(final String kafkaHostname, final String kafkaPort) {
+        requireNonNull(kafkaHostname);
+        requireNonNull(kafkaPort);
+        bootstrapServers = kafkaHostname + ":" + kafkaPort;
+    }
+
+    @Override
+    public QueryResultStream fromStart(final UUID queryId) throws RyaStreamsException {
+        requireNonNull(queryId);
+
+        // Always start at the earliest point within the topic.
+        return makeStream(queryId, "earliest");
+    }
+
+    @Override
+    public QueryResultStream fromNow(final UUID queryId) throws RyaStreamsException {
+        requireNonNull(queryId);
+
+        // Always start at the latest point within the topic.
+        return makeStream(queryId, "latest");
+    }
+
+    private QueryResultStream makeStream(final UUID queryId, final String autoOffsetResetConfig) {
+        // Configure which instance of Kafka to connect to.
+        final Properties props = new Properties();
+        props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
+
+        // Nothing meaningful is in the key and the value is a VisibilityBindingSet.
+        props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
+        props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, VisibilityBindingSetDeserializer.class);
+
+        // Use a UUID for the Group Id so that we never register as part of the same group as another consumer.
+        props.put(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
+
+        // Set a client id so that server side logging can be traced.
+        props.put(ConsumerConfig.CLIENT_ID_CONFIG, "Query-Result-Stream-" + queryId);
+
+        // These consumers always start at a specific point and move forwards until the caller is finished with
+        // the returned stream, so never commit the consumer's progress.
+        props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetResetConfig);
+        props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
+
+        // We are not closing the consumer here because the returned QueryResultStream is responsible for closing the
+        // underlying resources required to process it.
+        final KafkaConsumer<Object, VisibilityBindingSet> consumer = new KafkaConsumer<>(props);
+
+        // Register the consumer for the query's results.
+        final String resultTopic = KafkaTopics.queryResultsTopic(queryId);
+        consumer.subscribe(Arrays.asList(resultTopic));
+
+        // Return the result stream.
+        return new KafkaQueryResultStream(queryId, consumer);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/fc9775e2/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
new file mode 100644
index 0000000..3343f76
--- /dev/null
+++ b/extras/rya.streams/kafka/src/test/java/org/apache/rya/streams/kafka/interactor/KafkaGetQueryResultStreamIT.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.rya.streams.kafka.interactor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.rya.api.model.VisibilityBindingSet;
+import org.apache.rya.streams.api.entity.QueryResultStream;
+import org.apache.rya.streams.api.interactor.GetQueryResultStream;
+import org.apache.rya.streams.kafka.KafkaTopics;
+import org.apache.rya.streams.kafka.serialization.VisibilityBindingSetSerializer;
+import org.apache.rya.test.kafka.KafkaTestInstanceRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Integration tests the methods of {@link KafkaGetQueryResultStream}.
+ */
+public class KafkaGetQueryResultStreamIT {
+
+    @Rule
+    public KafkaTestInstanceRule kafka = new KafkaTestInstanceRule(true);
+
+    /**
+     * @return A {@link Producer} that is able to write {@link VisibilityBindingSet}s.
+     */
+    private Producer<?, VisibilityBindingSet> makeProducer() {
+        final Properties producerProps = kafka.createBootstrapServerConfig();
+        producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+        producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VisibilityBindingSetSerializer.class.getName());
+        return new KafkaProducer<>(producerProps);
+    }
+
+    /**
+     * Polls a {@link QueryResultStream} until it has either polled too many times without hitting
+     * the target number of results, or it hits the target number of results.
+     *
+     * @param pollMs - How long each poll could take.
+     * @param pollIterations - The maximum nubmer of polls that will be attempted.
+     * @param targetSize - The number of results to read before stopping.
+     * @param stream - The stream that will be polled.
+     * @return The results that were read from the stream.
+     * @throws Exception If the poll failed.
+     */
+    private List<VisibilityBindingSet> pollForResults(
+            final int pollMs,
+            final int pollIterations,
+            final int targetSize,
+            final QueryResultStream stream)  throws Exception{
+        final List<VisibilityBindingSet> read = new ArrayList<>();
+
+        int i = 0;
+        while(read.size() < targetSize && i < pollIterations) {
+            for(final VisibilityBindingSet visBs : stream.poll(pollMs)) {
+                read.add( visBs );
+            }
+            i++;
+        }
+
+        return read;
+    }
+
+    @Test
+    public void fromStart() throws Exception {
+        // Create an ID for the query.
+        final UUID queryId = UUID.randomUUID();
+
+        // Create a list of test VisibilityBindingSets.
+        final List<VisibilityBindingSet> original = new ArrayList<>();
+
+        final ValueFactory vf = new ValueFactoryImpl();
+        MapBindingSet bs = new MapBindingSet();
+        bs.addBinding("urn:name", vf.createLiteral("Alice"));
+        original.add(new VisibilityBindingSet(bs, "a|b|c"));
+
+        bs = new MapBindingSet();
+        bs.addBinding("urn:name", vf.createLiteral("Bob"));
+        original.add(new VisibilityBindingSet(bs, "a"));
+
+        bs = new MapBindingSet();
+        bs.addBinding("urn:name", vf.createLiteral("Charlie"));
+        original.add(new VisibilityBindingSet(bs, "b|c"));
+
+        // Write some entries to the query result topic in Kafka.
+        try(final Producer<?, VisibilityBindingSet> producer = makeProducer()) {
+            final String resultTopic = KafkaTopics.queryResultsTopic(queryId);
+            for(final VisibilityBindingSet visBs : original) {
+                producer.send(new ProducerRecord<>(resultTopic, visBs));
+            }
+        }
+
+        // Use the interactor that is being tested to read all of the visibility binding sets.
+        final GetQueryResultStream interactor = new KafkaGetQueryResultStream(kafka.getKafkaHostname(), kafka.getKafkaPort());
+        final List<VisibilityBindingSet> read = pollForResults(500, 3, 3, interactor.fromStart(queryId));
+
+        // Show the fetched binding sets match the original, as well as their order.
+        assertEquals(original, read);
+    }
+
+    @Test
+    public void fromNow() throws Exception {
+        // Create an ID for the query.
+        final UUID queryId = UUID.randomUUID();
+
+        try(final Producer<?, VisibilityBindingSet> producer = makeProducer()) {
+            final String resultTopic = KafkaTopics.queryResultsTopic(queryId);
+
+            // Write a single visibility binding set to the query's result topic. This will not appear in the expected results.
+            final ValueFactory vf = new ValueFactoryImpl();
+            MapBindingSet bs = new MapBindingSet();
+            bs.addBinding("urn:name", vf.createLiteral("Alice"));
+            producer.send(new ProducerRecord<>(resultTopic, new VisibilityBindingSet(bs, "a|b|c")));
+            producer.flush();
+
+            // Use the interactor that is being tested to read all of the visibility binding sets that appear after this point.
+            final GetQueryResultStream interactor = new KafkaGetQueryResultStream(kafka.getKafkaHostname(), kafka.getKafkaPort());
+            try(QueryResultStream results = interactor.fromNow(queryId)) {
+                // Read results from the stream.
+                List<VisibilityBindingSet> read = new ArrayList<>();
+                for(final VisibilityBindingSet visBs : results.poll(500)) {
+                    read.add(visBs);
+                }
+
+                // Show nothing has been read.
+                assertTrue(read.isEmpty());
+
+                // Write two more entries to the result topic. These will be seen by the result stream.
+                final List<VisibilityBindingSet> original = new ArrayList<>();
+
+                bs = new MapBindingSet();
+                bs.addBinding("urn:name", vf.createLiteral("Bob"));
+                original.add(new VisibilityBindingSet(bs, "a"));
+
+                bs = new MapBindingSet();
+                bs.addBinding("urn:name", vf.createLiteral("Charlie"));
+                original.add(new VisibilityBindingSet(bs, "b|c"));
+
+                for(final VisibilityBindingSet visBs : original) {
+                    producer.send(new ProducerRecord<>(resultTopic, visBs));
+                }
+                producer.flush();
+
+                // Read the results from the result stream.
+                read = pollForResults(500, 3, 2, results);
+
+                // Show the new entries were read.
+                assertEquals(original, read);
+            }
+        }
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void pollClosedStream() throws Exception {
+        // Create an ID for the query.
+        final UUID queryId = UUID.randomUUID();
+
+        // Use the interactor that is being tested to create a result stream and immediately close it.
+        final GetQueryResultStream interactor = new KafkaGetQueryResultStream(kafka.getKafkaHostname(), kafka.getKafkaPort());
+        final QueryResultStream results = interactor.fromStart(queryId);
+        results.close();
+
+        // Try to poll the closed stream.
+        results.poll(1);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/fc9775e2/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java
----------------------------------------------------------------------
diff --git a/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java b/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java
index 41a0a67..f76fa2b 100644
--- a/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java
+++ b/test/kafka/src/main/java/org/apache/rya/test/kafka/KafkaTestInstanceRule.java
@@ -103,4 +103,18 @@ public class KafkaTestInstanceRule extends ExternalResource {
     public Properties createBootstrapServerConfig() {
         return kafkaInstance.createBootstrapServerConfig();
     }
+
+    /**
+     * @return The hostname of the Kafka Broker.
+     */
+    public String getKafkaHostname() {
+        return kafkaInstance.getBrokerHost();
+    }
+
+    /**
+     * @return The port of the Kafka Broker.
+     */
+    public String getKafkaPort() {
+        return kafkaInstance.getBrokerPort();
+    }
 }
\ No newline at end of file