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/10 17:46:31 UTC

jena git commit: Add support for JSON-LD to RDF Tools for Hadoop

Repository: jena
Updated Branches:
  refs/heads/hadoop-rdf ab54442e9 -> 14f995f16


Add support for JSON-LD to RDF Tools for Hadoop

Supports JSON-LD as both triples and quads input.  Some test cases are
currently disabled or slow running due to JENA-813


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

Branch: refs/heads/hadoop-rdf
Commit: 14f995f1651ded8b901c46a891f9748aff9fe762
Parents: ab54442
Author: Rob Vesse <rv...@apache.org>
Authored: Mon Nov 10 16:45:30 2014 +0000
Committer: Rob Vesse <rv...@apache.org>
Committed: Mon Nov 10 16:45:30 2014 +0000

----------------------------------------------------------------------
 .../rdf/io/input/JsonLDQuadInputFormat.java     | 20 ++++++
 .../rdf/io/input/JsonLDTripleInputFormat.java   | 20 ++++++
 .../rdf/io/input/readers/JsonLDQuadReader.java  | 13 ++++
 .../io/input/readers/JsonLDTripleReader.java    | 11 +++
 .../hadoop/rdf/io/input/util/RdfIOUtils.java    | 10 ++-
 .../rdf/io/output/JsonLDQuadOutputFormat.java   | 24 +++++++
 .../rdf/io/output/JsonLDTripleOutputFormat.java | 24 +++++++
 .../hadoop/rdf/io/output/TriGOutputFormat.java  | 34 +++++----
 .../rdf/io/output/TurtleOutputFormat.java       | 34 +++++----
 .../rdf/io/output/writers/JsonLDQuadWriter.java | 19 +++++
 .../io/output/writers/JsonLDTripleWriter.java   | 19 +++++
 .../rdf/io/input/JsonLDQuadInputTest.java       | 49 +++++++++++++
 .../rdf/io/input/JsonLDTripleInputTest.java     | 49 +++++++++++++
 ...actCompressedJsonLDQuadInputFormatTests.java | 74 ++++++++++++++++++++
 ...tCompressedJsonLDTripleInputFormatTests.java | 74 ++++++++++++++++++++
 .../jsonld/BZippedJsonLDQuadInputTest.java      | 34 +++++++++
 .../jsonld/BZippedJsonLDTripleInputTest.java    | 34 +++++++++
 .../jsonld/DeflatedJsonLDQuadInputTest.java     | 34 +++++++++
 .../jsonld/DeflatedJsonLDTripleInputTest.java   | 34 +++++++++
 .../jsonld/GZippedJsonLDQuadInputTest.java      | 34 +++++++++
 .../jsonld/GZippedJsonLDTripleInputTest.java    | 34 +++++++++
 .../AbstractNodeTupleOutputFormatTests.java     |  8 +--
 .../rdf/io/output/JsonLdQuadOutputTest.java     | 66 +++++++++++++++++
 .../rdf/io/output/JsonLdTripleOutputTest.java   | 66 +++++++++++++++++
 jena-hadoop-rdf/pom.xml                         |  2 +-
 25 files changed, 773 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/JsonLDQuadInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/JsonLDQuadInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/JsonLDQuadInputFormat.java
