You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by rv...@apache.org on 2015/01/05 16:07:49 UTC

[41/52] [abbrv] jena git commit: Further rebranding to Elephas

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java
new file mode 100644
index 0000000..7c56d1c
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.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.jena.hadoop.rdf.mapreduce.count;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * Abstract mapper class for mappers which split node tuple values into pairs of
+ * node keys with a long value of 1. Can be used in conjunction with a
+ * {@link NodeCountReducer} to count the usages of each unique node.
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleNodeCountMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, NodeWritable, LongWritable> {
+    
+    private LongWritable initialCount = new LongWritable(1);
+
+    @Override
+    protected void map(TKey key, T value, Context context) throws IOException,
+            InterruptedException {
+        NodeWritable[] ns = this.getNodes(value);
+        for (NodeWritable n : ns) {
+            context.write(n, this.initialCount);
+        }
+    }
+
+    /**
+     * Gets the nodes of the tuple which are to be counted
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] getNodes(T tuple);
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java
new file mode 100644
index 0000000..a5460f6
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.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.jena.hadoop.rdf.mapreduce.count;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * A reducer which takes node keys with a sequence of longs representing counts
+ * as the values and sums the counts together into pairs consisting of a node
+ * key and a count value.
+ * 
+ * 
+ * 
+ */
+public class NodeCountReducer extends Reducer<NodeWritable, LongWritable, NodeWritable, LongWritable> {
+
+    @Override
+    protected void reduce(NodeWritable key, Iterable<LongWritable> values, Context context) throws IOException,
+            InterruptedException {
+        long count = 0;
+        Iterator<LongWritable> iter = values.iterator();
+        while (iter.hasNext()) {
+            count += iter.next().get();
+        }
+        context.write(key, new LongWritable(count));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java
new file mode 100644
index 0000000..5c2485b
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.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.jena.hadoop.rdf.mapreduce.count;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper for counting node usages within quads designed primarily for use in
+ * conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadNodeCountMapper<TKey> extends AbstractNodeTupleNodeCountMapper<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java
new file mode 100644
index 0000000..4108841
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.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.jena.hadoop.rdf.mapreduce.count;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper for counting node usages within triples designed primarily for use
+ * in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleNodeCountMapper<TKey> extends AbstractNodeTupleNodeCountMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java
new file mode 100644
index 0000000..7ea7ead
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.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.jena.hadoop.rdf.mapreduce.count.datatypes;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A mapper for counting data type usages within quads designed primarily for
+ * use in conjunction with {@link NodeCountReducer}
+ * <p>
+ * This mapper extracts the data types for typed literal objects and converts
+ * them into nodes so they can be counted
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadDataTypeCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
+
+    private static final NodeWritable[] EMPTY = new NodeWritable[0];
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Node object = tuple.get().getObject();
+        if (!object.isLiteral())
+            return EMPTY;
+        String dtUri = object.getLiteralDatatypeURI();
+        if (dtUri == null)
+            return EMPTY;
+        return new NodeWritable[] { new NodeWritable(NodeFactory.createURI(dtUri)) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java
new file mode 100644
index 0000000..2c294be
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.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.jena.hadoop.rdf.mapreduce.count.datatypes;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A mapper for counting data type usages within triples designed primarily for
+ * use in conjunction with {@link NodeCountReducer}
+ * <p>
+ * This mapper extracts the data types for typed literal objects and converts
+ * them into nodes so they can be counted
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleDataTypeCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
+
+    private static final NodeWritable[] EMPTY = new NodeWritable[0];
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Node object = tuple.get().getObject();
+        if (!object.isLiteral())
+            return EMPTY;
+        String dtUri = object.getLiteralDatatypeURI();
+        if (dtUri == null)
+            return EMPTY;
+        return new NodeWritable[] { new NodeWritable(NodeFactory.createURI(dtUri)) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java
new file mode 100644
index 0000000..ba47765
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.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.jena.hadoop.rdf.mapreduce.count.namespaces;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.graph.Node;
+
+/**
+ * Abstract mapper class for mappers which split node tuple values and extract
+ * the namespace URIs they use and outputs pairs of namespaces keys with a long
+ * value of 1. Can be used in conjunction with a {@link TextCountReducer} to
+ * count the usages of each unique namespace.
+ * 
+ * 
+ * 
+ * @param <TKey>
+ * @param <TValue>
+ * @param <T>
+ */
+public abstract class AbstractNodeTupleNamespaceCountMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, Text, LongWritable> {
+
+    private LongWritable initialCount = new LongWritable(1);
+    protected static final String NO_NAMESPACE = null;
+
+    @Override
+    protected void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        NodeWritable[] ns = this.getNodes(value);
+        for (NodeWritable n : ns) {
+            String namespace = this.extractNamespace(n);
+            if (namespace != null) {
+                context.write(new Text(namespace), this.initialCount);
+            }
+        }
+    }
+
+    /**
+     * Extracts the namespace from a node
+     * <p>
+     * Finds the URI for the node (if any) and then invokes
+     * {@link #extractNamespace(String)} to extract the actual namespace URI.
+     * </p>
+     * <p>
+     * Derived classes may override this to change the logic of how namespaces
+     * are extracted.
+     * </p>
+     * 
+     * @param nw
+     *            Node
+     * @return Namespace
+     */
+    protected String extractNamespace(NodeWritable nw) {
+        Node n = nw.get();
+        if (n.isBlank() || n.isVariable())
+            return NO_NAMESPACE;
+        if (n.isLiteral()) {
+            String dtUri = n.getLiteralDatatypeURI();
+            if (dtUri == null)
+                return NO_NAMESPACE;
+            return extractNamespace(dtUri);
+        }
+        return extractNamespace(n.getURI());
+    }
+
+    /**
+     * Extracts the namespace from a URI
+     * <p>
+     * First tries to extract a hash based namespace. If that is not possible it
+     * tries to extract a slash based namespace, if this is not possible then
+     * the full URI is returned.
+     * </p>
+     * <p>
+     * Derived classes may override this to change the logic of how namespaces
+     * are extracted.
+     * </p>
+     * 
+     * @param uri
+     *            URI
+     * @return Namespace
+     */
+    protected String extractNamespace(String uri) {
+        if (uri.contains("#")) {
+            // Extract hash namespace
+            return uri.substring(0, uri.lastIndexOf('#') + 1);
+        } else if (uri.contains("/")) {
+            // Ensure that this is not immediately after the scheme component or
+            // at end of URI
+            int index = uri.lastIndexOf('/');
+            int schemeSepIndex = uri.indexOf(':');
+            if (index - schemeSepIndex <= 2 || index == uri.length() - 1) {
+                // Use full URI
+                return uri;
+            }
+
+            // Otherwise safe to extract slash namespace
+            return uri.substring(0, uri.lastIndexOf('/') + 1);
+        } else {
+            // Use full URI
+            return uri;
+        }
+    }
+
+    /**
+     * Gets the nodes of the tuple whose namespaces are to be counted
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] getNodes(T tuple);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java
new file mode 100644
index 0000000..97ee546
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.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.jena.hadoop.rdf.mapreduce.count.namespaces;
+
+import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper for counting namespace usages within quads designed primarily for
+ * use in conjunction with the {@link TextCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class QuadNamespaceCountMapper<TKey> extends AbstractNodeTupleNamespaceCountMapper<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java
new file mode 100644
index 0000000..eacef7a
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.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.jena.hadoop.rdf.mapreduce.count.namespaces;
+
+import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper for counting namespace usages within triples designed primarily for
+ * use in conjunction with the {@link TextCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class TripleNamespaceCountMapper<TKey> extends AbstractNodeTupleNamespaceCountMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java
new file mode 100644
index 0000000..ef19623
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.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.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * A mapper for counting object node usages within quads designed primarily for
+ * use in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadObjectCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java
new file mode 100644
index 0000000..0ef0731
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.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.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * A mapper for counting predicate node usages within quads designed primarily
+ * for use in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadPredicateCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getPredicate()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java
new file mode 100644
index 0000000..2b48e79
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.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.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * A mapper for counting subject node usages within quads designed primarily for use
+ * in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class QuadSubjectCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getSubject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java
new file mode 100644
index 0000000..16250eb
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.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.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * A mapper for counting object node usages within triples designed primarily for use
+ * in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleObjectCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java
new file mode 100644
index 0000000..de97a9b
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.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.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * A mapper for counting predicate node usages within triples designed primarily
+ * for use in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TriplePredicateCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getPredicate()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java
new file mode 100644
index 0000000..5dc3838
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.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.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * A mapper for counting subject node usages within triples designed primarily for use
+ * in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleSubjectCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getSubject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java
new file mode 100644
index 0000000..1ecec75
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java
@@ -0,0 +1,76 @@
+/*
+ * 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.jena.hadoop.rdf.mapreduce.filter;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+
+
+/**
+ * Abstract mapper implementation which helps in filtering tuples from the
+ * input, derived implementations provide an implementation of the
+ * {@link #accepts(TKey, T)}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+@SuppressWarnings("javadoc")
+public abstract class AbstractNodeTupleFilterMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, TKey, T> {
+
+    private boolean invert = false;
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.invert = context.getConfiguration().getBoolean(RdfMapReduceConstants.FILTER_INVERT, this.invert);
+    }
+
+    @Override
+    protected final void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        if (this.accepts(key, value)) {
+            if (!this.invert)
+                context.write(key, value);
+        } else if (this.invert) {
+            context.write(key, value);
+        }
+    }
+
+    /**
+     * Gets whether the mapper accepts the key value pair and will pass it as
+     * output
+     * 
+     * @param key
+     *            Key
+     * @param tuple
+     *            Tuple value
+     * @return True if the mapper accepts the given key value pair, false
+     *         otherwise
+     */
+    protected abstract boolean accepts(TKey key, T tuple);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java
new file mode 100644
index 0000000..3caf051
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java
@@ -0,0 +1,35 @@
+/*
+ * 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.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract mapper implementation for filtering quads
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractQuadFilterMapper<TKey> extends AbstractNodeTupleFilterMapper<TKey, Quad, QuadWritable> {
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java
new file mode 100644
index 0000000..e99e369
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java
@@ -0,0 +1,35 @@
+/*
+ * 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.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract mapper implementation for filtering triples
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractTripleFilterMapper<TKey> extends AbstractNodeTupleFilterMapper<TKey, Triple, TripleWritable> {
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundQuadFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundQuadFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundQuadFilterMapper.java
new file mode 100644
index 0000000..c2a6ab9
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundQuadFilterMapper.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A quad filter which accepts only ground quads i.e. those with no blank nodes
+ * or variables
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class GroundQuadFilterMapper<TKey> extends AbstractQuadFilterMapper<TKey> {
+
+    @Override
+    protected boolean accepts(Object key, QuadWritable tuple) {
+        Quad q = tuple.get();
+        if (!q.isConcrete())
+            return false;
+        // Ground if all nodes are URI/Literal
+        return (q.getGraph().isURI() || q.getGraph().isLiteral()) && (q.getSubject().isURI() || q.getSubject().isLiteral())
+                && (q.getPredicate().isURI() || q.getPredicate().isLiteral())
+                && (q.getObject().isURI() || q.getObject().isLiteral());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundTripleFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundTripleFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundTripleFilterMapper.java
new file mode 100644
index 0000000..f83a0e5
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundTripleFilterMapper.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A triple filter which accepts only ground triples i.e. those with no blank
+ * nodes or variables
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class GroundTripleFilterMapper<TKey> extends AbstractTripleFilterMapper<TKey> {
+
+    @Override
+    protected boolean accepts(Object key, TripleWritable tuple) {
+        Triple t = tuple.get();
+        if (!t.isConcrete())
+            return false;
+        // Ground if all nodes are URI/Literal
+        return (t.getSubject().isURI() || t.getSubject().isLiteral())
+                && (t.getPredicate().isURI() || t.getPredicate().isLiteral())
+                && (t.getObject().isURI() || t.getObject().isLiteral());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidQuadFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidQuadFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidQuadFilterMapper.java
new file mode 100644
index 0000000..86771fa
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidQuadFilterMapper.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.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A quad filter mapper which accepts only valid quads, by which we mean they
+ * meet the following criteria:
+ * <ul>
+ * <li>Graph is a URI or Blank Node</li>
+ * <li>Subject is a URI or Blank Node</li>
+ * <li>Predicate is a URI</li>
+ * <li>Object is a URI, Blank Node or Literal</li>
+ * </ul>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public final class ValidQuadFilterMapper<TKey> extends AbstractQuadFilterMapper<TKey> {
+
+    @Override
+    protected final boolean accepts(TKey key, QuadWritable tuple) {
+        Quad q = tuple.get();
+        return (q.getGraph().isURI() || q.getGraph().isBlank()) && (q.getSubject().isURI() || q.getSubject().isBlank())
+                && q.getPredicate().isURI() && (q.getObject().isURI() || q.getObject().isBlank() || q.getObject().isLiteral());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidTripleFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidTripleFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidTripleFilterMapper.java
new file mode 100644
index 0000000..3a13172
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidTripleFilterMapper.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A triple filter mapper which accepts only valid triples, by which we mean they
+ * meet the following criteria:
+ * <ul>
+ * <li>Subject is a URI or Blank Node</li>
+ * <li>Predicate is a URI</li>
+ * <li>Object is a URI, Blank Node or Literal</li>
+ * </ul>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public final class ValidTripleFilterMapper<TKey> extends AbstractTripleFilterMapper<TKey> {
+
+    @Override
+    protected final boolean accepts(TKey key, TripleWritable tuple) {
+        Triple t = tuple.get();
+        return (t.getSubject().isURI() || t.getSubject().isBlank()) && t.getPredicate().isURI()
+                && (t.getObject().isURI() || t.getObject().isBlank() || t.getObject().isLiteral());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
new file mode 100644
index 0000000..f5b5876
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
@@ -0,0 +1,171 @@
+/*
+ * 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.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.filter.AbstractQuadFilterMapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract triple filter that filters quads based on different criteria for
+ * each position (graph, subject, predicate and object) within the quad.
+ * <p>
+ * By default this implementation eliminates all quads it sees, derived
+ * implementations need to override one or more of the specific accept methods
+ * in order to actually accept some triples. See
+ * {@link QuadFilterByPredicateMapper} for an example implementation.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractQuadFilterByPositionMapper<TKey> extends AbstractQuadFilterMapper<TKey> {
+
+    @Override
+    protected final boolean accepts(Object key, QuadWritable tuple) {
+        Quad q = tuple.get();
+        if (!this.acceptsAllGraphs()) {
+            Node g = q.getGraph();
+            if (!this.acceptsGraph(g))
+                return false;
+        }
+        if (!this.acceptsAllSubjects()) {
+            Node s = q.getSubject();
+            if (!this.acceptsSubject(s))
+                return false;
+        }
+        if (!this.acceptsAllPredicates()) {
+            Node p = q.getPredicate();
+            if (!this.acceptsPredicate(p))
+                return false;
+        }
+        if (!this.acceptsAllObjects()) {
+            Node o = q.getObject();
+            if (!this.acceptsObject(o))
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Gets whether this filter accepts all graphs, if false then the
+     * {@link #acceptsGraph(Node)} method will be called to determine if a
+     * specific graph is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all graphs are accepted, false otherwise
+     */
+    protected boolean acceptsAllGraphs() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific graph is acceptable
+     * 
+     * @param graph
+     *            Graph
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsGraph(Node graph) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all subjects, if false then the
+     * {@link #acceptsSubject(Node)} method will be called to determine if a
+     * specific subject is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all subjects are accepted, false otherwise
+     */
+    protected boolean acceptsAllSubjects() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific subject is acceptable
+     * 
+     * @param subject
+     *            Subject
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsSubject(Node subject) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all predicate, if false then the
+     * {@link #acceptsPredicate(Node)} method will be called to determine if a
+     * specific predicate is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all predicates are accepted, false otherwise
+     */
+    protected boolean acceptsAllPredicates() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific predicate is acceptable
+     * 
+     * @param predicate
+     *            Predicate
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsPredicate(Node predicate) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all objects, if false then the
+     * {@link #acceptsObject(Node)} method will be called to determine if a
+     * specific object is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all objects are accepted, false otherwise
+     */
+    protected boolean acceptsAllObjects() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific object is acceptable
+     * 
+     * @param object
+     *            Object
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsObject(Node object) {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java
new file mode 100644
index 0000000..973d651
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.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.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.filter.AbstractTripleFilterMapper;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An abstract triple filter that filters triples based on different criteria
+ * for each position (subject, predicate and object) within the triple.
+ * <p>
+ * By default this implementation eliminates all triples it sees, derived
+ * implementations need to override one or more of the specific accept methods
+ * in order to actually accept some triples. See
+ * {@link TripleFilterByPredicateUriMapper} for an example implementation.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractTripleFilterByPositionMapper<TKey> extends AbstractTripleFilterMapper<TKey> {
+
+    @Override
+    protected final boolean accepts(Object key, TripleWritable tuple) {
+        Triple t = tuple.get();
+        if (!this.acceptsAllSubjects()) {
+            Node s = t.getSubject();
+            if (!this.acceptsSubject(s))
+                return false;
+        }
+        if (!this.acceptsAllPredicates()) {
+            Node p = t.getPredicate();
+            if (!this.acceptsPredicate(p))
+                return false;
+        }
+        if (!this.acceptsAllObjects()) {
+            Node o = t.getObject();
+            if (!this.acceptsObject(o))
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Gets whether this filter accepts all subjects, if false then the
+     * {@link #acceptsSubject(Node)} method will be called to determine if a
+     * specific subject is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all subjects are accepted, false otherwise
+     */
+    protected boolean acceptsAllSubjects() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific subject is acceptable
+     * 
+     * @param subject
+     *            Subject
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsSubject(Node subject) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all predicate, if false then the
+     * {@link #acceptsPredicate(Node)} method will be called to determine if a
+     * specific predicate is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all predicates are accepted, false otherwise
+     */
+    protected boolean acceptsAllPredicates() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific predicate is acceptable
+     * 
+     * @param predicate
+     *            Predicate
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsPredicate(Node predicate) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all objects, if false then the
+     * {@link #acceptsObject(Node)} method will be called to determine if a
+     * specific object is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all objects are accepted, false otherwise
+     */
+    protected boolean acceptsAllObjects() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific object is acceptable
+     * 
+     * @param object
+     *            Object
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsObject(Node object) {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
new file mode 100644
index 0000000..1c2b29e
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
@@ -0,0 +1,76 @@
+/*
+ * 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.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A quad filter which selects quads which have matching subjects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadFilterByGraphUriMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
+
+    private List<Node> graphs = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the subject URIs we are filtering on
+        String[] graphUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_GRAPH_URIS);
+        if (graphUris != null) {
+            for (String graphUri : graphUris) {
+                this.graphs.add(NodeFactory.createURI(graphUri));
+            }
+        }
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsGraph(Node graph) {
+        if (this.graphs.size() == 0)
+            return false;
+        return this.graphs.contains(graph);
+    }
+
+    @Override
+    protected boolean acceptsAllPredicates() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllObjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
new file mode 100644
index 0000000..a3be806
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
@@ -0,0 +1,76 @@
+/*
+ * 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.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A quad filter which selects quads which have matching objects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadFilterByObjectUriMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
+
+    private List<Node> objects = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the subject URIs we are filtering on
+        String[] objectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_OBJECT_URIS);
+        if (objectUris != null) {
+            for (String objectUri : objectUris) {
+                this.objects.add(NodeFactory.createURI(objectUri));
+            }
+        }
+    }
+    
+    @Override
+    protected boolean acceptsAllGraphs() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsObject(Node object) {
+        if (this.objects.size() == 0)
+            return false;
+        return this.objects.contains(object);
+    }
+
+    @Override
+    protected boolean acceptsAllPredicates() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
new file mode 100644
index 0000000..1be64bf
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
@@ -0,0 +1,76 @@
+/*
+ * 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.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A quad filter which selects quads which have matching predicates
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadFilterByPredicateMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
+
+    private List<Node> predicates = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the predicate URIs we are filtering on
+        String[] predicateUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS);
+        if (predicateUris != null) {
+            for (String predicateUri : predicateUris) {
+                this.predicates.add(NodeFactory.createURI(predicateUri));
+            }
+        }
+    }
+    
+    @Override
+    protected boolean acceptsAllGraphs() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsPredicate(Node predicate) {
+        if (this.predicates.size() == 0)
+            return false;
+        return this.predicates.contains(predicate);
+    }
+
+    @Override
+    protected boolean acceptsAllObjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
new file mode 100644
index 0000000..2a4f37b
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
@@ -0,0 +1,76 @@
+/*
+ * 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.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A quad filter which selects quads which have matching subjects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadFilterBySubjectUriMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
+
+    private List<Node> subjects = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the subject URIs we are filtering on
+        String[] subjectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_SUBJECT_URIS);
+        if (subjectUris != null) {
+            for (String subjectUri : subjectUris) {
+                this.subjects.add(NodeFactory.createURI(subjectUri));
+            }
+        }
+    }
+    
+    @Override
+    protected boolean acceptsAllGraphs() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsSubject(Node subject) {
+        if (this.subjects.size() == 0)
+            return false;
+        return this.subjects.contains(subject);
+    }
+
+    @Override
+    protected boolean acceptsAllPredicates() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllObjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java
new file mode 100644
index 0000000..035aabc
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.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.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A triple filter which selects triples which have matching objects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleFilterByObjectUriMapper<TKey> extends AbstractTripleFilterByPositionMapper<TKey> {
+
+    private List<Node> objects = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the subject URIs we are filtering on
+        String[] objectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_OBJECT_URIS);
+        if (objectUris != null) {
+            for (String objectUri : objectUris) {
+                this.objects.add(NodeFactory.createURI(objectUri));
+            }
+        }
+    }
+
+    @Override
+    protected boolean acceptsObject(Node object) {
+        if (this.objects.size() == 0)
+            return false;
+        return this.objects.contains(object);
+    }
+
+    @Override
+    protected boolean acceptsAllPredicates() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java
new file mode 100644
index 0000000..061a3e7
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.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.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A triple filter which selects triples which have matching predicates
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleFilterByPredicateUriMapper<TKey> extends AbstractTripleFilterByPositionMapper<TKey> {
+
+    private List<Node> predicates = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the predicate URIs we are filtering on
+        String[] predicateUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS);
+        if (predicateUris != null) {
+            for (String predicateUri : predicateUris) {
+                this.predicates.add(NodeFactory.createURI(predicateUri));
+            }
+        }
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsPredicate(Node predicate) {
+        if (this.predicates.size() == 0)
+            return false;
+        return this.predicates.contains(predicate);
+    }
+
+    @Override
+    protected boolean acceptsAllObjects() {
+        return true;
+    }
+}