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/11/27 15:32:23 UTC

jena git commit: Improve AbstractNodeWriter to handle RDF primitive values

Repository: jena
Updated Branches:
  refs/heads/hadoop-rdf 80bccb965 -> de9a24715


Improve AbstractNodeWriter to handle RDF primitive values

If the AbstractNodeWriter receives an RDF primitive as the value of the
key value pair it will now apply proper node formatting to each node in
the primitive separating them with the separator if necessary.


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

Branch: refs/heads/hadoop-rdf
Commit: de9a24715ddc3b190d2c980abd94c3909dba1f08
Parents: 80bccb9
Author: Rob Vesse <rv...@apache.org>
Authored: Thu Nov 27 14:31:06 2014 +0000
Committer: Rob Vesse <rv...@apache.org>
Committed: Thu Nov 27 14:31:06 2014 +0000

----------------------------------------------------------------------
 .../io/output/writers/AbstractNodeWriter.java   | 291 +++++++++++--------
 1 file changed, 173 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/de9a2471/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
index 5d0826d..9d8eeb2 100644
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
@@ -16,122 +16,177 @@
  * limitations under the License.
  */
 
-package org.apache.jena.hadoop.rdf.io.output.writers;
-
-import java.io.IOException;
-import java.io.Writer;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.jena.atlas.io.AWriter;
-import org.apache.jena.atlas.io.Writer2;
+package org.apache.jena.hadoop.rdf.io.output.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.jena.atlas.io.AWriter;
+import org.apache.jena.atlas.io.Writer2;
+import org.apache.jena.atlas.lib.Tuple;
+import org.apache.jena.hadoop.rdf.types.NodeTupleWritable;
 import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.riot.out.NodeFormatter;