new file mode 100644
index 0000000..7106716
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/JsonLDQuadInputFormat.java
@@ -0,0 +1,20 @@
+package org.apache.jena.hadoop.rdf.io.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.jena.hadoop.rdf.io.input.readers.JsonLDQuadReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+public class JsonLDQuadInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new JsonLDQuadReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/JsonLDTripleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/JsonLDTripleInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/JsonLDTripleInputFormat.java
new file mode 100644
index 0000000..3240081
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/JsonLDTripleInputFormat.java
@@ -0,0 +1,20 @@
+package org.apache.jena.hadoop.rdf.io.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.jena.hadoop.rdf.io.input.readers.JsonLDTripleReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+public class JsonLDTripleInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new JsonLDTripleReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/JsonLDQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/JsonLDQuadReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/JsonLDQuadReader.java
new file mode 100644
index 0000000..5c13201
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/JsonLDQuadReader.java
@@ -0,0 +1,13 @@
+package org.apache.jena.hadoop.rdf.io.input.readers;
+
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class JsonLDQuadReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.JSONLD;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/JsonLDTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/JsonLDTripleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/JsonLDTripleReader.java
new file mode 100644
index 0000000..c67954e
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/JsonLDTripleReader.java
@@ -0,0 +1,11 @@
+package org.apache.jena.hadoop.rdf.io.input.readers;
+
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class JsonLDTripleReader extends AbstractWholeFileTripleReader {
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.JSONLD;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
index f1c9dfe..06567e5 100644
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
@@ -22,7 +22,6 @@ import java.util.UUID;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
 import org.apache.jena.riot.lang.LabelToNode;
 import org.apache.jena.riot.system.ErrorHandlerFactory;
 import org.apache.jena.riot.system.IRIResolver;
@@ -32,7 +31,6 @@ import org.apache.jena.riot.system.Prologue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 /**
  * RDF IO utility functions
  * 
@@ -40,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * 
  */
 public class RdfIOUtils {
-    private static final Logger log = LoggerFactory.getLogger(RdfIOUtils.class);
+    private static final Logger LOGGER = LoggerFactory.getLogger(RdfIOUtils.class);
 
     /**
      * Private constructor prevents instantiation
@@ -79,11 +77,11 @@ public class RdfIOUtils {
         String jobId = context.getJobID().toString();
         if (jobId == null) {
             jobId = String.valueOf(System.currentTimeMillis());
-            log.warn(
-                    "Job ID was not set, using current milliseconds of {}. Sequence of MapReduce jobs must handle carefully blank nodes.",
+            LOGGER.warn(
+                    "Job ID was not set, using current milliseconds of {}. Sequence of MapReduce jobs must carefully handle blank nodes.",
                     jobId);
         }
-        log.debug("MapReduceAllocator({}, {})", jobId, path);
+        LOGGER.debug("Generating Blank Node Seed from Job Details (ID={}, Input Path={})", jobId, path);
 
         // Form a reproducible seed for the run
         return new UUID(jobId.hashCode(), path.hashCode());

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/JsonLDQuadOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/JsonLDQuadOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/JsonLDQuadOutputFormat.java
new file mode 100644
index 0000000..e8fe3ca
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/JsonLDQuadOutputFormat.java
@@ -0,0 +1,24 @@
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.writers.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) {
+        return new JsonLDQuadWriter<TKey>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/JsonLDTripleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/JsonLDTripleOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/JsonLDTripleOutputFormat.java
new file mode 100644
index 0000000..100d63c
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/JsonLDTripleOutputFormat.java
@@ -0,0 +1,24 @@
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.output.writers.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) {
+        return new JsonLDTripleWriter<TKey>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriGOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriGOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriGOutputFormat.java
index 218ac87..bd2e436 100644
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriGOutputFormat.java
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriGOutputFormat.java
@@ -37,23 +37,21 @@ import com.hp.hpl.jena.sparql.core.Quad;
  * @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";
-	}
+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/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TurtleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TurtleOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TurtleOutputFormat.java
index cb340a3..0b3c5e9 100644
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TurtleOutputFormat.java
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TurtleOutputFormat.java
@@ -37,23 +37,21 @@ import com.hp.hpl.jena.graph.Triple;
  * @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);
-	}
+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/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/JsonLDQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/JsonLDQuadWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/JsonLDQuadWriter.java
new file mode 100644
index 0000000..113441b
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/JsonLDQuadWriter.java
@@ -0,0 +1,19 @@
+package org.apache.jena.hadoop.rdf.io.output.writers;
+
+import java.io.Writer;
+
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class JsonLDQuadWriter<TKey> extends AbstractWholeFileQuadWriter<TKey> {
+
+    public JsonLDQuadWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.JSONLD;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/JsonLDTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/JsonLDTripleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/JsonLDTripleWriter.java
new file mode 100644
index 0000000..b6a1d46
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/JsonLDTripleWriter.java
@@ -0,0 +1,19 @@
+package org.apache.jena.hadoop.rdf.io.output.writers;
+
+import java.io.Writer;
+
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class JsonLDTripleWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
+
+    public JsonLDTripleWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.JSONLD;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/JsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/JsonLDQuadInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/JsonLDQuadInputTest.java
new file mode 100644
index 0000000..200f1ac
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/JsonLDQuadInputTest.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for JSON-LD input
+ * 
+ *
+ */
+public class JsonLDQuadInputTest extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.JSONLD;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".jsonld";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new JsonLDQuadInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/JsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/JsonLDTripleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/JsonLDTripleInputTest.java
new file mode 100644
index 0000000..dcef2c0
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/JsonLDTripleInputTest.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for JSON-LD input
+ * 
+ *
+ */
+public class JsonLDTripleInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.JSONLD;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".jsonld";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new JsonLDTripleInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
new file mode 100644
index 0000000..1af1656
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.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.input.compressed.jsonld;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.JsonLDQuadInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed JSON-LD input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedJsonLDQuadInputFormatTests extends AbstractCompressedWholeFileQuadInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedJsonLDQuadInputFormatTests(String ext, CompressionCodec codec) {
+        this.ext = ext;
+        this.codec = codec;
+    }
+
+    @Override
+    protected final String getFileExtension() {
+        return this.ext;
+    }
+
+    @Override
+    protected final CompressionCodec getCompressionCodec() {
+        return this.codec;
+    }
+
+    @Override
+    protected final Lang getRdfLanguage() {
+        return Lang.JSONLD;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new JsonLDQuadInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
new file mode 100644
index 0000000..3927e21
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.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.input.compressed.jsonld;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.JsonLDTripleInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed JSON-LD input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedJsonLDTripleInputFormatTests extends AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedJsonLDTripleInputFormatTests(String ext, CompressionCodec codec) {
+        this.ext = ext;
+        this.codec = codec;
+    }
+
+    @Override
+    protected final String getFileExtension() {
+        return this.ext;
+    }
+
+    @Override
+    protected final CompressionCodec getCompressionCodec() {
+        return this.codec;
+    }
+
+    @Override
+    protected final Lang getRdfLanguage() {
+        return Lang.JSONLD;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new JsonLDTripleInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java
new file mode 100644
index 0000000..e5e7066
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.compressed.jsonld;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped JSON-LD input
+ */
+public class BZippedJsonLDQuadInputTest extends AbstractCompressedJsonLDQuadInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedJsonLDQuadInputTest() {
+        super(".jsonld.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java
new file mode 100644
index 0000000..8d2e122
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.compressed.jsonld;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped JSON-LD input
+ */
+public class BZippedJsonLDTripleInputTest extends AbstractCompressedJsonLDTripleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedJsonLDTripleInputTest() {
+        super(".jsonld.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java
new file mode 100644
index 0000000..292b17f
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.compressed.jsonld;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated JSON-LD input
+ */
+public class DeflatedJsonLDQuadInputTest extends AbstractCompressedJsonLDQuadInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedJsonLDQuadInputTest() {
+        super(".jsonld.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java
new file mode 100644
index 0000000..e5edd6a
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.compressed.jsonld;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated JSON-LD input
+ */
+public class DeflatedJsonLDTripleInputTest extends AbstractCompressedJsonLDTripleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedJsonLDTripleInputTest() {
+        super(".jsonld.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java
new file mode 100644
index 0000000..0a4a240
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.compressed.jsonld;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped JSON-LD input
+ */
+public class GZippedJsonLDQuadInputTest extends AbstractCompressedJsonLDQuadInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedJsonLDQuadInputTest() {
+        super(".jsonld.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java
new file mode 100644
index 0000000..8a6324d
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.compressed.jsonld;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped JSON-LD input
+ */
+public class GZippedJsonLDTripleInputTest extends AbstractCompressedJsonLDTripleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedJsonLDTripleInputTest() {
+        super(".jsonld.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
index 31c3a67..adbea48 100644
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
@@ -213,7 +213,7 @@ public abstract class AbstractNodeTupleOutputFormatTests<TValue, T extends Abstr
      * @throws InterruptedException 
      */
     @Test
-    public final void output_01() throws IOException, InterruptedException {
+    public void output_01() throws IOException, InterruptedException {
         this.testOutput(this.folder.getRoot(), EMPTY_SIZE);
     }
     
@@ -224,7 +224,7 @@ public abstract class AbstractNodeTupleOutputFormatTests<TValue, T extends Abstr
      * @throws InterruptedException 
      */
     @Test
-    public final void output_02() throws IOException, InterruptedException {
+    public void output_02() throws IOException, InterruptedException {
         this.testOutput(this.folder.getRoot(), SMALL_SIZE);
     }
     
@@ -235,7 +235,7 @@ public abstract class AbstractNodeTupleOutputFormatTests<TValue, T extends Abstr
      * @throws InterruptedException 
      */
     @Test
-    public final void output_03() throws IOException, InterruptedException {
+    public void output_03() throws IOException, InterruptedException {
         this.testOutput(this.folder.getRoot(), LARGE_SIZE);
     }
     
@@ -246,7 +246,7 @@ public abstract class AbstractNodeTupleOutputFormatTests<TValue, T extends Abstr
      * @throws InterruptedException 
      */
     @Test
-    public final void output_04() throws IOException, InterruptedException {
+    public void output_04() throws IOException, InterruptedException {
         this.testOutput(this.folder.getRoot(), VERY_LARGE_SIZE);
     }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/JsonLdQuadOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/JsonLdQuadOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/JsonLdQuadOutputTest.java
new file mode 100644
index 0000000..23873d3
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/JsonLdQuadOutputTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * Tests for JSON-LD output format
+ */
+public class JsonLdQuadOutputTest extends AbstractQuadOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".jsonld";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.JSONLD;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
+        return new JsonLDQuadOutputFormat<NullWritable>();
+    }
+
+    @Override
+    @Test
+    @Ignore //Ignored because of JENA-813 performance effect being compounded due to the nature of how JSON-LD is written
+    public void output_03() throws IOException, InterruptedException {
+        // TODO Auto-generated method stub
+        super.output_03();
+    }
+
+    @Override
+    @Test
+    @Ignore //Ignored because of JENA-813 performance effect being compounded due to the nature of how JSON-LD is written
+    public void output_04() throws IOException, InterruptedException {
+        // TODO Auto-generated method stub
+        super.output_04();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/JsonLdTripleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/JsonLdTripleOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/JsonLdTripleOutputTest.java
new file mode 100644
index 0000000..397c55b
--- /dev/null
+++ b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/JsonLdTripleOutputTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * Tests for JSON-LD output format
+ */
+public class JsonLdTripleOutputTest extends AbstractTripleOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".jsonld";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.JSONLD;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new JsonLDTripleOutputFormat<NullWritable>();
+    }
+    
+    @Override
+    @Test
+    @Ignore //Ignored because of JENA-813 performance effect being compounded due to the nature of how JSON-LD is written
+    public void output_03() throws IOException, InterruptedException {
+        // TODO Auto-generated method stub
+        super.output_03();
+    }
+
+    @Override
+    @Test
+    @Ignore //Ignored because of JENA-813 performance effect being compounded due to the nature of how JSON-LD is written
+    public void output_04() throws IOException, InterruptedException {
+        // TODO Auto-generated method stub
+        super.output_04();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/14f995f1/jena-hadoop-rdf/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/pom.xml b/jena-hadoop-rdf/pom.xml
index dd897d7..dc613de 100644
--- a/jena-hadoop-rdf/pom.xml
+++ b/jena-hadoop-rdf/pom.xml
@@ -35,7 +35,7 @@
   <!-- Properties common across all profiles -->
   <properties>
     <plugin.compiler.version>2.5.1</plugin.compiler.version>
-    <arq.version>2.12.1</arq.version>
+    <arq.version>2.12.2-SNAPSHOT</arq.version>
     <junit.version>4.11</junit.version>
     <mrunit.version>1.0.0</mrunit.version>
   </properties>