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 2014/04/08 15:48:18 UTC

svn commit: r1585725 [2/4] - in /jena/Experimental/hadoop-rdf: hadoop-rdf-mapreduce/src/main/java/com/ hadoop-rdf-mapreduce/src/main/java/org/ hadoop-rdf-mapreduce/src/main/java/org/apache/ hadoop-rdf-mapreduce/src/main/java/org/apache/jena/ hadoop-rdf...

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundQuadFilterMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundQuadFilterMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundQuadFilterMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundQuadFilterMapper.java Tue Apr  8 13:48:16 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ * @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());
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundTripleFilterMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundTripleFilterMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundTripleFilterMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/GroundTripleFilterMapper.java Tue Apr  8 13:48:16 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ * @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());
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidQuadFilterMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidQuadFilterMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidQuadFilterMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidQuadFilterMapper.java Tue Apr  8 13:48:16 2014
@@ -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>
+ * 
+ * @author rvesse
+ * 
+ * @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());
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidTripleFilterMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidTripleFilterMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidTripleFilterMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/ValidTripleFilterMapper.java Tue Apr  8 13:48:16 2014
@@ -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>
+ * 
+ * @author rvesse
+ * 
+ * @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());
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java Tue Apr  8 13:48:16 2014
@@ -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>
+ * 
+ * @author rvesse
+ * 
+ * @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;
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java Tue Apr  8 13:48:16 2014
@@ -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>
+ * 
+ * @author rvesse
+ * 
+ * @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;
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java Tue Apr  8 13:48:16 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ * @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;
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java Tue Apr  8 13:48:16 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ * @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;
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java Tue Apr  8 13:48:16 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ * @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;
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java Tue Apr  8 13:48:16 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ * @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;
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java Tue Apr  8 13:48:16 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ * @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;
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java Tue Apr  8 13:48:16 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ * @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;
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java Tue Apr  8 13:48:16 2014
@@ -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 subjects
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleFilterBySubjectUriMapper<TKey> extends AbstractTripleFilterByPositionMapper<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 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;
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.group;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * Abstract mapper implementation which helps in grouping tuples by assigning
+ * them a {@link NodeWritable} key in place of their existing key. Derived
+ * implementations of this may select the key based on some component of the
+ * tuple or by other custom logic.
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleGroupingMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, NodeWritable, T> {
+
+    @Override
+    protected final void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        NodeWritable newKey = this.selectKey(value);
+        context.write(newKey, value);
+    }
+
+    /**
+     * Gets the key to associated with the tuple
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Node to use as key
+     */
+    protected abstract NodeWritable selectKey(T tuple);
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java Tue Apr  8 13:48:16 2014
@@ -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.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract mapper implementation which helps in grouping quads by assigning
+ * them a {@link NodeWritable} key in place of their existing key. Derived
+ * implementations of this may select the key based on some component of the
+ * quad or by other custom logic.
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ */
+public abstract class AbstractQuadGroupingMapper<TKey> extends AbstractNodeTupleGroupingMapper<TKey, Quad, QuadWritable> {
+
+    protected final NodeWritable selectKey(QuadWritable tuple) {
+        return this.selectKey(tuple.get());
+    }
+
+    /**
+     * Selects the key to use
+     * 
+     * @param quad
+     *            Quad
+     * @return Key to use
+     */
+    protected abstract NodeWritable selectKey(Quad quad);
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java Tue Apr  8 13:48:16 2014
@@ -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.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract mapper implementation which helps in grouping triples by assigning
+ * them a {@link NodeWritable} key in place of their existing key. Derived
+ * implementations of this may select the key based on some component of the
+ * triple or by other custom logic.
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ */
+public abstract class AbstractTripleGroupingMapper<TKey> extends AbstractNodeTupleGroupingMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected final NodeWritable selectKey(TripleWritable tuple) {
+        return this.selectKey(tuple.get());
+    }
+    
+    protected abstract NodeWritable selectKey(Triple triple);
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which assists in grouping quads by graph by reassigning their keys
+ * to be their graphs
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ */
+public class QuadGroupByGraphMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Quad quad) {
+        return new NodeWritable(quad.getGraph());
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which assists in grouping quads by object by reassigning their keys
+ * to be their objects
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ */
+public class QuadGroupByObjectMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Quad quad) {
+        return new NodeWritable(quad.getObject());
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which assists in grouping quads by predicate by reassigning their keys
+ * to be their predicates
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ */
+public class QuadGroupByPredicateMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Quad quad) {
+        return new NodeWritable(quad.getPredicate());
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which assists in grouping quads by subject by reassigning their keys
+ * to be their subjects
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ */
+public class QuadGroupBySubjectMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Quad quad) {
+        return new NodeWritable(quad.getSubject());
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java Tue Apr  8 13:48:16 2014
@@ -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.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which assists in grouping triples by object by reassigning their
+ * keys to be their objects
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleGroupByObjectMapper<TKey> extends AbstractTripleGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Triple triple) {
+        return new NodeWritable(triple.getObject());
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java Tue Apr  8 13:48:16 2014
@@ -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.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which assists in grouping triples by predicate by reassigning their
+ * keys to be their predicates
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleGroupByPredicateMapper<TKey> extends AbstractTripleGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Triple triple) {
+        return new NodeWritable(triple.getPredicate());
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java Tue Apr  8 13:48:16 2014
@@ -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.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which assists in grouping triples by subject by reassigning their
+ * keys to be their subjects
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleGroupBySubjectMapper<TKey> extends AbstractTripleGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Triple triple) {
+        return new NodeWritable(triple.getSubject());
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.split;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * Abstract mapper implementation which splits the tuples into their constituent
+ * nodes preserving the keys as-is
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleSplitToNodesMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, TKey, NodeWritable> {
+
+    @Override
+    protected final void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        NodeWritable[] ns = this.split(value);
+        for (NodeWritable n : ns) {
+            context.write(key, n);
+        }
+    }
+
+    /**
+     * Splits the node tuple type into the individual nodes
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] split(T tuple);
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.split;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * Abstract mapper implementation which splits the tuples into their constituent
+ * nodes using the tuples as the keys and the nodes as the values
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleSplitWithNodesMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, T, NodeWritable> {
+
+    @Override
+    protected void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        NodeWritable[] ns = this.split(value);
+        for (NodeWritable n : ns) {
+            context.write(value, n);
+        }
+    }
+
+    /**
+     * Splits the node tuple type into the individual nodes
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] split(T tuple);
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.split;
+
+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 which splits quads into their constituent nodes preserving the
+ * existing keys as-is
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadSplitToNodesMapper<TKey> extends AbstractNodeTupleSplitToNodesMapper<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected NodeWritable[] split(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()) };
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.split;
+
+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 which splits quads into their constituent nodes using the quad as
+ * the key and the nodes as the values
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadSplitWithNodesMapper<TKey> extends AbstractNodeTupleSplitWithNodesMapper<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected NodeWritable[] split(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()) };
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java Tue Apr  8 13:48:16 2014
@@ -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.split;
+
+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 which splits triples into their constituent nodes
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleSplitToNodesMapper<TKey> extends AbstractNodeTupleSplitToNodesMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected NodeWritable[] split(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java Tue Apr  8 13:48:16 2014
@@ -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.split;
+
+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 which splits triples into their constituent nodes
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleSplitWithNodesMapper<TKey> extends AbstractNodeTupleSplitWithNodesMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected NodeWritable[] split(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.transform;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract mapper which transforms triples into quads. Derived
+ * implementations may choose how the graph to which triples are assigned is
+ * decided.
+ * <p>
+ * Keys are left as is by this mapper.
+ * </p>
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ */
+public abstract class AbstractTriplesToQuadsMapper<TKey> extends Mapper<TKey, TripleWritable, TKey, QuadWritable> {
+
+    @Override
+    protected final void map(TKey key, TripleWritable value, Context context) throws IOException, InterruptedException {
+        Triple triple = value.get();
+        Node graphNode = this.selectGraph(triple);
+        context.write(key, new QuadWritable(new Quad(graphNode, triple)));
+    }
+
+    /**
+     * Selects the graph name to use for converting the given triple into a quad
+     * 
+     * @param triple
+     *            Triple
+     * @return Tuple
+     */
+    protected abstract Node selectGraph(Triple triple);
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java Tue Apr  8 13:48:16 2014
@@ -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.jena.hadoop.rdf.mapreduce.transform;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * A mapper which transforms quads into triples
+ * <p>
+ * Keys are left as is by this mapper.
+ * </p>
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadsToTriplesMapper<TKey> extends Mapper<TKey, QuadWritable, TKey, TripleWritable> {
+
+    @Override
+    protected void map(TKey key, QuadWritable value, Context context) throws IOException, InterruptedException {
+        context.write(key, new TripleWritable(value.get().asTriple()));
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java?rev=1585725&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java Tue Apr  8 13:48:16 2014
@@ -0,0 +1,40 @@
+/*
+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.transform;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which converts triples into quads using the subjects of the triples
+ * as the graph nodes
+ * 
+ * @author rvesse
+ * @param <TKey>
+ *            Key type
+ * 
+ */
+public class TriplesToQuadsBySubjectMapper<TKey> extends AbstractTriplesToQuadsMapper<TKey> {
+
+    @Override
+    protected final Node selectGraph(Triple triple) {
+        return triple.getSubject();
+    }
+
+}