-import org.apache.jena.riot.out.NodeFormatterNT;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.hp.hpl.jena.graph.Node;
-
-/**
- * Abstract implementation of a record writer which writes pairs of nodes and
- * arbitrary values to text based files
- * 
- * 
- * 
- * @param <TValue>
- */
-public abstract class AbstractNodeWriter<TValue> extends RecordWriter<NodeWritable, TValue> {
-
-    /**
-     * Default separator written between nodes and their associated values
-     */
-    public static final String DEFAULT_SEPARATOR = "\t";
-
-    private static final Logger log = LoggerFactory.getLogger(AbstractNodeWriter.class);
-
-    protected AWriter writer;
-    private NodeFormatter formatter;
-
-    /**
-     * Creates a new tuple writer using the default NTriples node formatter
-     * 
-     * @param writer
-     *            Writer
-     */
-    public AbstractNodeWriter(Writer writer) {
-        this(writer, new NodeFormatterNT());
-    }
-
-    /**
-     * Creates a new tuple writer
-     * 
-     * @param writer
-     *            Writer
-     * @param formatter
-     *            Node formatter
-     */
-    public AbstractNodeWriter(Writer writer, NodeFormatter formatter) {
-        if (writer == null)
-            throw new NullPointerException("writer cannot be null");
-        if (formatter == null)
-            throw new NullPointerException("formatter cannot be null");
-        this.formatter = formatter;
-        this.writer = Writer2.wrap(writer);
-    }
-    
-    @Override
-    public final void write(NodeWritable key, TValue value) throws IOException, InterruptedException {
-        this.writeKey(key);
-        this.writer.write(this.getSeparator());
-        this.writeValue(value);
-        this.writer.write('\n');
-    }
-
-    /**
-     * Writes the given key
-     * 
-     * @param key
-     *            Key
-     */
-    protected void writeKey(NodeWritable key) {
-        Node n = key.get();
-        this.getNodeFormatter().format(this.writer, n);
-    }
-
-    /**
-     * Writes the given value
-     * 
-     * @param value
-     */
-    protected void writeValue(TValue value) {
-        if (value instanceof NullWritable)
-            return;
-        this.writer.write(value.toString());
-    }
-
-    @Override
-    public void close(TaskAttemptContext context) throws IOException, InterruptedException {
-        log.debug("close({})", context);
-        writer.close();
-    }
-
-    /**
-     * Gets the node formatter to use for formatting nodes
-     * 
-     * @return Node formatter
-     */
-    protected NodeFormatter getNodeFormatter() {
-        return this.formatter;
-    }
-
-    /**
-     * Gets the separator that is written between nodes
-     * 
-     * @return Separator
-     */
-    protected String getSeparator() {
-        return DEFAULT_SEPARATOR;
-    }
-}
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.out.NodeFormatter;
+import org.apache.jena.riot.out.NodeFormatterNT;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract implementation of a record writer which writes pairs of nodes and
+ * arbitrary values to text based files
+ * 
+ * 
+ * 
+ * @param <TValue>
+ */
+public abstract class AbstractNodeWriter<TValue> extends RecordWriter<NodeWritable, TValue> {
+
+    /**
+     * Default separator written between nodes and their associated values
+     */
+    public static final String DEFAULT_SEPARATOR = "\t";
+
+    private static final Logger log = LoggerFactory.getLogger(AbstractNodeWriter.class);
+
+    protected AWriter writer;
+    private NodeFormatter formatter;
+
+    /**
+     * Creates a new tuple writer using the default NTriples node formatter
+     * 
+     * @param writer
+     *            Writer
+     */
+    public AbstractNodeWriter(Writer writer) {
+        this(writer, new NodeFormatterNT());
+    }
+
+    /**
+     * Creates a new tuple writer
+     * 
+     * @param writer
+     *            Writer
+     * @param formatter
+     *            Node formatter
+     */
+    public AbstractNodeWriter(Writer writer, NodeFormatter formatter) {
+        if (writer == null)
+            throw new NullPointerException("writer cannot be null");
+        if (formatter == null)
+            throw new NullPointerException("formatter cannot be null");
+        this.formatter = formatter;
+        this.writer = Writer2.wrap(writer);
+    }
+
+    @Override
+    public final void write(NodeWritable key, TValue value) throws IOException, InterruptedException {
+        this.writeKey(key);
+        this.writer.write(this.getSeparator());
+        this.writeValue(value);
+        this.writer.write('\n');
+    }
+
+    /**
+     * Writes the given key
+     * 
+     * @param key
+     *            Key
+     */
+    protected void writeKey(NodeWritable key) {
+        writeNode(key.get());
+    }
+
+    /**
+     * Writes a Node
+     * 
+     * @param n
+     *            Node
+     */
+    protected void writeNode(Node n) {
+        this.getNodeFormatter().format(this.writer, n);
+    }
+
+    /**
+     * Writes a sequence of nodes
+     * 
+     * @param ns
+     *            Nodes
+     */
+    protected void writeNodes(Node... ns) {
+        String sep = this.getSeparator();
+        for (int i = 0; i < ns.length; i++) {
+            writeNode(ns[i]);
+            if (i < ns.length - 1)
+                this.writer.write(sep);
+        }
+    }
+
+    /**
+     * Writes the given value
+     * <p>
+     * If the value is one of the RDF primitives - {@link NodeWritable},
+     * {@link TripleWritable}, {@link QuadWritable} and
+     * {@link NodeTupleWritable} - then it is formatted as a series of nodes
+     * separated by the separator. Otherwise it is formatted by simply calling
+     * {@code toString()} on it.
+     * </p>
+     * 
+     * @param value
+     *            Values
+     */
+    protected void writeValue(TValue value) {
+        // Handle null specially
+        if (value instanceof NullWritable || value == null)
+            return;
+
+        // Handle RDF primitives specially and format them as proper nodes
+        if (value instanceof NodeWritable) {
+            this.writeKey((NodeWritable) value);
+        } else if (value instanceof TripleWritable) {
+            Triple t = ((TripleWritable) value).get();
+            this.writeNodes(t.getSubject(), t.getPredicate(), t.getObject());
+        } else if (value instanceof QuadWritable) {
+            Quad q = ((QuadWritable) value).get();
+            this.writeNodes(q.getGraph(), q.getSubject(), q.getPredicate(), q.getObject());
+        } else if (value instanceof NodeTupleWritable) {
+            Tuple<Node> tuple = ((NodeTupleWritable) value).get();
+            this.writeNodes(tuple.tuple());
+        } else {
+            // For arbitrary values just toString() them
+            this.writer.write(value.toString());
+        }
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+        log.debug("close({})", context);
+        writer.close();
+    }
+
+    /**
+     * Gets the node formatter to use for formatting nodes
+     * 
+     * @return Node formatter
+     */
+    protected NodeFormatter getNodeFormatter() {
+        return this.formatter;
+    }
+
+    /**
+     * Gets the separator that is written between nodes
+     * 
+     * @return Separator
+     */
+    protected String getSeparator() {
+        return DEFAULT_SEPARATOR;
+    }
+}