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

[08/59] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java
new file mode 100644
index 0000000..30999ae
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.system.StreamRDF;
+
+/**
+ * Abstract output format for formats that use the RIOT {@link StreamRDF} API to
+ * stream the writes
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TTuple>
+ *            Tuple type
+ * @param <TValue>
+ *            Writable tuple type i.e. the value type
+ */
+public abstract class AbstractStreamRdfNodeTupleOutputFormat<TKey, TTuple, TValue extends AbstractNodeTupleWritable<TTuple>>
+        extends AbstractNodeTupleOutputFormat<TKey, TTuple, TValue> {
+
+    @Override
+    protected RecordWriter<TKey, TValue> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
+        return getRecordWriter(getStream(writer, config), writer, config);
+    }
+
+    /**
+     * Gets a writer which provides a bridge between the {@link RecordWriter}
+     * and {@link StreamRDF} APIs
+     * 
+     * @param stream
+     *            RDF Stream
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * @return Record Writer
+     */
+    protected abstract RecordWriter<TKey, TValue> getRecordWriter(StreamRDF stream, Writer writer, Configuration config);
+
+    /**
+     * Gets a {@link StreamRDF} to which the tuples to be output should be
+     * passed
+     * 
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * @return RDF Stream
+     */
+    protected abstract StreamRDF getStream(Writer writer, Configuration config);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java
new file mode 100644
index 0000000..cc9fe2f
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.registry.HadoopRdfIORegistry;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An output format for RDF quads that dynamically selects the appropriate quad
+ * writer to use based on the file extension of the output file.
+ * <p>
+ * For example this is useful when the output format may be controlled by a user
+ * supplied filename i.e. the desired RDF output format is not precisely known
+ * in advance
+ * </p>
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class QuadsOutputFormat<TKey> extends AbstractNodeTupleOutputFormat<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected RecordWriter<TKey, QuadWritable> getRecordWriter(Writer writer, Configuration config, Path outputPath)
+            throws IOException {
+        Lang lang = RDFLanguages.filenameToLang(outputPath.getName());
+        if (lang == null)
+            throw new IOException("There is no registered RDF language for the output file " + outputPath.toString());
+
+        if (!RDFLanguages.isQuads(lang))
+            throw new IOException(
+                    lang.getName()
+                            + " is not a RDF quads format, perhaps you wanted TriplesOutputFormat or TriplesOrQuadsOutputFormat instead?");
+
+        // This will throw an appropriate error if the language does not support
+        // writing quads
+        return HadoopRdfIORegistry.<TKey> createQuadWriter(lang, writer, config);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java
new file mode 100644
index 0000000..3eaf0d7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.writers.QuadsToTriplesWriter;
+import org.apache.jena.hadoop.rdf.io.registry.HadoopRdfIORegistry;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An output format for RDF triples/quads that dynamically selects the
+ * appropriate triple/quad writer to use based on the file extension of the
+ * output file.
+ * <p>
+ * For example this is useful when the output format may be controlled by a user
+ * supplied filename i.e. the desired RDF output format is not precisely known
+ * in advance.
+ * </p>
+ * <h3>Warning</h3>
+ * <p>
+ * Where the format is determined to be triples the quads are converted into
+ * triples are thus will lose any graph information that might be carried.
+ * </p>
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class TriplesOrQuadsOutputFormat<TKey> extends AbstractNodeTupleOutputFormat<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected RecordWriter<TKey, QuadWritable> getRecordWriter(Writer writer, Configuration config, Path outputPath)
+            throws IOException {
+        Lang lang = RDFLanguages.filenameToLang(outputPath.getName());
+        if (lang == null)
+            throw new IOException("There is no registered RDF language for the output file " + outputPath.toString());
+
+        if (!RDFLanguages.isQuads(lang) && !RDFLanguages.isTriples(lang))
+            throw new IOException(lang.getName() + " is not a RDF triples/quads format");
+
+        if (HadoopRdfIORegistry.hasQuadWriter(lang)) {
+            // Supports quads directly
+            return HadoopRdfIORegistry.<TKey> createQuadWriter(lang, writer, config);
+        } else {
+            // Try to create a triples writer and wrap downwards from quads
+            // This will throw an error if a triple writer is not available
+            return new QuadsToTriplesWriter<TKey>(HadoopRdfIORegistry.<TKey> createTripleWriter(lang, writer, config));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java
new file mode 100644
index 0000000..d9d4189
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.registry.HadoopRdfIORegistry;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An output format for RDF triples that dynamically selects the appropriate triple
+ * writer to use based on the file extension of the output file.
+ * <p>
+ * For example this is useful when the output format may be controlled by a user
+ * supplied filename i.e. the desired RDF output format is not precisely known
+ * in advance
+ * </p>
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class TriplesOutputFormat<TKey> extends AbstractNodeTupleOutputFormat<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected RecordWriter<TKey, TripleWritable> getRecordWriter(Writer writer, Configuration config, Path outputPath) throws IOException {
+        Lang lang = RDFLanguages.filenameToLang(outputPath.getName());
+        if (lang == null)
+            throw new IOException("There is no registered RDF language for the output file " + outputPath.toString());
+        
+        if (!RDFLanguages.isTriples(lang)) throw new IOException(
+                lang.getName()
+                + " is not a RDF triples format, perhaps you wanted QuadsOutputFormat or TriplesOrQuadsOutputFormat instead?");
+        
+        // This will throw an appropriate error if the language does not support writing triples
+        return HadoopRdfIORegistry.<TKey>createTripleWriter(lang, writer, config);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java
new file mode 100644
index 0000000..8f4797a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.jsonld;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.jsonld.JsonLDQuadWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+public class JsonLDQuadOutputFormat<TKey> extends AbstractNodeTupleOutputFormat<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected String getFileExtension() {
+        return ".jsonld";
+    }
+
+    @Override
+    protected RecordWriter<TKey, QuadWritable> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
+        return new JsonLDQuadWriter<TKey>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java
new file mode 100644
index 0000000..a8cbeac
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.jsonld;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.jsonld.JsonLDTripleWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+public class JsonLDTripleOutputFormat<TKey> extends AbstractNodeTupleOutputFormat<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected String getFileExtension() {
+        return ".jsonld";
+    }
+
+    @Override
+    protected RecordWriter<TKey, TripleWritable> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
+        return new JsonLDTripleWriter<TKey>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java
new file mode 100644
index 0000000..a8ab017
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.nquads;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.nquads.NQuadsWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * NQuads output format
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class NQuadsOutputFormat<TKey> extends AbstractNodeTupleOutputFormat<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected RecordWriter<TKey, QuadWritable> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
+        return new NQuadsWriter<TKey>(writer);
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nq";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java
new file mode 100644
index 0000000..56935bb
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.ntriples;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractNodeOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.ntriples.NTriplesNodeWriter;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * NTriples based node output format
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Value type
+ */
+public class NTriplesNodeOutputFormat<TValue> extends AbstractNodeOutputFormat<TValue> {
+
+    @Override
+    protected RecordWriter<NodeWritable, TValue> getRecordWriter(Writer writer, Configuration config) {
+        return new NTriplesNodeWriter<TValue>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java
new file mode 100644
index 0000000..51b9b75
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.ntriples;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.ntriples.NTriplesWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * NTriples output format
+ * 
+ * 
+ * @param <TKey> 
+ * 
+ */
+public class NTriplesOutputFormat<TKey> extends AbstractNodeTupleOutputFormat<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected RecordWriter<TKey, TripleWritable> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
+        return new NTriplesWriter<TKey>(writer);
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nt";
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java
new file mode 100644
index 0000000..e5fa114
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.rdfjson;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.rdfjson.RdfJsonWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * RDF/JSON output format
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class RdfJsonOutputFormat<TKey> extends AbstractNodeTupleOutputFormat<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected String getFileExtension() {
+        return ".rj";
+    }
+
+    @Override
+    protected RecordWriter<TKey, TripleWritable> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
+        return new RdfJsonWriter<TKey>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java
new file mode 100644
index 0000000..6c9a9ea
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.rdfxml;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.rdfxml.RdfXmlWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * RDF/XML output format
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class RdfXmlOutputFormat<TKey> extends AbstractNodeTupleOutputFormat<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected String getFileExtension() {
+        return ".rdf";
+    }
+
+    @Override
+    protected RecordWriter<TKey, TripleWritable> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
+        return new RdfXmlWriter<TKey>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java
new file mode 100644
index 0000000..bd07bff
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.thrift;
+
+import java.io.Writer;
+import java.nio.charset.Charset;
+
+import org.apache.commons.io.output.WriterOutputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractStreamRdfNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.StreamRdfQuadWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.system.StreamRDF;
+import org.apache.jena.riot.thrift.StreamRDF2Thrift;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+public class ThriftQuadOutputFormat<TKey> extends AbstractStreamRdfNodeTupleOutputFormat<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected String getFileExtension() {
+        return ".trdf";
+    }
+
+    @Override
+    protected RecordWriter<TKey, QuadWritable> getRecordWriter(StreamRDF stream, Writer writer, Configuration config) {
+        return new StreamRdfQuadWriter<TKey>(stream, writer);
+    }
+
+    @Override
+    protected StreamRDF getStream(Writer writer, Configuration config) {
+        return new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")), false);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java
new file mode 100644
index 0000000..73e40bc
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.thrift;
+
+import java.io.Writer;
+import java.nio.charset.Charset;
+
+import org.apache.commons.io.output.WriterOutputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractStreamRdfNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.StreamRdfTripleWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.system.StreamRDF;
+import org.apache.jena.riot.thrift.StreamRDF2Thrift;
+
+import com.hp.hpl.jena.graph.Triple;
+
+public class ThriftTripleOutputFormat<TKey> extends AbstractStreamRdfNodeTupleOutputFormat<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected String getFileExtension() {
+        return ".trdf";
+    }
+
+    @Override
+    protected RecordWriter<TKey, TripleWritable> getRecordWriter(StreamRDF stream, Writer writer, Configuration config) {
+        return new StreamRdfTripleWriter<TKey>(stream, writer);
+    }
+
+    @Override
+    protected StreamRDF getStream(Writer writer, Configuration config) {
+        return new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")), false);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java
new file mode 100644
index 0000000..6f33e29
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.trig;
+
+import java.io.Writer;
+
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractBatchedNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.trig.BatchedTriGWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Output format for TriG that uses a batched approach, note that this will
+ * produce invalid data where blank nodes span batches so it is typically better
+ * to use the {@link TriGOutputFormat} instead
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class BatchedTriGOutputFormat<TKey> extends
+		AbstractBatchedNodeTupleOutputFormat<TKey, Quad, QuadWritable> {
+
+	@Override
+	protected RecordWriter<TKey, QuadWritable> getRecordWriter(Writer writer,
+			long batchSize) {
+		return new BatchedTriGWriter<TKey>(writer, batchSize);
+	}
+
+	@Override
+	protected String getFileExtension() {
+		return ".trig";
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java
new file mode 100644
index 0000000..0047095
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java
@@ -0,0 +1,58 @@
+/*
+ * 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.io.output.trig;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractStreamRdfNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.StreamRdfQuadWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.system.StreamRDF;
+import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Output format for TriG
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TriGOutputFormat<TKey> extends AbstractStreamRdfNodeTupleOutputFormat<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected RecordWriter<TKey, QuadWritable> getRecordWriter(StreamRDF stream, Writer writer, Configuration config) {
+        return new StreamRdfQuadWriter<TKey>(stream, writer);
+    }
+
+    @Override
+    protected StreamRDF getStream(Writer writer, Configuration config) {
+        return new WriterStreamRDFBlocks(writer);
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trig";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java
new file mode 100644
index 0000000..c67b3da
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.trix;
+
+import java.io.Writer;
+import java.nio.charset.Charset;
+
+import org.apache.commons.io.output.WriterOutputStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractStreamRdfNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.StreamRdfQuadWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.system.StreamRDF;
+import org.apache.jena.riot.writer.StreamWriterTriX;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Output format for TriX
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TriXOutputFormat<TKey> extends AbstractStreamRdfNodeTupleOutputFormat<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected RecordWriter<TKey, QuadWritable> getRecordWriter(StreamRDF stream, Writer writer, Configuration config) {
+        return new StreamRdfQuadWriter<TKey>(stream, writer);
+    }
+
+    @Override
+    protected StreamRDF getStream(Writer writer, Configuration config) {
+        return new StreamWriterTriX(new WriterOutputStream(writer, Charset.forName("utf-8")));
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trix";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java
new file mode 100644
index 0000000..c7564ac
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.turtle;
+
+import java.io.Writer;
+
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractBatchedNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.turtle.BatchedTurtleWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Output format for Turtle that uses a batched approach, note that this will
+ * produce invalid data where blank nodes span batches so it is typically better
+ * to use the {@link TurtleOutputFormat} instead
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class BatchedTurtleOutputFormat<TKey> extends AbstractBatchedNodeTupleOutputFormat<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected RecordWriter<TKey, TripleWritable> getRecordWriter(Writer writer, long batchSize) {
+        return new BatchedTurtleWriter<TKey>(writer, batchSize);
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".ttl";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java
new file mode 100644
index 0000000..c0202d8
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.turtle;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.AbstractStreamRdfNodeTupleOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.writers.StreamRdfTripleWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.system.StreamRDF;
+import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Turtle output format
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TurtleOutputFormat<TKey> extends AbstractStreamRdfNodeTupleOutputFormat<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected String getFileExtension() {
+        return ".ttl";
+    }
+
+    @Override
+    protected RecordWriter<TKey, TripleWritable> getRecordWriter(StreamRDF stream, Writer writer, Configuration config) {
+        return new StreamRdfTripleWriter<TKey>(stream, writer);
+    }
+
+    @Override
+    protected StreamRDF getStream(Writer writer, Configuration config) {
+        return new WriterStreamRDFBlocks(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java
new file mode 100644
index 0000000..c4eafd8
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java
@@ -0,0 +1,113 @@
+/*
+ * 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.io.output.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * An abstract implementation of a record writer that writes records in batches.
+ * <p>
+ * It is important to note that the writer will write output periodically once
+ * sufficient tuples have been gathered. If there is an incomplete batch when
+ * the {@link #close(TaskAttemptContext)} method is called then the final batch
+ * will be written then. Writing in batches increases the chances that the
+ * writer will be able to effectively use the syntax compressions of the RDF
+ * serialization being used.
+ * </p>
+ * <p>
+ * The implementation only writes the value portion of the key value pair since
+ * it is the value portion that is used to convey the node tuples
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ * @param <TValue>
+ * @param <T>
+ */
+public abstract class AbstractBatchedNodeTupleWriter<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        RecordWriter<TKey, T> {
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractBatchedNodeTupleWriter.class);
+
+    private Writer writer;
+    private long batchSize;
+
+    protected AbstractBatchedNodeTupleWriter(Writer writer, long batchSize) {
+        if (writer == null)
+            throw new NullPointerException("writer cannot be null");
+        if (batchSize <= 0)
+            throw new IllegalArgumentException("batchSize must be >= 1");
+        this.writer = writer;
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    public final void write(TKey key, T value) throws IOException, InterruptedException {
+        LOG.debug("write({}={})", key, value);
+        if (this.add(value) >= this.batchSize) {
+            long size = this.writeOutput(writer);
+            if (size > 0)
+                throw new IOException("Derived implementation failed to empty the current batch after writing");
+        }
+    }
+
+    /**
+     * Adds the tuple to the batch of tuples that will be written when the batch
+     * threshold is reached or when the {@link #close(TaskAttemptContext)}
+     * method is called.
+     * 
+     * @param value
+     *            Tuple
+     * @return The current size of the batch waiting to be written
+     */
+    protected abstract long add(T value);
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+        if (this.writer != null) {
+            long size = this.writeOutput(writer);
+            if (size > 0)
+                throw new IOException("Derived implementation failed to empty the current batch after writing");
+            this.writer.close();
+            this.writer = null;
+        }
+    }
+
+    /**
+     * Writes the current batch of tuples to the writer, the writer should not
+     * be closed and the batch should be emptied by the implementation.
+     * <p>
+     * If the current batch is empty then this should be a no-op
+     * </p>
+     * 
+     * @param writer
+     *            Writer
+     * @return Current batch size which should always be zero
+     */
+    protected abstract long writeOutput(Writer writer);
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java
new file mode 100644
index 0000000..13c2799
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java
@@ -0,0 +1,80 @@
+/*
+ * 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.io.output.writers;
+
+import java.io.Writer;
+import java.util.List;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.RDFWriterRegistry;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.core.DatasetGraph;
+import com.hp.hpl.jena.sparql.core.DatasetGraphFactory;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract batched record writer for quad formats
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public abstract class AbstractBatchedQuadWriter<TKey> extends AbstractBatchedNodeTupleWriter<TKey, Quad, QuadWritable> {
+
+    private DatasetGraph g = DatasetGraphFactory.createMem();
+
+    protected AbstractBatchedQuadWriter(Writer writer, long batchSize) {
+        super(writer, batchSize);
+    }
+
+    @Override
+    protected final long add(QuadWritable value) {
+        g.add(value.get());
+        return g.size();
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected final long writeOutput(Writer writer) {
+        if (this.g.size() == 0)
+            return 0;
+        RDFDataMgr.write(writer, this.g, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
+
+        // Clear the dataset graph
+        @SuppressWarnings("unchecked")
+        List<Node> graphNames = IteratorUtils.toList(this.g.listGraphNodes());
+        for (Node graphName : graphNames) {
+            this.g.removeGraph(graphName);
+        }
+        this.g.getDefaultGraph().clear();
+
+        return this.g.size();
+    }
+
+    /**
+     * Gets the RDF language used for output
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java
new file mode 100644
index 0000000..881cf15
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java
@@ -0,0 +1,68 @@
+/*
+ * 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.io.output.writers;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+
+import com.hp.hpl.jena.graph.Graph;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.graph.GraphFactory;
+
+/**
+ * Abstract batched record writer for triple formats
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public abstract class AbstractBatchedTripleWriter<TKey> extends AbstractBatchedNodeTupleWriter<TKey, Triple, TripleWritable> {
+
+    private Graph g = GraphFactory.createDefaultGraph();
+
+    protected AbstractBatchedTripleWriter(Writer writer, long batchSize) {
+        super(writer, batchSize);
+    }
+
+    @Override
+    protected final long add(TripleWritable value) {
+        g.add(value.get());
+        return g.size();
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected final long writeOutput(Writer writer) {
+        if (this.g.size() == 0)
+            return 0;
+        RDFDataMgr.write(writer, this.g, this.getRdfLanguage());
+        this.g.clear();
+        return this.g.size();
+    }
+
+    /**
+     * Gets the RDF language used for output
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java
new file mode 100644
index 0000000..89b8f4b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+
+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.hadoop.rdf.types.AbstractNodeTupleWritable;
+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;
+
+/**
+ * An abstract implementation of a record writer that writes records to a line
+ * based tuple formats.
+ * <p>
+ * The implementation only writes the value portion of the key value pair since
+ * it is the value portion that is used to convey the node tuples
+ * </p>
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable node tuple type
+ * 
+ */
+public abstract class AbstractLineBasedNodeTupleWriter<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        RecordWriter<TKey, T> {
+    /**
+     * Default separator written between nodes
+     */
+    public static final String DEFAULT_SEPARATOR = " ";
+    /**
+     * Default terminator written at the end of each line
+     */
+    public static final String DEFAULT_TERMINATOR = ".";
+
+    private static final Logger log = LoggerFactory.getLogger(AbstractLineBasedNodeTupleWriter.class);
+
+    private AWriter writer;
+    private NodeFormatter formatter;
+
+    /**
+     * Creates a new tuple writer using the default NTriples node formatter
+     * 
+     * @param writer
+     *            Writer
+     */
+    public AbstractLineBasedNodeTupleWriter(Writer writer) {
+        this(writer, new NodeFormatterNT());
+    }
+
+    /**
+     * Creates a new tuple writer
+     * 
+     * @param writer
+     *            Writer
+     * @param formatter
+     *            Node formatter
+     */
+    public AbstractLineBasedNodeTupleWriter(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 void write(TKey key, T value) throws IOException, InterruptedException {
+        log.debug("write({}={})", key, value);
+
+        Node[] ns = this.getNodes(value);
+        String sep = this.getSeparator();
+        NodeFormatter formatter = this.getNodeFormatter();
+        for (int i = 0; i < ns.length; i++) {
+            formatter.format(this.writer, ns[i]);
+            this.writer.print(sep);
+        }
+        this.writer.println(this.getTerminator());
+        this.writer.flush();
+    }
+
+    /**
+     * Gets the nodes of the tuple in the order they should be written
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Nodes
+     */
+    protected abstract Node[] getNodes(T tuple);
+
+    /**
+     * 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;
+    }
+
+    /**
+     * Gets the terminator that is written at the end of each tuple
+     * 
+     * @return Terminator
+     */
+    protected String getTerminator() {
+        return DEFAULT_TERMINATOR;
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+        log.debug("close({})", context);
+        writer.close();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java
new file mode 100644
index 0000000..9ecef61
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.writers;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.out.NodeFormatter;
+import org.apache.jena.riot.out.NodeFormatterNT;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract writer for line based quad formats
+ * 
+ * 
+ * @param <TKey>
+ * 
+ */
+public abstract class AbstractLineBasedQuadWriter<TKey> extends AbstractLineBasedNodeTupleWriter<TKey, Quad, QuadWritable> {
+
+    /**
+     * Creates a new writer using the default NTriples node formatter
+     * 
+     * @param writer
+     *            Writer
+     */
+    public AbstractLineBasedQuadWriter(Writer writer) {
+        this(writer, new NodeFormatterNT());
+    }
+
+    /**
+     * Creates a new writer using the specified node formatter
+     * 
+     * @param writer
+     *            Writer
+     * @param formatter
+     *            Node formatter
+     */
+    public AbstractLineBasedQuadWriter(Writer writer, NodeFormatter formatter) {
+        super(writer, formatter);
+    }
+
+    @Override
+    protected Node[] getNodes(QuadWritable tuple) {
+        Quad q = tuple.get();
+        if (q.isDefaultGraph()) {
+            return new Node[] { q.getSubject(), q.getPredicate(), q.getObject() };
+        } else {
+            return new Node[] { q.getSubject(), q.getPredicate(), q.getObject(), q.getGraph() };
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java
new file mode 100644
index 0000000..161c067
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java
@@ -0,0 +1,68 @@
+/*
+ * 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.io.output.writers;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.out.NodeFormatter;
+import org.apache.jena.riot.out.NodeFormatterNT;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An abstract writer for line based triple formats
+ * 
+ * @param <TKey>
+ *            Key type
+ * 
+ */
+public abstract class AbstractLineBasedTripleWriter<TKey> extends
+		AbstractLineBasedNodeTupleWriter<TKey, Triple, TripleWritable> {
+
+	/**
+	 * Creates a new writer using the default NTriples node formatter
+	 * 
+	 * @param writer
+	 *            Writer
+	 */
+	public AbstractLineBasedTripleWriter(Writer writer) {
+		this(writer, new NodeFormatterNT());
+	}
+
+	/**
+	 * Creates a new writer using the specified node formatter
+	 * 
+	 * @param writer
+	 *            Writer
+	 * @param formatter
+	 *            Node formatter
+	 */
+	public AbstractLineBasedTripleWriter(Writer writer, NodeFormatter formatter) {
+		super(writer, formatter);
+	}
+
+	@Override
+	protected Node[] getNodes(TripleWritable tuple) {
+		Triple t = tuple.get();
+		return new Node[] { t.getSubject(), t.getPredicate(), t.getObject() };
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
new file mode 100644
index 0000000..9d8eeb2
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
@@ -0,0 +1,192 @@
+/*
+ * 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.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.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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java
new file mode 100644
index 0000000..aa178b2
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.system.StreamRDF;
+
+public abstract class AbstractStreamRdfNodeTupleWriter<TKey, TTuple, TValue extends AbstractNodeTupleWritable<TTuple>>
+		extends RecordWriter<TKey, TValue> {
+
+	private StreamRDF stream;
+	private Writer writer;
+
+	public AbstractStreamRdfNodeTupleWriter(StreamRDF stream, Writer writer) {
+		if (stream == null)
+			throw new NullPointerException("stream cannot be null");
+		if (writer == null)
+			throw new NullPointerException("writer cannot be null");
+		this.stream = stream;
+		this.stream.start();
+		this.writer = writer;
+	}
+
+	@Override
+	public void close(TaskAttemptContext context) throws IOException,
+			InterruptedException {
+		this.stream.finish();
+		this.writer.close();
+	}
+
+	@Override
+	public void write(TKey key, TValue value) throws IOException,
+			InterruptedException {
+		this.sendOutput(key, value, this.stream);
+	}
+
+	/**
+	 * Method that handles an actual key value pair passing it to the
+	 * {@link StreamRDF} instance as appropriate
+	 * 
+	 * @param key
+	 *            Key
+	 * @param value
+	 *            Value
+	 * @param stream
+	 *            RDF Stream
+	 */
+	protected abstract void sendOutput(TKey key, TValue value, StreamRDF stream);
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java
new file mode 100644
index 0000000..d48546b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * An abstract implementation of a record writer that writes records to whole
+ * file formats.
+ * <p>
+ * It is important to note that the writer does not actually write any output
+ * until the {@link #close(TaskAttemptContext)} method is called as it must
+ * write the entire output in one go otherwise the output would be invalid. Also
+ * writing in one go increases the chances that the writer will be able to
+ * effectively use the syntax compressions of the RDF serialization being used.
+ * </p>
+ * <p>
+ * The implementation only writes the value portion of the key value pair since
+ * it is the value portion that is used to convey the node tuples
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ * @param <TValue>
+ * @param <T>
+ */
+public abstract class AbstractWholeFileNodeTupleWriter<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        RecordWriter<TKey, T> {
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractWholeFileNodeTupleWriter.class);
+
+    private Writer writer;
+
+    protected AbstractWholeFileNodeTupleWriter(Writer writer) {
+        if (writer == null)
+            throw new NullPointerException("writer cannot be null");
+        this.writer = writer;
+    }
+
+    @Override
+    public final void write(TKey key, T value) throws IOException, InterruptedException {
+        LOG.debug("write({}={})", key, value);
+        this.add(value);
+    }
+
+    /**
+     * Adds the tuple to the cache of tuples that will be written when the
+     * {@link #close(TaskAttemptContext)} method is called
+     * 
+     * @param value
+     */
+    protected abstract void add(T value);
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+        if (this.writer != null) {
+            this.writeOutput(writer);
+            this.writer.close();
+            this.writer = null;
+        }
+    }
+
+    /**
+     * Writes the cached tuples to the writer, the writer should not be closed
+     * by this method implementation
+     * 
+     * @param writer
+     *            Writer
+     */
+    protected abstract void writeOutput(Writer writer);
+
+}