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

[01/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Repository: jena
Updated Branches:
  refs/heads/hadoop-rdf d55492a2e -> 49c4cffe6


http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java b/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
new file mode 100644
index 0000000..5f870ee
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
@@ -0,0 +1,405 @@
+/*
+ * 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.stats;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Help;
+import io.airlift.command.HelpOption;
+import io.airlift.command.Option;
+import io.airlift.command.OptionType;
+import io.airlift.command.ParseArgumentsMissingException;
+import io.airlift.command.ParseArgumentsUnexpectedException;
+import io.airlift.command.ParseException;
+import io.airlift.command.ParseOptionMissingException;
+import io.airlift.command.ParseOptionMissingValueException;
+import io.airlift.command.SingleCommand;
+import io.airlift.command.model.CommandMetadata;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import javax.inject.Inject;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.jena.hadoop.rdf.stats.jobs.JobFactory;
+
+
+/**
+ * Entry point for the Hadoop job, handles launching all the relevant Hadoop
+ * jobs
+ */
+@Command(name = "bin/hadoop jar PATH_TO_JAR com.yarcdata.urika.hadoop.rdf.stats.RdfStats", description = "A command which computes statistics on RDF data using Hadoop")
+public class RdfStats implements Tool {
+
+    static final String ANSI_RED = "\u001B[31m";
+    static final String ANSI_RESET = "\u001B[0m";
+
+    private static final String DATA_TYPE_TRIPLES = "triples", DATA_TYPE_QUADS = "quads", DATA_TYPE_MIXED = "mixed";
+
+    /**
+     * Help option
+     */
+    @Inject
+    public HelpOption helpOption;
+
+    /**
+     * Gets/Sets whether all available statistics will be calculated
+     */
+    @Option(name = { "-a", "--all" }, description = "Requests that all available statistics be calculated", type = OptionType.COMMAND)
+    public boolean all = false;
+
+    /**
+     * Gets/Sets whether node usage counts will be calculated
+     */
+    @Option(name = { "-n", "--node-count" }, description = "Requests that node usage counts be calculated", type = OptionType.COMMAND)
+    public boolean nodeCount = false;
+
+    /**
+     * Gets/Sets whether characteristic sets will be calculated
+     */
+    @Option(name = { "-c", "--characteristic-sets" }, description = "Requests that characteristic sets be calculated", type = OptionType.COMMAND)
+    public boolean characteristicSets = false;
+
+    /**
+     * Gets/Sets whether type counts will be calculated
+     */
+    @Option(name = { "-t", "--type-counts" }, description = "Requests that rdf:type usage counts be calculated", type = OptionType.COMMAND)
+    public boolean typeCount = false;
+
+    /**
+     * Gets/Sets whether data type counts will be calculated
+     */
+    @Option(name = { "-d", "--data-types" }, description = "Requests that literal data type usage counts be calculated", type = OptionType.COMMAND)
+    public boolean dataTypeCount = false;
+
+    /**
+     * Gets/Sets whether namespace counts will be calculated
+     */
+    @Option(name = { "--namespaces" }, description = "Requests that namespace usage counts be calculated", type = OptionType.COMMAND)
+    public boolean namespaceCount = false;
+
+    /**
+     * Gets/Sets the input data type used
+     */
+    @Option(name = { "--input-type" }, allowedValues = { DATA_TYPE_MIXED, DATA_TYPE_QUADS, DATA_TYPE_TRIPLES }, description = "Specifies whether the input data is a mixture of quads and triples, just quads or just triples.  Using the most specific data type will yield the most accurrate statistics")
+    public String inputType = DATA_TYPE_MIXED;
+
+    /**
+     * Gets/Sets the output path
+     */
+    @Option(name = { "-o", "--output" }, title = "OutputPath", description = "Sets the output path", arity = 1, required = true)
+    public String outputPath = null;
+
+    /**
+     * Gets/Sets the input path(s)
+     */
+    @Arguments(description = "Sets the input path(s)", title = "InputPath", required = true)
+    public List<String> inputPaths = new ArrayList<String>();
+
+    private Configuration config;
+
+    /**
+     * Entry point method
+     * 
+     * @param args
+     *            Arguments
+     * @throws Exception
+     */
+    public static void main(String[] args) throws Exception {
+        try {
+            // Run and exit with result code if no errors bubble up
+            // Note that the exit code may still be a error code
+            int res = ToolRunner.run(new Configuration(true), new RdfStats(), args);
+            System.exit(res);
+        } catch (Exception e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            e.printStackTrace(System.err);
+        } finally {
+            System.err.print(ANSI_RESET);
+        }
+        // If any errors bubble up exit with non-zero code
+        System.exit(1);
+    }
+
+    private static void showUsage() {
+        CommandMetadata metadata = SingleCommand.singleCommand(RdfStats.class).getCommandMetadata();
+        StringBuilder builder = new StringBuilder();
+        Help.help(metadata, builder);
+        System.err.print(ANSI_RESET);
+        System.err.println(builder.toString());
+        System.exit(1);
+    }
+
+    @Override
+    public void setConf(Configuration conf) {
+        this.config = conf;
+    }
+
+    @Override
+    public Configuration getConf() {
+        return this.config;
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+        try {
+            // Parse custom arguments
+            RdfStats cmd = SingleCommand.singleCommand(RdfStats.class).parse(args);
+
+            // Copy Hadoop configuration across
+            cmd.setConf(this.getConf());
+
+            // Show help if requested and exit with success
+            if (cmd.helpOption.showHelpIfRequested()) {
+                return 0;
+            }
+
+            // Run the command and exit with success
+            cmd.run();
+            return 0;
+
+        } catch (ParseOptionMissingException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+        } catch (ParseOptionMissingValueException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+        } catch (ParseArgumentsMissingException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+        } catch (ParseArgumentsUnexpectedException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+            // TODO Re-enable as and when we upgrade Airline
+            // } catch (ParseOptionIllegalValueException e) {
+            // System.err.println(ANSI_RED + e.getMessage());
+            // System.err.println();
+            // showUsage();
+        } catch (ParseException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+        } catch (UnsupportedOperationException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+        } catch (Throwable e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            e.printStackTrace(System.err);
+        } finally {
+            System.err.print(ANSI_RESET);
+        }
+        return 1;
+    }
+
+    private void run() throws Throwable {
+        if (!this.outputPath.endsWith("/")) {
+            this.outputPath += "/";
+        }
+
+        // If all statistics requested turn on all statistics
+        if (this.all) {
+            this.nodeCount = true;
+            this.characteristicSets = true;
+            this.typeCount = true;
+            this.dataTypeCount = true;
+            this.namespaceCount = true;
+        }
+
+        // How many statistics were requested?
+        int statsRequested = 0;
+        if (this.nodeCount)
+            statsRequested++;
+        if (this.characteristicSets)
+            statsRequested++;
+        if (this.typeCount)
+            statsRequested++;
+        if (this.dataTypeCount)
+            statsRequested++;
+        if (this.namespaceCount)
+            statsRequested++;
+
+        // Error if no statistics requested
+        if (statsRequested == 0) {
+            System.err
+                    .println("You did not request any statistics to be calculated, please use one/more of the relevant options to select the statistics to be computed");
+            return;
+        }
+        int statsComputed = 1;
+
+        // Compute statistics
+        if (this.nodeCount) {
+            Job job = this.selectNodeCountJob();
+            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
+        }
+        if (this.typeCount) {
+            Job[] jobs = this.selectTypeCountJobs();
+            statsComputed = this.computeStatistic(jobs, false, false, statsComputed, statsRequested);
+        }
+        if (this.dataTypeCount) {
+            Job job = this.selectDataTypeCountJob();
+            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
+        }
+        if (this.namespaceCount) {
+            Job job = this.selectNamespaceCountJob();
+            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
+        }
+        if (this.characteristicSets) {
+            Job[] jobs = this.selectCharacteristicSetJobs();
+            statsComputed = this.computeStatistic(jobs, false, false, statsComputed, statsRequested);
+        }
+    }
+
+    private int computeStatistic(Job job, int statsComputed, int statsRequested) throws Throwable {
+        System.out.println(String.format("Computing Statistic %d of %d requested", statsComputed, statsRequested));
+        this.runJob(job);
+        System.out.println(String.format("Computed Statistic %d of %d requested", statsComputed, statsRequested));
+        System.out.println();
+        return ++statsComputed;
+    }
+
+    private int computeStatistic(Job[] jobs, boolean continueOnFailure, boolean continueOnError, int statsComputed,
+            int statsRequested) {
+        System.out.println(String.format("Computing Statistic %d of %d requested", statsComputed, statsRequested));
+        this.runJobSequence(jobs, continueOnFailure, continueOnError);
+        System.out.println(String.format("Computed Statistic %d of %d requested", statsComputed, statsRequested));
+        System.out.println();
+        return ++statsComputed;
+    }
+
+    private boolean runJob(Job job) throws Throwable {
+        System.out.println("Submitting Job " + job.getJobName());
+        long start = System.nanoTime();
+        try {
+            job.submit();
+            if (job.monitorAndPrintJob()) {
+                System.out.println("Job " + job.getJobName() + " succeeded");
+                return true;
+            } else {
+                System.out.println("Job " + job.getJobName() + " failed");
+                return false;
+            }
+        } catch (Throwable e) {
+            System.out.println("Unexpected failure in Job " + job.getJobName());
+            throw e;
+        } finally {
+            long end = System.nanoTime();
+            System.out.println("Job " + job.getJobName() + " finished after "
+                    + String.format("%,d milliseconds", TimeUnit.NANOSECONDS.toMillis(end - start)));
+            System.out.println();
+        }
+    }
+
+    private void runJobSequence(Job[] jobs, boolean continueOnFailure, boolean continueOnError) {
+        for (int i = 0; i < jobs.length; i++) {
+            Job job = jobs[i];
+            try {
+                boolean success = this.runJob(job);
+                if (!success && !continueOnFailure)
+                    throw new IllegalStateException("Unable to complete job sequence because Job " + job.getJobName() + " failed");
+            } catch (IllegalStateException e) {
+                throw e;
+            } catch (Throwable e) {
+                if (!continueOnError)
+                    throw new IllegalStateException("Unable to complete job sequence because job " + job.getJobName()
+                            + " errorred", e);
+            }
+        }
+    }
+
+    private Job selectNodeCountJob() throws IOException {
+        String realOutputPath = outputPath + "node-counts/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadNodeCountJob(this.config, inputs, realOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleNodeCountJob(this.config, inputs, realOutputPath);
+        } else {
+            return JobFactory.getNodeCountJob(this.config, inputs, realOutputPath);
+        }
+    }
+
+    private Job selectDataTypeCountJob() throws IOException {
+        String realOutputPath = outputPath + "data-type-counts/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadDataTypeCountJob(this.config, inputs, realOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleDataTypeCountJob(this.config, inputs, realOutputPath);
+        } else {
+            return JobFactory.getDataTypeCountJob(this.config, inputs, realOutputPath);
+        }
+    }
+
+    private Job selectNamespaceCountJob() throws IOException {
+        String realOutputPath = outputPath + "namespace-counts/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadNamespaceCountJob(this.config, inputs, realOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleNamespaceCountJob(this.config, inputs, realOutputPath);
+        } else {
+            return JobFactory.getNamespaceCountJob(this.config, inputs, realOutputPath);
+        }
+    }
+
+    private Job[] selectCharacteristicSetJobs() throws IOException {
+        String intermediateOutputPath = outputPath + "characteristics/intermediate/";
+        String finalOutputPath = outputPath + "characteristics/final/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        } else {
+            return JobFactory.getCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        }
+    }
+
+    private Job[] selectTypeCountJobs() throws IOException {
+        String intermediateOutputPath = outputPath + "type-declarations/";
+        String finalOutputPath = outputPath + "type-counts/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        } else {
+            return JobFactory.getTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java b/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
new file mode 100644
index 0000000..55bb8af
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
@@ -0,0 +1,757 @@
+/*
+ * 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.stats.jobs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.BZip2Codec;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.jena.hadoop.rdf.io.input.QuadsInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.TriplesInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.TriplesOrQuadsInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.nquads.NQuadsInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.nquads.NQuadsOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesNodeOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesOutputFormat;
+import org.apache.jena.hadoop.rdf.mapreduce.KeyMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.characteristics.CharacteristicSetReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.characteristics.QuadCharacteristicSetGeneratingReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.characteristics.TripleCharacteristicSetGeneratingReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.datatypes.QuadDataTypeCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.datatypes.TripleDataTypeCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.namespaces.QuadNamespaceCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.namespaces.TripleNamespaceCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.positional.QuadObjectCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.positional.TripleObjectCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.QuadFilterByPredicateMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupBySubjectMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupBySubjectMapper;
+import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
+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 com.hp.hpl.jena.vocabulary.RDF;
+
+/**
+ * Factory that can produce {@link Job} instances for computing various RDF
+ * statistics
+ * 
+ * 
+ * 
+ */
+public class JobFactory {
+
+    /**
+     * Private constructor prevents instantiation
+     */
+    private JobFactory() {
+    }
+
+    /**
+     * Gets a job for computing node counts on RDF triple inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getTripleNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Node Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleNodeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing node counts on RDF quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getQuadNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Node Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadNodeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing node counts on RDF triple and/or quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Node Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadNodeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a sequence of jobs that can be used to compute characteristic sets
+     * for RDF triples
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Intermediate output path
+     * @param outputPath
+     *            Final output path
+     * @return Sequence of jobs
+     * @throws IOException
+     */
+    public static Job[] getTripleCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Characteristic Set (Generation)");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleGroupBySubjectMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(TripleWritable.class);
+        job.setReducerClass(TripleCharacteristicSetGeneratingReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(NullWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+        SequenceFileOutputFormat.setCompressOutput(job, true);
+        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
+        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
+
+        jobs[0] = job;
+
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Characteristic Set (Reduction)");
+
+        // Map/Reduce classes
+        job.setMapperClass(KeyMapper.class);
+        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
+        job.setMapOutputValueClass(CharacteristicSetWritable.class);
+        job.setReducerClass(CharacteristicSetReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(CharacteristicSetWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(SequenceFileInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+        return jobs;
+    }
+
+    /**
+     * Gets a sequence of jobs that can be used to compute characteristic sets
+     * for RDF quads
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Intermediate output path
+     * @param outputPath
+     *            Final output path
+     * @return Sequence of jobs
+     * @throws IOException
+     */
+    public static Job[] getQuadCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Characteristic Set (Generation)");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadGroupBySubjectMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(QuadWritable.class);
+        job.setReducerClass(QuadCharacteristicSetGeneratingReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(NullWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+        SequenceFileOutputFormat.setCompressOutput(job, true);
+        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
+        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
+
+        jobs[0] = job;
+
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Characteristic Set (Reduction)");
+
+        // Map/Reduce classes
+        job.setMapperClass(KeyMapper.class);
+        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
+        job.setMapOutputValueClass(CharacteristicSetWritable.class);
+        job.setReducerClass(CharacteristicSetReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(CharacteristicSetWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(SequenceFileInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+        return jobs;
+    }
+
+    /**
+     * Gets a sequence of jobs that can be used to compute characteristic sets
+     * for RDF triple and/or quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Intermediate output path
+     * @param outputPath
+     *            Final output path
+     * @return Sequence of jobs
+     * @throws IOException
+     */
+    public static Job[] getCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Characteristic Set (Generation)");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadGroupBySubjectMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(QuadWritable.class);
+        job.setReducerClass(QuadCharacteristicSetGeneratingReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(NullWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+        SequenceFileOutputFormat.setCompressOutput(job, true);
+        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
+        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
+
+        jobs[0] = job;
+
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Characteristic Set (Reduction)");
+
+        // Map/Reduce classes
+        job.setMapperClass(KeyMapper.class);
+        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
+        job.setMapOutputValueClass(CharacteristicSetWritable.class);
+        job.setReducerClass(CharacteristicSetReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(CharacteristicSetWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(SequenceFileInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+        return jobs;
+    }
+
+    /**
+     * Gets a job for computing type counts on RDF triple inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Path for intermediate output which will be all the type
+     *            declaration triples present in the inputs
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job[] getTripleTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Type Triples Extraction");
+
+        // Map/Reduce classes
+        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
+        job.setMapperClass(TripleFilterByPredicateUriMapper.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(TripleWritable.class);
+
+        // Input and Output Format
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(NTriplesOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+
+        jobs[0] = job;
+
+        // Object Node Usage count job
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleObjectCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(NTriplesInputFormat.class);
+        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
+                                                          // better if this was
+                                                          // intelligently
+                                                          // configured
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+
+        return jobs;
+    }
+
+    /**
+     * Gets a job for computing type counts on RDF quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Path for intermediate output which will be all the type
+     *            declaration quads present in the inputs
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job[] getQuadTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Type Quads Extraction");
+
+        // Map/Reduce classes
+        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
+        job.setMapperClass(QuadFilterByPredicateMapper.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(QuadWritable.class);
+
+        // Input and Output Format
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(NQuadsOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+
+        jobs[0] = job;
+
+        // Object Node Usage count job
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadObjectCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(NQuadsInputFormat.class);
+        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
+                                                          // better if this was
+                                                          // intelligently
+                                                          // configured
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+
+        return jobs;
+    }
+
+    /**
+     * Gets a job for computing type counts on RDF triple and/or quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Path for intermediate output which will be all the type
+     *            declaration quads present in the inputs
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job[] getTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Type Extraction");
+
+        // Map/Reduce classes
+        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
+        job.setMapperClass(QuadFilterByPredicateMapper.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(QuadWritable.class);
+
+        // Input and Output Format
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(NQuadsOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+
+        jobs[0] = job;
+
+        // Object Node Usage count job
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadObjectCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(NQuadsInputFormat.class);
+        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
+                                                          // better if this was
+                                                          // intelligently
+                                                          // configured
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+
+        return jobs;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF triple inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getTripleDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Literal Data Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleDataTypeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getQuadDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Literal Data Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadDataTypeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF triple and/or
+     * quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Literal Data Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadDataTypeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF triple inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getTripleNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Namespace Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleNamespaceCountMapper.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(TextCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getQuadNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Namespace Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadNamespaceCountMapper.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(TextCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF triple and/or
+     * quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Namespace Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadNamespaceCountMapper.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(TextCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/pom.xml b/jena-hadoop-rdf/pom.xml
index dc613de..83f2819 100644
--- a/jena-hadoop-rdf/pom.xml
+++ b/jena-hadoop-rdf/pom.xml
@@ -11,7 +11,7 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
   xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
-  <artifactId>jena-hadoop-rdf</artifactId>
+  <artifactId>jena-elephas</artifactId>
   <version>0.9.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
@@ -22,14 +22,14 @@
     <relativePath>../jena-parent</relativePath>
   </parent>
 
-  <name>Apache Jena - RDF Tools for Hadoop</name>
+  <name>Apache Jena - Elephas</name>
   <description>A collection of tools for working with RDF on the Hadoop platform</description>
 
   <modules>
-    <module>hadoop-rdf-io</module>
-    <module>hadoop-rdf-common</module>
-    <module>hadoop-rdf-mapreduce</module>
-    <module>hadoop-rdf-stats</module>
+    <module>jena-elephas-io</module>
+    <module>jena-elephas-common</module>
+    <module>jena-elephas-mapreduce</module>
+    <module>jena-elephas-stats</module>
   </modules>
 
   <!-- Properties common across all profiles -->
@@ -42,22 +42,14 @@
 
 	<!-- Profiles to allow building for different Hadoop versions -->
   <profiles>
-    <!-- Hadoop 2.x Stable -->
+    <!-- Hadoop 2.x Latest -->
     <profile>
       <id>hadoop_2x</id>
       <activation>
         <activeByDefault>true</activeByDefault>
       </activation>
       <properties>
-        <hadoop.version>2.5.1</hadoop.version>
-      </properties>
-    </profile>
-
-    <!-- Hadoop 0.23 -->
-    <profile>
-      <id>hadoop_023x</id>
-      <properties>
-        <hadoop.version>0.23.11</hadoop.version>
+        <hadoop.version>2.5.0</hadoop.version>
       </properties>
     </profile>
   </profiles>
@@ -75,7 +67,7 @@
         <artifactId>hadoop-mapreduce-client-common</artifactId>
         <version>${hadoop.version}</version>
       </dependency>
-
+      
       <!-- Jena Dependencies -->
       <dependency>
         <groupId>org.apache.jena</groupId>


[06/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
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/registry/writers/AbstractQuadsOnlyWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
new file mode 100644
index 0000000..0cf137e
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
@@ -0,0 +1,86 @@
+/*
+ * 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.registry.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract writer factory for languages that only support quads
+ */
+public abstract class AbstractQuadsOnlyWriterFactory implements WriterFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang> emptyList());
+
+    public AbstractQuadsOnlyWriterFactory(Lang lang) {
+        this(lang, (Collection<Lang>) null);
+    }
+
+    public AbstractQuadsOnlyWriterFactory(Lang lang, Lang... altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractQuadsOnlyWriterFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canWriteQuads() {
+        return true;
+    }
+
+    @Override
+    public final boolean canWriteTriples() {
+        return false;
+    }
+
+    @Override
+    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException;
+
+    @Override
+    public final <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        throw new IOException(this.lang.getName() + " does not support writing triples");
+    }
+
+}

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/registry/writers/AbstractTriplesOnlyWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
new file mode 100644
index 0000000..e45c3da
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
@@ -0,0 +1,85 @@
+/*
+ * 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.registry.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract writer factory for languages that only support triples
+ */
+public abstract class AbstractTriplesOnlyWriterFactory implements WriterFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang> emptyList());
+
+    public AbstractTriplesOnlyWriterFactory(Lang lang) {
+        this(lang, (Collection<Lang>) null);
+    }
+
+    public AbstractTriplesOnlyWriterFactory(Lang lang, Lang... altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractTriplesOnlyWriterFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canWriteQuads() {
+        return false;
+    }
+
+    @Override
+    public final boolean canWriteTriples() {
+        return true;
+    }
+
+    @Override
+    public final <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        throw new IOException(this.lang.getName() + " does not support writing quads");
+    }
+
+    @Override
+    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException;
+
+}

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/registry/writers/AbstractWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
new file mode 100644
index 0000000..669b9c4
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
@@ -0,0 +1,82 @@
+/*
+ * 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.registry.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract writer factory for languages that support triples and quads
+ */
+public abstract class AbstractWriterFactory implements WriterFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
+
+    public AbstractWriterFactory(Lang lang) {
+        this(lang, (Collection<Lang>)null);
+    }
+    
+    public AbstractWriterFactory(Lang lang, Lang...altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractWriterFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+    
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canWriteQuads() {
+        return true;
+    }
+
+    @Override
+    public final boolean canWriteTriples() {
+        return true;
+    }
+
+    @Override
+    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config) throws IOException;
+
+    @Override
+    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config) throws IOException;
+
+}

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/registry/writers/JsonLDWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java
new file mode 100644
index 0000000..89e93ed
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.jsonld.JsonLDQuadWriter;
+import org.apache.jena.hadoop.rdf.io.output.writers.jsonld.JsonLDTripleWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class JsonLDWriterFactory extends AbstractWriterFactory {
+    
+    public JsonLDWriterFactory() {
+        super(Lang.JSONLD);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new JsonLDQuadWriter<>(writer);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new JsonLDTripleWriter<>(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/registry/writers/NQuadsWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java
new file mode 100644
index 0000000..abbbd0f
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.nquads.NQuadsWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class NQuadsWriterFactory extends AbstractQuadsOnlyWriterFactory {
+    
+    public NQuadsWriterFactory() {
+        super(Lang.NQUADS, Lang.NQ);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new NQuadsWriter<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/registry/writers/NTriplesWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java
new file mode 100644
index 0000000..88c9551
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.ntriples.NTriplesWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class NTriplesWriterFactory extends AbstractTriplesOnlyWriterFactory {
+    
+    public NTriplesWriterFactory() {
+        super(Lang.NTRIPLES, Lang.NT);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new NTriplesWriter<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/registry/writers/RdfJsonWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
new file mode 100644
index 0000000..8252422
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.writers;
+
+import java.io.IOException;
+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.rdfjson.RdfJsonWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class RdfJsonWriterFactory extends AbstractTriplesOnlyWriterFactory {
+
+    public RdfJsonWriterFactory() {
+        super(Lang.RDFJSON);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        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/registry/writers/RdfXmlWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java
new file mode 100644
index 0000000..b4ac8e3
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.rdfxml.RdfXmlWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class RdfXmlWriterFactory extends AbstractTriplesOnlyWriterFactory {
+    
+    public RdfXmlWriterFactory() {
+        super(Lang.RDFXML);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        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/registry/writers/ThriftWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java
new file mode 100644
index 0000000..757472c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.writers.StreamRdfQuadWriter;
+import org.apache.jena.hadoop.rdf.io.output.writers.StreamRdfTripleWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.RDFLanguages;
+import org.apache.jena.riot.thrift.StreamRDF2Thrift;
+
+/**
+ *
+ */
+public class ThriftWriterFactory extends AbstractWriterFactory {
+
+    public ThriftWriterFactory() {
+        super(RDFLanguages.THRIFT);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfQuadWriter<TKey>(new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")),
+                false), writer);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfTripleWriter<TKey>(new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")),
+                false), 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/registry/writers/TriGWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java
new file mode 100644
index 0000000..6d8b08a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.StreamRdfQuadWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
+
+/**
+ *
+ */
+public class TriGWriterFactory extends AbstractQuadsOnlyWriterFactory {
+    
+    public TriGWriterFactory() {
+        super(Lang.TRIG);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfQuadWriter<TKey>(new WriterStreamRDFBlocks(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/registry/writers/TriXWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
new file mode 100644
index 0000000..0e1b7b2
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.writers;
+
+import java.io.IOException;
+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.writers.StreamRdfQuadWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.writer.StreamWriterTriX;
+
+/**
+ *
+ */
+public class TriXWriterFactory extends AbstractQuadsOnlyWriterFactory {
+    
+    public TriXWriterFactory() {
+        super(Lang.TRIX);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfQuadWriter<>(new StreamWriterTriX(new WriterOutputStream(writer, Charset.forName("utf-8"))), 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/registry/writers/TurtleWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java
new file mode 100644
index 0000000..c837f12
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.StreamRdfTripleWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
+
+/**
+ *
+ */
+public class TurtleWriterFactory extends AbstractTriplesOnlyWriterFactory {
+    
+    public TurtleWriterFactory() {
+        super(Lang.TURTLE, Lang.TTL, Lang.N3);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfTripleWriter<>(new WriterStreamRDFBlocks(writer), writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory b/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
new file mode 100644
index 0000000..ec0e48a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
@@ -0,0 +1,10 @@
+# Default Reader Factory implementations
+org.apache.jena.hadoop.rdf.io.registry.readers.JsonLDReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.NQuadsReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.NTriplesReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.RdfJsonReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.RdfXmlReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.ThriftReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.TriGReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.TriXReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.TurtleReaderFactory
\ 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/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory b/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
new file mode 100644
index 0000000..164880d
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
@@ -0,0 +1,10 @@
+# Default Writer Factory implementations
+org.apache.jena.hadoop.rdf.io.registry.writers.JsonLDWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.NQuadsWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.NTriplesWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.RdfJsonWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.RdfXmlWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.ThriftWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.TriGWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.TriXWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.TurtleWriterFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
new file mode 100644
index 0000000..5762fb7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.log4j.Logger;
+
+
+/**
+ * A test mapper which takes in line based RDF triple input and just produces triples
+ * 
+ *
+ */
+public class RdfTriplesInputTestMapper extends Mapper<LongWritable, TripleWritable, NullWritable, TripleWritable> {
+    
+    private static final Logger LOG = Logger.getLogger(RdfTriplesInputTestMapper.class);
+
+    @Override
+    protected void map(LongWritable key, TripleWritable value, Context context)
+            throws IOException, InterruptedException {
+        LOG.info("Line " + key.toString() + " => " + value.toString());
+        context.write(NullWritable.get(), value);
+    }
+
+    
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
new file mode 100644
index 0000000..1cda0bd
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * Abstract tests for blocked triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractBlockedQuadInputFormatTests extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected boolean canSplitInputs() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
new file mode 100644
index 0000000..2e1e865
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * Abstract tests for blocked triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractBlockedTripleInputFormatTests extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected boolean canSplitInputs() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
new file mode 100644
index 0000000..e22650f
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
@@ -0,0 +1,612 @@
+/*
+ * 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 java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract node tuple input format tests
+ * 
+ * 
+ * 
+ * @param <TValue>
+ * @param <T>
+ */
+public abstract class AbstractNodeTupleInputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleInputFormatTests.class);
+
+    protected static final int EMPTY_SIZE = 0, SMALL_SIZE = 100, LARGE_SIZE = 10000, BAD_SIZE = 100, MIXED_SIZE = 100;
+    protected static final String EMPTY = "empty";
+    protected static final String SMALL = "small";
+    protected static final String LARGE = "large";
+    protected static final String BAD = "bad";
+    protected static final String MIXED = "mixed";
+
+    /**
+     * Temporary folder for the tests
+     */
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+
+    protected File empty, small, large, bad, mixed;
+
+    /**
+     * Prepares the inputs for the tests
+     * 
+     * @throws IOException
+     */
+    @Before
+    public void beforeTest() throws IOException {
+        this.prepareInputs();
+    }
+
+    /**
+     * Cleans up the inputs after each test
+     */
+    @After
+    public void afterTest() {
+        // Should be unnecessary since JUnit will clean up the temporary folder
+        // anyway but best to do this regardless
+        if (empty != null)
+            empty.delete();
+        if (small != null)
+            small.delete();
+        if (large != null)
+            large.delete();
+        if (bad != null)
+            bad.delete();
+        if (mixed != null)
+            mixed.delete();
+    }
+
+    /**
+     * Prepares a fresh configuration
+     * 
+     * @return Configuration
+     */
+    protected Configuration prepareConfiguration() {
+        Configuration config = new Configuration(true);
+        // Nothing else to do
+        return config;
+    }
+
+    /**
+     * Prepares the inputs
+     * 
+     * @throws IOException
+     */
+    protected void prepareInputs() throws IOException {
+        String ext = this.getFileExtension();
+        empty = folder.newFile(EMPTY + ext);
+        this.generateTuples(empty, EMPTY_SIZE);
+        small = folder.newFile(SMALL + ext);
+        this.generateTuples(small, SMALL_SIZE);
+        large = folder.newFile(LARGE + ext);
+        this.generateTuples(large, LARGE_SIZE);
+        bad = folder.newFile(BAD + ext);
+        this.generateBadTuples(bad, BAD_SIZE);
+        mixed = folder.newFile(MIXED + ext);
+        this.generateMixedTuples(mixed, MIXED_SIZE);
+    }
+
+    /**
+     * Gets the extra file extension to add to the filenames
+     * 
+     * @return File extension
+     */
+    protected abstract String getFileExtension();
+
+    /**
+     * Generates tuples used for tests
+     * 
+     * @param f
+     *            File
+     * @param num
+     *            Number of tuples to generate
+     * @throws IOException
+     */
+    protected final void generateTuples(File f, int num) throws IOException {
+        this.generateTuples(this.getOutputStream(f), num);
+    }
+
+    /**
+     * Gets the output stream to use for generating tuples
+     * 
+     * @param f
+     *            File
+     * @return Output Stream
+     * @throws IOException
+     */
+    protected OutputStream getOutputStream(File f) throws IOException {
+        return new FileOutputStream(f, false);
+    }
+
+    /**
+     * Generates tuples used for tests
+     * 
+     * @param output
+     *            Output Stream to write to
+     * @param num
+     *            Number of tuples to generate
+     * @throws IOException
+     */
+    protected abstract void generateTuples(OutputStream output, int num) throws IOException;
+
+    /**
+     * Generates bad tuples used for tests
+     * 
+     * @param f
+     *            File
+     * @param num
+     *            Number of bad tuples to generate
+     * @throws IOException
+     */
+    protected final void generateBadTuples(File f, int num) throws IOException {
+        this.generateBadTuples(this.getOutputStream(f), num);
+    }
+
+    /**
+     * Generates bad tuples used for tests
+     * 
+     * @param output
+     *            Output Stream to write to
+     * @param num
+     *            Number of bad tuples to generate
+     * @throws IOException
+     */
+    protected abstract void generateBadTuples(OutputStream output, int num) throws IOException;
+
+    /**
+     * Generates a mixture of good and bad tuples used for tests
+     * 
+     * @param f
+     *            File
+     * @param num
+     *            Number of tuples to generate, they should be a 50/50 mix of
+     *            good and bad tuples
+     * @throws IOException
+     */
+    protected final void generateMixedTuples(File f, int num) throws IOException {
+        this.generateMixedTuples(this.getOutputStream(f), num);
+    }
+
+    /**
+     * Generates a mixture of good and bad tuples used for tests
+     * 
+     * @param output
+     *            Output Stream to write to
+     * @param num
+     *            Number of tuples to generate, they should be a 50/50 mix of
+     *            good and bad tuples
+     * @throws IOException
+     */
+    protected abstract void generateMixedTuples(OutputStream output, int num) throws IOException;
+
+    /**
+     * Adds an input path to the job configuration
+     * 
+     * @param f
+     *            File
+     * @param config
+     *            Configuration
+     * @param job
+     *            Job
+     * @throws IOException
+     */
+    protected void addInputPath(File f, Configuration config, Job job) throws IOException {
+        FileSystem fs = FileSystem.getLocal(config);
+        Path inputPath = fs.makeQualified(new Path(f.getAbsolutePath()));
+        FileInputFormat.addInputPath(job, inputPath);
+    }
+
+    protected final int countTuples(RecordReader<LongWritable, T> reader) throws IOException, InterruptedException {
+        int count = 0;
+
+        // Check initial progress
+        LOG.info(String.format("Initial Reported Progress %f", reader.getProgress()));
+        float progress = reader.getProgress();
+        if (Float.compare(0.0f, progress) == 0) {
+            Assert.assertEquals(0.0d, reader.getProgress(), 0.0d);
+        } else if (Float.compare(1.0f, progress) == 0) {
+            // If reader is reported 1.0 straight away then we expect there to
+            // be no key values
+            Assert.assertEquals(1.0d, reader.getProgress(), 0.0d);
+            Assert.assertFalse(reader.nextKeyValue());
+        } else {
+            Assert.fail(String.format(
+                    "Expected progress of 0.0 or 1.0 before reader has been accessed for first time but got %f",
+                    progress));
+        }
+
+        // Count tuples
+        boolean debug = LOG.isDebugEnabled();
+        while (reader.nextKeyValue()) {
+            count++;
+            progress = reader.getProgress();
+            if (debug)
+                LOG.debug(String.format("Current Reported Progress %f", progress));
+            Assert.assertTrue(String.format("Progress should be in the range 0.0 < p <= 1.0 but got %f", progress),
+                    progress > 0.0f && progress <= 1.0f);
+        }
+        reader.close();
+        LOG.info(String.format("Got %d tuples from this record reader", count));
+
+        // Check final progress
+        LOG.info(String.format("Final Reported Progress %f", reader.getProgress()));
+        Assert.assertEquals(1.0d, reader.getProgress(), 0.0d);
+
+        return count;
+    }
+
+    protected final void checkTuples(RecordReader<LongWritable, T> reader, int expected) throws IOException,
+            InterruptedException {
+        Assert.assertEquals(expected, this.countTuples(reader));
+    }
+
+    /**
+     * Runs a test with a single input
+     * 
+     * @param input
+     *            Input
+     * @param expectedTuples
+     *            Expected tuples
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected final void testSingleInput(File input, int expectedSplits, int expectedTuples) throws IOException,
+            InterruptedException {
+        // Prepare configuration
+        Configuration config = this.prepareConfiguration();
+        this.testSingleInput(config, input, expectedSplits, expectedTuples);
+    }
+
+    /**
+     * Runs a test with a single input
+     * 
+     * @param config
+     *            Configuration
+     * @param input
+     *            Input
+     * @param expectedTuples
+     *            Expected tuples
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected final void testSingleInput(Configuration config, File input, int expectedSplits, int expectedTuples)
+            throws IOException, InterruptedException {
+        // Set up fake job
+        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
+        Job job = Job.getInstance(config);
+        job.setInputFormatClass(inputFormat.getClass());
+        this.addInputPath(input, job.getConfiguration(), job);
+        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+        Assert.assertEquals(1, FileInputFormat.getInputPaths(context).length);
+        NLineInputFormat.setNumLinesPerSplit(job, LARGE_SIZE);
+
+        // Check splits
+        List<InputSplit> splits = inputFormat.getSplits(context);
+        Assert.assertEquals(expectedSplits, splits.size());
+
+        // Check tuples
+        for (InputSplit split : splits) {
+            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
+            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
+            reader.initialize(split, taskContext);
+            this.checkTuples(reader, expectedTuples);
+        }
+    }
+
+    protected abstract InputFormat<LongWritable, T> getInputFormat();
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_01() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(empty, this.canSplitInputs() ? 0 : 1, EMPTY_SIZE);
+    }
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_02() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(small, 1, SMALL_SIZE);
+    }
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_03() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(large, 1, LARGE_SIZE);
+    }
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_04() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(bad, 1, 0);
+    }
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_05() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(mixed, 1, MIXED_SIZE / 2);
+    }
+
+    /**
+     * Tests behaviour when ignoring bad tuples is disabled
+     * 
+     * @throws InterruptedException
+     * @throws IOException
+     */
+    @Test(expected = IOException.class)
+    public final void fail_on_bad_input_01() throws IOException, InterruptedException {
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        Assert.assertFalse(config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true));
+        testSingleInput(config, bad, 1, 0);
+    }
+
+    /**
+     * Tests behaviour when ignoring bad tuples is disabled
+     * 
+     * @throws InterruptedException
+     * @throws IOException
+     */
+    @Test(expected = IOException.class)
+    public final void fail_on_bad_input_02() throws IOException, InterruptedException {
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        Assert.assertFalse(config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true));
+        testSingleInput(config, mixed, 1, MIXED_SIZE / 2);
+    }
+
+    /**
+     * Runs a multiple input test
+     * 
+     * @param inputs
+     *            Inputs
+     * @param expectedSplits
+     *            Number of splits expected
+     * @param expectedTuples
+     *            Number of tuples expected
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected final void testMultipleInputs(File[] inputs, int expectedSplits, int expectedTuples) throws IOException,
+            InterruptedException {
+        // Prepare configuration and inputs
+        Configuration config = this.prepareConfiguration();
+
+        // Set up fake job
+        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
+        Job job = Job.getInstance(config);
+        job.setInputFormatClass(inputFormat.getClass());
+        for (File input : inputs) {
+            this.addInputPath(input, job.getConfiguration(), job);
+        }
+        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+        Assert.assertEquals(inputs.length, FileInputFormat.getInputPaths(context).length);
+        NLineInputFormat.setNumLinesPerSplit(job, expectedTuples);
+
+        // Check splits
+        List<InputSplit> splits = inputFormat.getSplits(context);
+        Assert.assertEquals(expectedSplits, splits.size());
+
+        // Check tuples
+        int count = 0;
+        for (InputSplit split : splits) {
+            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
+            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
+            reader.initialize(split, taskContext);
+            count += this.countTuples(reader);
+        }
+        Assert.assertEquals(expectedTuples, count);
+    }
+
+    /**
+     * tuples test with multiple inputs
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void multiple_inputs_01() throws IOException, InterruptedException, ClassNotFoundException {
+        testMultipleInputs(new File[] { empty, small, large }, this.canSplitInputs() ? 2 : 3, EMPTY_SIZE + SMALL_SIZE
+                + LARGE_SIZE);
+    }
+
+    /**
+     * tuples test with multiple inputs
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void multiple_inputs_02() throws IOException, InterruptedException, ClassNotFoundException {
+        testMultipleInputs(new File[] { folder.getRoot() }, this.canSplitInputs() ? 4 : 5, EMPTY_SIZE + SMALL_SIZE
+                + LARGE_SIZE + (MIXED_SIZE / 2));
+    }
+
+    protected final void testSplitInputs(Configuration config, File[] inputs, int expectedSplits, int expectedTuples)
+            throws IOException, InterruptedException {
+        // Set up fake job
+        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
+        Job job = Job.getInstance(config);
+        job.setInputFormatClass(inputFormat.getClass());
+        for (File input : inputs) {
+            this.addInputPath(input, job.getConfiguration(), job);
+        }
+        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+        Assert.assertEquals(inputs.length, FileInputFormat.getInputPaths(context).length);
+
+        // Check splits
+        List<InputSplit> splits = inputFormat.getSplits(context);
+        Assert.assertEquals(expectedSplits, splits.size());
+
+        // Check tuples
+        int count = 0;
+        for (InputSplit split : splits) {
+            // Validate split
+            Assert.assertTrue(this.isValidSplit(split, config));
+
+            // Read split
+            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
+            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
+            reader.initialize(split, taskContext);
+            count += this.countTuples(reader);
+        }
+        Assert.assertEquals(expectedTuples, count);
+    }
+
+    /**
+     * Determines whether an input split is valid
+     * 
+     * @param split
+     *            Input split
+     * @return True if a valid split, false otherwise
+     * @throws IOException
+     */
+    protected boolean isValidSplit(InputSplit split, Configuration config) throws IOException {
+        return split instanceof FileSplit;
+    }
+
+    /**
+     * Indicates whether inputs can be split, defaults to true
+     * 
+     * @return Whether inputs can be split
+     */
+    protected boolean canSplitInputs() {
+        return true;
+    }
+
+    /**
+     * Tests for input splitting
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public final void split_input_01() throws IOException, InterruptedException, ClassNotFoundException {
+        Assume.assumeTrue(this.canSplitInputs());
+
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
+        this.testSplitInputs(config, new File[] { small }, 100, SMALL_SIZE);
+    }
+
+    /**
+     * Tests for input splitting
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public final void split_input_02() throws IOException, InterruptedException, ClassNotFoundException {
+        Assume.assumeTrue(this.canSplitInputs());
+
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        config.setLong(NLineInputFormat.LINES_PER_MAP, 10);
+        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
+        this.testSplitInputs(config, new File[] { small }, 10, SMALL_SIZE);
+    }
+
+    /**
+     * Tests for input splitting
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public final void split_input_03() throws IOException, InterruptedException, ClassNotFoundException {
+        Assume.assumeTrue(this.canSplitInputs());
+
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        config.setLong(NLineInputFormat.LINES_PER_MAP, 100);
+        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
+        this.testSplitInputs(config, new File[] { large }, 100, LARGE_SIZE);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
new file mode 100644
index 0000000..78d7f33
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
@@ -0,0 +1,70 @@
+/*
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for Quad input formats
+ * 
+ *
+ */
+public abstract class AbstractQuadsInputFormatTests extends AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected void generateTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+    
+    @Override
+    protected void generateBadTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write("<http://broken\n".getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
+        boolean bad = false;
+        for (int i = 0; i < num; i++, bad = !bad) {
+            if (bad) {
+                output.write("<http://broken\n".getBytes(utf8));
+            } else {
+                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
+            }
+        }
+        output.flush();
+        output.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
new file mode 100644
index 0000000..65a9889
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for Triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractTriplesInputFormatTests extends AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected void generateTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "<http://broken\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
+        boolean bad = false;
+        for (int i = 0; i < num; i++, bad = !bad) {
+            if (bad) {
+                output.write("<http://broken\n".getBytes(utf8));
+            } else {
+                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
+            }
+        }
+        output.flush();
+        output.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
new file mode 100644
index 0000000..0b6cfde
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
@@ -0,0 +1,115 @@
+/*
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+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.query.Dataset;
+import com.hp.hpl.jena.query.DatasetFactory;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Property;
+import com.hp.hpl.jena.rdf.model.Resource;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for Quad input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractWholeFileQuadInputFormatTests extends AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+
+    private void writeTuples(Dataset ds, OutputStream output) {
+        RDFDataMgr.write(output, ds, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
+    }
+
+    /**
+     * Gets the RDF language to write out generate tuples in
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+    private void writeGoodTuples(OutputStream output, int num) throws IOException {
+        Dataset ds = DatasetFactory.createMem();
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num; i++) {
+            if (i % 100 == 0) {
+                ds.addNamedModel("http://example.org/graphs/" + (i / 100), m);
+                m = ModelFactory.createDefaultModel();
+            }
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        if (!m.isEmpty()) {
+            ds.addNamedModel("http://example.org/graphs/extra", m);
+        }
+        this.writeTuples(ds, output);
+    }
+
+    @Override
+    protected final void generateTuples(OutputStream output, int num) throws IOException {
+        this.writeGoodTuples(output, num);
+        output.close();
+    }
+
+    @Override
+    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
+        // Write good data
+        this.writeGoodTuples(output, num / 2);
+
+        // Write junk data
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num / 2; i++) {
+            output.write(junk);
+        }
+
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
new file mode 100644
index 0000000..b68d662
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
@@ -0,0 +1,108 @@
+/*
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+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.Triple;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Property;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * Abstract tests for Triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractWholeFileTripleInputFormatTests extends AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+    
+    private void writeTuples(Model m, OutputStream output) {
+        RDFDataMgr.write(output, m, this.getRdfLanguage());
+    }
+        
+    /**
+     * Gets the RDF language to write out generate tuples in
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+    
+    @Override
+    protected final void generateTuples(OutputStream output, int num) throws IOException {
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num; i++) {
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        this.writeTuples(m, output);
+        output.close();
+    }
+    
+    @Override
+    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
+        // Write good data
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num / 2; i++) {
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        this.writeTuples(m, output);
+        
+        // Write junk data
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num / 2; i++) {
+            output.write(junk);
+        }
+        
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
deleted file mode 100644
index c9b3a26..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * 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.File;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.RDFDataMgr;
-import org.junit.Assert;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ResIterator;
-import com.hp.hpl.jena.rdf.model.Resource;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Tests for TriG output with blank nodes
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class TriGBlankNodeOutputTests extends StreamedTriGOutputTest {
-
-	static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-	static long $bs2 = 1000;
-	static long $bs3 = 100;
-	static long $bs4 = 1;
-
-	/**
-	 * @return Test parameters
-	 */
-	@Parameters
-	public static Collection<Object[]> data() {
-		return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 },
-				{ $bs4 } });
-	}
-
-	/**
-	 * Creates new tests
-	 * 
-	 * @param batchSize
-	 *            Batch size
-	 */
-	public TriGBlankNodeOutputTests(long batchSize) {
-		super(batchSize);
-	}
-
-	@Override
-	protected Iterator<QuadWritable> generateTuples(int num) {
-		List<QuadWritable> qs = new ArrayList<QuadWritable>();
-		Node subject = NodeFactory.createAnon();
-		for (int i = 0; i < num; i++) {
-			Quad t = new Quad(
-					NodeFactory.createURI("http://example.org/graphs/" + i),
-					subject,
-					NodeFactory.createURI("http://example.org/predicate"),
-					NodeFactory.createLiteral(Integer.toString(i),
-							XSDDatatype.XSDinteger));
-			qs.add(new QuadWritable(t));
-		}
-		return qs.iterator();
-	}
-
-	@Override
-	protected void checkTuples(File f, long expected) {
-		super.checkTuples(f, expected);
-
-		Model m = RDFDataMgr.loadModel("file://" + f.getAbsolutePath(),
-				this.getRdfLanguage());
-		ResIterator iter = m.listSubjects();
-		Set<Node> subjects = new HashSet<Node>();
-		while (iter.hasNext()) {
-			Resource res = iter.next();
-			Assert.assertTrue(res.isAnon());
-			subjects.add(res.asNode());
-		}
-		// Should only be one subject unless the data was empty in which case
-		// there will be zero subjects
-		Assert.assertEquals(expected == 0 ? 0 : 1, subjects.size());
-	}
-
-	@Override
-	protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-		return new TriGOutputFormat<NullWritable>();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputTest.java
deleted file mode 100644
index 9b6e307..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests for TriX output format
- */
-public class TriXOutputTest extends AbstractQuadOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".trix";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIX;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new TriXOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
deleted file mode 100644
index a6c4d70..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.util.Arrays;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-
-/**
- * Tests for Turtle output
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class BatchedTurtleOutputTest extends AbstractTripleOutputFormatTests {
-
-    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-    static long $bs2 = 1000;
-    static long $bs3 = 100;
-    static long $bs4 = 1;
-
-    /**
-     * @return Test parameters
-     */
-    @Parameters
-    public static Collection<Object[]> data() {
-        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
-    }
-
-    private final long batchSize;
-
-    /**
-     * Creates new tests
-     * 
-     * @param batchSize
-     *            Batch size
-     */
-    public BatchedTurtleOutputTest(long batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".ttl";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TURTLE;
-    }
-    
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
-        return config;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new BatchedTurtleOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
deleted file mode 100644
index d8843d3..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.util.Arrays;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-
-/**
- * Tests for Turtle output
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class StreamedTurtleOutputTest extends AbstractTripleOutputFormatTests {
-
-    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-    static long $bs2 = 1000;
-    static long $bs3 = 100;
-    static long $bs4 = 1;
-
-    /**
-     * @return Test parameters
-     */
-    @Parameters
-    public static Collection<Object[]> data() {
-        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
-    }
-
-    private final long batchSize;
-
-    /**
-     * Creates new tests
-     * 
-     * @param batchSize
-     *            Batch size
-     */
-    public StreamedTurtleOutputTest(long batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".ttl";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TURTLE;
-    }
-    
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
-        return config;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new TurtleOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
deleted file mode 100644
index 8dcae4e..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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.File;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.RDFDataMgr;
-import org.junit.Assert;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ResIterator;
-import com.hp.hpl.jena.rdf.model.Resource;
-
-/**
- * Tests for Turtle output with blank nodes
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class TurtleBlankNodeOutputTests extends StreamedTurtleOutputTest {
-
-	static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-	static long $bs2 = 1000;
-	static long $bs3 = 100;
-	static long $bs4 = 1;
-
-	/**
-	 * @return Test parameters
-	 */
-	@Parameters
-	public static Collection<Object[]> data() {
-		return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 },
-				{ $bs4 } });
-	}
-
-	/**
-	 * Creates new tests
-	 * 
-	 * @param batchSize
-	 *            Batch size
-	 */
-	public TurtleBlankNodeOutputTests(long batchSize) {
-		super(batchSize);
-	}
-
-	@Override
-	protected Iterator<TripleWritable> generateTuples(int num) {
-		List<TripleWritable> ts = new ArrayList<TripleWritable>();
-		Node subject = NodeFactory.createAnon();
-		for (int i = 0; i < num; i++) {
-			Triple t = new Triple(subject,
-					NodeFactory.createURI("http://example.org/predicate"),
-					NodeFactory.createLiteral(Integer.toString(i),
-							XSDDatatype.XSDinteger));
-			ts.add(new TripleWritable(t));
-		}
-		return ts.iterator();
-	}
-
-	@Override
-	protected void checkTuples(File f, long expected) {
-		super.checkTuples(f, expected);
-
-		Model m = RDFDataMgr.loadModel("file://" + f.getAbsolutePath(),
-				this.getRdfLanguage());
-		ResIterator iter = m.listSubjects();
-		Set<Node> subjects = new HashSet<Node>();
-		while (iter.hasNext()) {
-			Resource res = iter.next();
-			Assert.assertTrue(res.isAnon());
-			subjects.add(res.asNode());
-		}
-		// Should only be one subject unless the data was empty in which case
-		// there will be zero subjects
-		Assert.assertEquals(expected == 0 ? 0 : 1, subjects.size());
-	}
-
-	@Override
-	protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-		return new TurtleOutputFormat<NullWritable>();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
deleted file mode 100644
index 2eae232..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * 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.registry;
-
-import java.io.IOException;
-import java.io.StringWriter;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for the {@link HadoopRdfIORegistry}
- */
-public class TestHadoopRdfIORegistry {
-
-    private void testLang(Lang lang, boolean triples, boolean quads, boolean writesSupported) {
-        Assert.assertEquals(triples, HadoopRdfIORegistry.hasTriplesReader(lang));
-        Assert.assertEquals(quads, HadoopRdfIORegistry.hasQuadReader(lang));
-
-        // Some formats may be asymmetric
-        if (writesSupported) {
-            Assert.assertEquals(triples, HadoopRdfIORegistry.hasTriplesWriter(lang));
-            Assert.assertEquals(quads, HadoopRdfIORegistry.hasQuadWriter(lang));
-        } else {
-            Assert.assertFalse(HadoopRdfIORegistry.hasTriplesWriter(lang));
-            Assert.assertFalse(HadoopRdfIORegistry.hasQuadWriter(lang));
-        }
-
-        if (triples) {
-            // Check that triples are supported
-            RecordReader<LongWritable, TripleWritable> tripleReader;
-            try {
-                tripleReader = HadoopRdfIORegistry.createTripleReader(lang);
-                Assert.assertNotNull(tripleReader);
-            } catch (IOException e) {
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " can read triples but fails to produce a triple reader when asked: " + e.getMessage());
-            }
-
-            if (writesSupported) {
-                RecordWriter<NullWritable, TripleWritable> tripleWriter;
-                try {
-                    tripleWriter = HadoopRdfIORegistry.createTripleWriter(lang, new StringWriter(), new Configuration(
-                            false));
-                    Assert.assertNotNull(tripleWriter);
-                } catch (IOException e) {
-                    Assert.fail("Registry indicates that " + lang.getName()
-                            + " can write triples but fails to produce a triple writer when asked: " + e.getMessage());
-                }
-            }
-        } else {
-            // Check that triples are not supported
-            try {
-                HadoopRdfIORegistry.createTripleReader(lang);
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " cannot read triples but produced a triple reader when asked (error was expected)");
-            } catch (IOException e) {
-                // This is expected
-            }
-            try {
-                HadoopRdfIORegistry.createTripleWriter(lang, new StringWriter(), new Configuration(false));
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " cannot write triples but produced a triple write when asked (error was expected)");
-            } catch (IOException e) {
-                // This is expected
-            }
-        }
-
-        if (quads) {
-            // Check that quads are supported
-            RecordReader<LongWritable, QuadWritable> quadReader;
-            try {
-                quadReader = HadoopRdfIORegistry.createQuadReader(lang);
-                Assert.assertNotNull(quadReader);
-            } catch (IOException e) {
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " can read quads but fails to produce a quad reader when asked: " + e.getMessage());
-            }
-
-            if (writesSupported) {
-                RecordWriter<NullWritable, QuadWritable> quadWriter;
-                try {
-                    quadWriter = HadoopRdfIORegistry.createQuadWriter(lang, new StringWriter(),
-                            new Configuration(false));
-                    Assert.assertNotNull(quadWriter);
-                } catch (IOException e) {
-                    Assert.fail("Registry indicates that " + lang.getName()
-                            + " can write quads but fails to produce a triple writer when asked: " + e.getMessage());
-                }
-            }
-        } else {
-            try {
-                HadoopRdfIORegistry.createQuadReader(lang);
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " cannot read quads but produced a quad reader when asked (error was expected)");
-            } catch (IOException e) {
-                // This is expected
-            }
-            try {
-                HadoopRdfIORegistry.createQuadWriter(lang, new StringWriter(), new Configuration(false));
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " cannot write quads but produced a quad writer when asked (error was expected)");
-            } catch (IOException e) {
-                // This is expected
-            }
-        }
-    }
-
-    @Test
-    public void json_ld_registered() {
-        testLang(Lang.JSONLD, true, true, true);
-    }
-
-    @Test
-    public void nquads_registered() {
-        testLang(Lang.NQUADS, false, true, true);
-        testLang(Lang.NQ, false, true, true);
-    }
-
-    @Test
-    public void ntriples_registered() {
-        testLang(Lang.NTRIPLES, true, false, true);
-        testLang(Lang.NT, true, false, true);
-    }
-
-    @Test
-    public void rdf_json_registered() {
-        testLang(Lang.RDFJSON, true, false, true);
-    }
-
-    @Test
-    public void rdf_xml_registered() {
-        testLang(Lang.RDFXML, true, false, true);
-    }
-
-    @Test
-    public void rdf_thrift_registered() {
-        testLang(RDFLanguages.THRIFT, true, true, true);
-    }
-
-    @Test
-    public void trig_registered() {
-        testLang(Lang.TRIG, false, true, true);
-    }
-
-    @Test
-    public void trix_registered() {
-        testLang(Lang.TRIX, false, true, true);
-    }
-
-    @Test
-    public void turtle_registered() {
-        testLang(Lang.TURTLE, true, false, true);
-        testLang(Lang.TTL, true, false, true);
-        testLang(Lang.N3, true, false, true);
-    }
-
-    @Test
-    public void unregistered() {
-        testLang(Lang.RDFNULL, false, false, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-stats/hadoop-job.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-stats/hadoop-job.xml b/jena-hadoop-rdf/jena-elephas-stats/hadoop-job.xml
deleted file mode 100644
index de72645..0000000
--- a/jena-hadoop-rdf/jena-elephas-stats/hadoop-job.xml
+++ /dev/null
@@ -1,46 +0,0 @@
-<!--
-   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.
--->
-
-<assembly>
-  <id>hadoop-job</id>
-  <formats>
-    <format>jar</format>
-  </formats>
-  <includeBaseDirectory>false</includeBaseDirectory>
-  <dependencySets>
-    <dependencySet>
-      <unpack>false</unpack>
-      <scope>runtime</scope>
-      <outputDirectory>lib</outputDirectory>
-      <excludes>
-        <exclude>${groupId}:${artifactId}</exclude>
-      </excludes>
-    </dependencySet>
-    <dependencySet>
-      <unpack>true</unpack>
-      <includes>
-        <include>${groupId}:${artifactId}</include>
-      </includes>
-    </dependencySet>
-  </dependencySets>
-  <fileSets>
-    <fileSet>
-      <directory>${basedir}/target/test-classes</directory>
-      <outputDirectory>/</outputDirectory>
-    </fileSet>
- </fileSets>
-</assembly>

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-stats/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-stats/pom.xml b/jena-hadoop-rdf/jena-elephas-stats/pom.xml
deleted file mode 100644
index 899d612..0000000
--- a/jena-hadoop-rdf/jena-elephas-stats/pom.xml
+++ /dev/null
@@ -1,103 +0,0 @@
-<!--
-   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.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<groupId>org.apache.jena</groupId>
-		<artifactId>jena-elephas</artifactId>
-		<version>0.9.0-SNAPSHOT</version>
-	</parent>
-	<artifactId>jena-elephas-stats</artifactId>
-	<name>Apache Jena - RDF Tools for Hadoop - Statistics Demo App</name>
-	<description>A demo application that can be run on Hadoop to produce a statistical analysis on arbitrary RDF inputs</description>
-
-	<dependencies>
-		<!-- Internal Project Dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-hadoop-rdf-io</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-hadoop-rdf-mapreduce</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<!-- CLI related Dependencies -->
-		<dependency>
-			<groupId>io.airlift</groupId>
-			<artifactId>airline</artifactId>
-			<version>0.6</version>
-		</dependency>
-
-		<!-- Hadoop Dependencies -->
-		<!-- Note these will be provided on the Hadoop cluster hence the provided 
-			scope -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-mapreduce-client-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Test Dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-hadoop-rdf-mapreduce</artifactId>
-			<version>${project.version}</version>
-			<classifier>tests</classifier>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.mrunit</groupId>
-			<artifactId>mrunit</artifactId>
-			<scope>test</scope>
-			<classifier>hadoop2</classifier>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<plugins>
-			<!-- Assembly plugin is used to produce the runnable Hadoop JAR with all 
-				dependencies contained therein -->
-			<plugin>
-				<artifactId>maven-assembly-plugin</artifactId>
-				<configuration>
-					<descriptors>
-						<descriptor>hadoop-job.xml</descriptor>
-					</descriptors>
-				</configuration>
-				<executions>
-					<execution>
-						<id>make-assembly</id>
-						<phase>package</phase>
-						<goals>
-							<goal>single</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java b/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
deleted file mode 100644
index 5f870ee..0000000
--- a/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/*
- * 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.stats;
-
-import io.airlift.command.Arguments;
-import io.airlift.command.Command;
-import io.airlift.command.Help;
-import io.airlift.command.HelpOption;
-import io.airlift.command.Option;
-import io.airlift.command.OptionType;
-import io.airlift.command.ParseArgumentsMissingException;
-import io.airlift.command.ParseArgumentsUnexpectedException;
-import io.airlift.command.ParseException;
-import io.airlift.command.ParseOptionMissingException;
-import io.airlift.command.ParseOptionMissingValueException;
-import io.airlift.command.SingleCommand;
-import io.airlift.command.model.CommandMetadata;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import javax.inject.Inject;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.jena.hadoop.rdf.stats.jobs.JobFactory;
-
-
-/**
- * Entry point for the Hadoop job, handles launching all the relevant Hadoop
- * jobs
- */
-@Command(name = "bin/hadoop jar PATH_TO_JAR com.yarcdata.urika.hadoop.rdf.stats.RdfStats", description = "A command which computes statistics on RDF data using Hadoop")
-public class RdfStats implements Tool {
-
-    static final String ANSI_RED = "\u001B[31m";
-    static final String ANSI_RESET = "\u001B[0m";
-
-    private static final String DATA_TYPE_TRIPLES = "triples", DATA_TYPE_QUADS = "quads", DATA_TYPE_MIXED = "mixed";
-
-    /**
-     * Help option
-     */
-    @Inject
-    public HelpOption helpOption;
-
-    /**
-     * Gets/Sets whether all available statistics will be calculated
-     */
-    @Option(name = { "-a", "--all" }, description = "Requests that all available statistics be calculated", type = OptionType.COMMAND)
-    public boolean all = false;
-
-    /**
-     * Gets/Sets whether node usage counts will be calculated
-     */
-    @Option(name = { "-n", "--node-count" }, description = "Requests that node usage counts be calculated", type = OptionType.COMMAND)
-    public boolean nodeCount = false;
-
-    /**
-     * Gets/Sets whether characteristic sets will be calculated
-     */
-    @Option(name = { "-c", "--characteristic-sets" }, description = "Requests that characteristic sets be calculated", type = OptionType.COMMAND)
-    public boolean characteristicSets = false;
-
-    /**
-     * Gets/Sets whether type counts will be calculated
-     */
-    @Option(name = { "-t", "--type-counts" }, description = "Requests that rdf:type usage counts be calculated", type = OptionType.COMMAND)
-    public boolean typeCount = false;
-
-    /**
-     * Gets/Sets whether data type counts will be calculated
-     */
-    @Option(name = { "-d", "--data-types" }, description = "Requests that literal data type usage counts be calculated", type = OptionType.COMMAND)
-    public boolean dataTypeCount = false;
-
-    /**
-     * Gets/Sets whether namespace counts will be calculated
-     */
-    @Option(name = { "--namespaces" }, description = "Requests that namespace usage counts be calculated", type = OptionType.COMMAND)
-    public boolean namespaceCount = false;
-
-    /**
-     * Gets/Sets the input data type used
-     */
-    @Option(name = { "--input-type" }, allowedValues = { DATA_TYPE_MIXED, DATA_TYPE_QUADS, DATA_TYPE_TRIPLES }, description = "Specifies whether the input data is a mixture of quads and triples, just quads or just triples.  Using the most specific data type will yield the most accurrate statistics")
-    public String inputType = DATA_TYPE_MIXED;
-
-    /**
-     * Gets/Sets the output path
-     */
-    @Option(name = { "-o", "--output" }, title = "OutputPath", description = "Sets the output path", arity = 1, required = true)
-    public String outputPath = null;
-
-    /**
-     * Gets/Sets the input path(s)
-     */
-    @Arguments(description = "Sets the input path(s)", title = "InputPath", required = true)
-    public List<String> inputPaths = new ArrayList<String>();
-
-    private Configuration config;
-
-    /**
-     * Entry point method
-     * 
-     * @param args
-     *            Arguments
-     * @throws Exception
-     */
-    public static void main(String[] args) throws Exception {
-        try {
-            // Run and exit with result code if no errors bubble up
-            // Note that the exit code may still be a error code
-            int res = ToolRunner.run(new Configuration(true), new RdfStats(), args);
-            System.exit(res);
-        } catch (Exception e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            e.printStackTrace(System.err);
-        } finally {
-            System.err.print(ANSI_RESET);
-        }
-        // If any errors bubble up exit with non-zero code
-        System.exit(1);
-    }
-
-    private static void showUsage() {
-        CommandMetadata metadata = SingleCommand.singleCommand(RdfStats.class).getCommandMetadata();
-        StringBuilder builder = new StringBuilder();
-        Help.help(metadata, builder);
-        System.err.print(ANSI_RESET);
-        System.err.println(builder.toString());
-        System.exit(1);
-    }
-
-    @Override
-    public void setConf(Configuration conf) {
-        this.config = conf;
-    }
-
-    @Override
-    public Configuration getConf() {
-        return this.config;
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-        try {
-            // Parse custom arguments
-            RdfStats cmd = SingleCommand.singleCommand(RdfStats.class).parse(args);
-
-            // Copy Hadoop configuration across
-            cmd.setConf(this.getConf());
-
-            // Show help if requested and exit with success
-            if (cmd.helpOption.showHelpIfRequested()) {
-                return 0;
-            }
-
-            // Run the command and exit with success
-            cmd.run();
-            return 0;
-
-        } catch (ParseOptionMissingException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-        } catch (ParseOptionMissingValueException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-        } catch (ParseArgumentsMissingException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-        } catch (ParseArgumentsUnexpectedException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-            // TODO Re-enable as and when we upgrade Airline
-            // } catch (ParseOptionIllegalValueException e) {
-            // System.err.println(ANSI_RED + e.getMessage());
-            // System.err.println();
-            // showUsage();
-        } catch (ParseException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-        } catch (UnsupportedOperationException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-        } catch (Throwable e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            e.printStackTrace(System.err);
-        } finally {
-            System.err.print(ANSI_RESET);
-        }
-        return 1;
-    }
-
-    private void run() throws Throwable {
-        if (!this.outputPath.endsWith("/")) {
-            this.outputPath += "/";
-        }
-
-        // If all statistics requested turn on all statistics
-        if (this.all) {
-            this.nodeCount = true;
-            this.characteristicSets = true;
-            this.typeCount = true;
-            this.dataTypeCount = true;
-            this.namespaceCount = true;
-        }
-
-        // How many statistics were requested?
-        int statsRequested = 0;
-        if (this.nodeCount)
-            statsRequested++;
-        if (this.characteristicSets)
-            statsRequested++;
-        if (this.typeCount)
-            statsRequested++;
-        if (this.dataTypeCount)
-            statsRequested++;
-        if (this.namespaceCount)
-            statsRequested++;
-
-        // Error if no statistics requested
-        if (statsRequested == 0) {
-            System.err
-                    .println("You did not request any statistics to be calculated, please use one/more of the relevant options to select the statistics to be computed");
-            return;
-        }
-        int statsComputed = 1;
-
-        // Compute statistics
-        if (this.nodeCount) {
-            Job job = this.selectNodeCountJob();
-            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
-        }
-        if (this.typeCount) {
-            Job[] jobs = this.selectTypeCountJobs();
-            statsComputed = this.computeStatistic(jobs, false, false, statsComputed, statsRequested);
-        }
-        if (this.dataTypeCount) {
-            Job job = this.selectDataTypeCountJob();
-            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
-        }
-        if (this.namespaceCount) {
-            Job job = this.selectNamespaceCountJob();
-            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
-        }
-        if (this.characteristicSets) {
-            Job[] jobs = this.selectCharacteristicSetJobs();
-            statsComputed = this.computeStatistic(jobs, false, false, statsComputed, statsRequested);
-        }
-    }
-
-    private int computeStatistic(Job job, int statsComputed, int statsRequested) throws Throwable {
-        System.out.println(String.format("Computing Statistic %d of %d requested", statsComputed, statsRequested));
-        this.runJob(job);
-        System.out.println(String.format("Computed Statistic %d of %d requested", statsComputed, statsRequested));
-        System.out.println();
-        return ++statsComputed;
-    }
-
-    private int computeStatistic(Job[] jobs, boolean continueOnFailure, boolean continueOnError, int statsComputed,
-            int statsRequested) {
-        System.out.println(String.format("Computing Statistic %d of %d requested", statsComputed, statsRequested));
-        this.runJobSequence(jobs, continueOnFailure, continueOnError);
-        System.out.println(String.format("Computed Statistic %d of %d requested", statsComputed, statsRequested));
-        System.out.println();
-        return ++statsComputed;
-    }
-
-    private boolean runJob(Job job) throws Throwable {
-        System.out.println("Submitting Job " + job.getJobName());
-        long start = System.nanoTime();
-        try {
-            job.submit();
-            if (job.monitorAndPrintJob()) {
-                System.out.println("Job " + job.getJobName() + " succeeded");
-                return true;
-            } else {
-                System.out.println("Job " + job.getJobName() + " failed");
-                return false;
-            }
-        } catch (Throwable e) {
-            System.out.println("Unexpected failure in Job " + job.getJobName());
-            throw e;
-        } finally {
-            long end = System.nanoTime();
-            System.out.println("Job " + job.getJobName() + " finished after "
-                    + String.format("%,d milliseconds", TimeUnit.NANOSECONDS.toMillis(end - start)));
-            System.out.println();
-        }
-    }
-
-    private void runJobSequence(Job[] jobs, boolean continueOnFailure, boolean continueOnError) {
-        for (int i = 0; i < jobs.length; i++) {
-            Job job = jobs[i];
-            try {
-                boolean success = this.runJob(job);
-                if (!success && !continueOnFailure)
-                    throw new IllegalStateException("Unable to complete job sequence because Job " + job.getJobName() + " failed");
-            } catch (IllegalStateException e) {
-                throw e;
-            } catch (Throwable e) {
-                if (!continueOnError)
-                    throw new IllegalStateException("Unable to complete job sequence because job " + job.getJobName()
-                            + " errorred", e);
-            }
-        }
-    }
-
-    private Job selectNodeCountJob() throws IOException {
-        String realOutputPath = outputPath + "node-counts/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadNodeCountJob(this.config, inputs, realOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleNodeCountJob(this.config, inputs, realOutputPath);
-        } else {
-            return JobFactory.getNodeCountJob(this.config, inputs, realOutputPath);
-        }
-    }
-
-    private Job selectDataTypeCountJob() throws IOException {
-        String realOutputPath = outputPath + "data-type-counts/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadDataTypeCountJob(this.config, inputs, realOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleDataTypeCountJob(this.config, inputs, realOutputPath);
-        } else {
-            return JobFactory.getDataTypeCountJob(this.config, inputs, realOutputPath);
-        }
-    }
-
-    private Job selectNamespaceCountJob() throws IOException {
-        String realOutputPath = outputPath + "namespace-counts/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadNamespaceCountJob(this.config, inputs, realOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleNamespaceCountJob(this.config, inputs, realOutputPath);
-        } else {
-            return JobFactory.getNamespaceCountJob(this.config, inputs, realOutputPath);
-        }
-    }
-
-    private Job[] selectCharacteristicSetJobs() throws IOException {
-        String intermediateOutputPath = outputPath + "characteristics/intermediate/";
-        String finalOutputPath = outputPath + "characteristics/final/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        } else {
-            return JobFactory.getCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        }
-    }
-
-    private Job[] selectTypeCountJobs() throws IOException {
-        String intermediateOutputPath = outputPath + "type-declarations/";
-        String finalOutputPath = outputPath + "type-counts/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        } else {
-            return JobFactory.getTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java b/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
deleted file mode 100644
index 55bb8af..0000000
--- a/jena-hadoop-rdf/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
+++ /dev/null
@@ -1,757 +0,0 @@
-/*
- * 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.stats.jobs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.BZip2Codec;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.jena.hadoop.rdf.io.input.QuadsInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.TriplesInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.TriplesOrQuadsInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.nquads.NQuadsInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.nquads.NQuadsOutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesNodeOutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesOutputFormat;
-import org.apache.jena.hadoop.rdf.mapreduce.KeyMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.characteristics.CharacteristicSetReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.characteristics.QuadCharacteristicSetGeneratingReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.characteristics.TripleCharacteristicSetGeneratingReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.datatypes.QuadDataTypeCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.datatypes.TripleDataTypeCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.namespaces.QuadNamespaceCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.namespaces.TripleNamespaceCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.positional.QuadObjectCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.positional.TripleObjectCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.QuadFilterByPredicateMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupBySubjectMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupBySubjectMapper;
-import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
-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 com.hp.hpl.jena.vocabulary.RDF;
-
-/**
- * Factory that can produce {@link Job} instances for computing various RDF
- * statistics
- * 
- * 
- * 
- */
-public class JobFactory {
-
-    /**
-     * Private constructor prevents instantiation
-     */
-    private JobFactory() {
-    }
-
-    /**
-     * Gets a job for computing node counts on RDF triple inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getTripleNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Node Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleNodeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing node counts on RDF quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getQuadNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Node Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadNodeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing node counts on RDF triple and/or quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Node Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadNodeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a sequence of jobs that can be used to compute characteristic sets
-     * for RDF triples
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Intermediate output path
-     * @param outputPath
-     *            Final output path
-     * @return Sequence of jobs
-     * @throws IOException
-     */
-    public static Job[] getTripleCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Characteristic Set (Generation)");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleGroupBySubjectMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(TripleWritable.class);
-        job.setReducerClass(TripleCharacteristicSetGeneratingReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(NullWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(SequenceFileOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-        SequenceFileOutputFormat.setCompressOutput(job, true);
-        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
-        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
-
-        jobs[0] = job;
-
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Characteristic Set (Reduction)");
-
-        // Map/Reduce classes
-        job.setMapperClass(KeyMapper.class);
-        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
-        job.setMapOutputValueClass(CharacteristicSetWritable.class);
-        job.setReducerClass(CharacteristicSetReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(CharacteristicSetWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(SequenceFileInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-        return jobs;
-    }
-
-    /**
-     * Gets a sequence of jobs that can be used to compute characteristic sets
-     * for RDF quads
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Intermediate output path
-     * @param outputPath
-     *            Final output path
-     * @return Sequence of jobs
-     * @throws IOException
-     */
-    public static Job[] getQuadCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Characteristic Set (Generation)");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadGroupBySubjectMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(QuadWritable.class);
-        job.setReducerClass(QuadCharacteristicSetGeneratingReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(NullWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(SequenceFileOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-        SequenceFileOutputFormat.setCompressOutput(job, true);
-        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
-        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
-
-        jobs[0] = job;
-
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Characteristic Set (Reduction)");
-
-        // Map/Reduce classes
-        job.setMapperClass(KeyMapper.class);
-        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
-        job.setMapOutputValueClass(CharacteristicSetWritable.class);
-        job.setReducerClass(CharacteristicSetReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(CharacteristicSetWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(SequenceFileInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-        return jobs;
-    }
-
-    /**
-     * Gets a sequence of jobs that can be used to compute characteristic sets
-     * for RDF triple and/or quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Intermediate output path
-     * @param outputPath
-     *            Final output path
-     * @return Sequence of jobs
-     * @throws IOException
-     */
-    public static Job[] getCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Characteristic Set (Generation)");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadGroupBySubjectMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(QuadWritable.class);
-        job.setReducerClass(QuadCharacteristicSetGeneratingReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(NullWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(SequenceFileOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-        SequenceFileOutputFormat.setCompressOutput(job, true);
-        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
-        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
-
-        jobs[0] = job;
-
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Characteristic Set (Reduction)");
-
-        // Map/Reduce classes
-        job.setMapperClass(KeyMapper.class);
-        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
-        job.setMapOutputValueClass(CharacteristicSetWritable.class);
-        job.setReducerClass(CharacteristicSetReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(CharacteristicSetWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(SequenceFileInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-        return jobs;
-    }
-
-    /**
-     * Gets a job for computing type counts on RDF triple inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Path for intermediate output which will be all the type
-     *            declaration triples present in the inputs
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job[] getTripleTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Type Triples Extraction");
-
-        // Map/Reduce classes
-        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
-        job.setMapperClass(TripleFilterByPredicateUriMapper.class);
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setMapOutputValueClass(TripleWritable.class);
-
-        // Input and Output Format
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(NTriplesOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-
-        jobs[0] = job;
-
-        // Object Node Usage count job
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleObjectCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(NTriplesInputFormat.class);
-        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
-                                                          // better if this was
-                                                          // intelligently
-                                                          // configured
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-
-        return jobs;
-    }
-
-    /**
-     * Gets a job for computing type counts on RDF quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Path for intermediate output which will be all the type
-     *            declaration quads present in the inputs
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job[] getQuadTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Type Quads Extraction");
-
-        // Map/Reduce classes
-        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
-        job.setMapperClass(QuadFilterByPredicateMapper.class);
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setMapOutputValueClass(QuadWritable.class);
-
-        // Input and Output Format
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(NQuadsOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-
-        jobs[0] = job;
-
-        // Object Node Usage count job
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadObjectCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(NQuadsInputFormat.class);
-        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
-                                                          // better if this was
-                                                          // intelligently
-                                                          // configured
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-
-        return jobs;
-    }
-
-    /**
-     * Gets a job for computing type counts on RDF triple and/or quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Path for intermediate output which will be all the type
-     *            declaration quads present in the inputs
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job[] getTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Type Extraction");
-
-        // Map/Reduce classes
-        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
-        job.setMapperClass(QuadFilterByPredicateMapper.class);
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setMapOutputValueClass(QuadWritable.class);
-
-        // Input and Output Format
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(NQuadsOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-
-        jobs[0] = job;
-
-        // Object Node Usage count job
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadObjectCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(NQuadsInputFormat.class);
-        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
-                                                          // better if this was
-                                                          // intelligently
-                                                          // configured
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-
-        return jobs;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF triple inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getTripleDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Literal Data Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleDataTypeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getQuadDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Literal Data Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadDataTypeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF triple and/or
-     * quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Literal Data Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadDataTypeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF triple inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getTripleNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Namespace Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleNamespaceCountMapper.class);
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(TextCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getQuadNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Namespace Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadNamespaceCountMapper.class);
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(TextCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF triple and/or
-     * quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Namespace Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadNamespaceCountMapper.class);
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(TextCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/pom.xml b/jena-hadoop-rdf/pom.xml
deleted file mode 100644
index 83f2819..0000000
--- a/jena-hadoop-rdf/pom.xml
+++ /dev/null
@@ -1,97 +0,0 @@
-<!-- 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. -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <artifactId>jena-elephas</artifactId>
-  <version>0.9.0-SNAPSHOT</version>
-  <packaging>pom</packaging>
-
-  <parent>
-    <groupId>org.apache.jena</groupId>
-    <artifactId>jena-parent</artifactId>
-    <version>12-SNAPSHOT</version>
-    <relativePath>../jena-parent</relativePath>
-  </parent>
-
-  <name>Apache Jena - Elephas</name>
-  <description>A collection of tools for working with RDF on the Hadoop platform</description>
-
-  <modules>
-    <module>jena-elephas-io</module>
-    <module>jena-elephas-common</module>
-    <module>jena-elephas-mapreduce</module>
-    <module>jena-elephas-stats</module>
-  </modules>
-
-  <!-- Properties common across all profiles -->
-  <properties>
-    <plugin.compiler.version>2.5.1</plugin.compiler.version>
-    <arq.version>2.12.2-SNAPSHOT</arq.version>
-    <junit.version>4.11</junit.version>
-    <mrunit.version>1.0.0</mrunit.version>
-  </properties>
-
-	<!-- Profiles to allow building for different Hadoop versions -->
-  <profiles>
-    <!-- Hadoop 2.x Latest -->
-    <profile>
-      <id>hadoop_2x</id>
-      <activation>
-        <activeByDefault>true</activeByDefault>
-      </activation>
-      <properties>
-        <hadoop.version>2.5.0</hadoop.version>
-      </properties>
-    </profile>
-  </profiles>
-
-  <dependencyManagement>
-    <dependencies>
-      <!-- Hadoop Dependencies -->
-      <dependency>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-common</artifactId>
-        <version>${hadoop.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-mapreduce-client-common</artifactId>
-        <version>${hadoop.version}</version>
-      </dependency>
-      
-      <!-- Jena Dependencies -->
-      <dependency>
-        <groupId>org.apache.jena</groupId>
-        <artifactId>jena-arq</artifactId>
-        <version>${arq.version}</version>
-      </dependency>
-
-      <!-- Test Dependencies -->
-      <dependency>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-minicluster</artifactId>
-        <version>${hadoop.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>junit</groupId>
-        <artifactId>junit</artifactId>
-        <version>${junit.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.mrunit</groupId>
-        <artifactId>mrunit</artifactId>
-        <version>${mrunit.version}</version>
-        <classifier>hadoop2</classifier>
-      </dependency>
-    </dependencies>
-  </dependencyManagement>
-</project>

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 89229f0..0a64185 100644
--- a/pom.xml
+++ b/pom.xml
@@ -77,7 +77,7 @@
         <!-- <module>jena-maven-tools</module>    -->
         <!-- <module>apache-jena-libs</module>    -->
         <!-- <module>apache-jena</module>         -->
-        <!-- <module>jena-hadoop-rdf</module> -->
+        <!-- <module>jena-elephas</module> -->
         <module>jena-extras</module>
       </modules>
     </profile>
@@ -116,7 +116,7 @@
         <module>jena-jdbc</module>
         <module>jena-maven-tools</module>
         
-        <module>jena-hadoop-rdf</module>
+        <module>jena-elephas</module>
 
         <module>apache-jena-libs</module>
         <module>apache-jena</module>


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountTests.java
new file mode 100644
index 0000000..e589b3a
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountTests.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
+import org.apache.jena.hadoop.rdf.mapreduce.count.AbstractNodeTupleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.junit.Test;
+
+
+/**
+ * Abstract tests for mappers derived from
+ * {@link AbstractNodeTupleNodeCountMapper}
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleNodeCountTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        AbstractMapperTests<LongWritable, T, NodeWritable, LongWritable> {
+
+    /**
+     * Generates tuples for the tests
+     * 
+     * @param driver
+     *            Driver
+     * @param num
+     *            Number of tuples to generate
+     */
+    protected void generateData(MapDriver<LongWritable, T, NodeWritable, LongWritable> driver, int num) {
+        LongWritable expectedCount = new LongWritable(1);
+        for (int i = 0; i < num; i++) {
+            LongWritable key = new LongWritable(i);
+            T value = this.createValue(i);
+            NodeWritable[] nodes = this.getNodes(value);
+
+            driver.addInput(key, value);
+            for (NodeWritable n : nodes) {
+                driver.addOutput(n, expectedCount);
+            }
+        }
+    }
+
+    /**
+     * Creates a tuple value
+     * 
+     * @param i
+     *            Index
+     * @return Tuple value
+     */
+    protected abstract T createValue(int i);
+
+    /**
+     * Splits the tuple value into its constituent nodes
+     * 
+     * @param tuple
+     *            Tuple value
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] getNodes(T tuple);
+
+    /**
+     * Runs a node count test
+     * 
+     * @param num
+     *            Number of tuples to generate
+     * @throws IOException
+     */
+    protected void testNodeCount(int num) throws IOException {
+        MapDriver<LongWritable, T, NodeWritable, LongWritable> driver = this.getMapDriver();
+        this.generateData(driver, num);
+        driver.runTest();
+    }
+
+    /**
+     * Tests node counting
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void node_count_01() throws IOException {
+        this.testNodeCount(1);
+    }
+
+    /**
+     * Tests node counting
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void node_count_02() throws IOException {
+        this.testNodeCount(100);
+    }
+
+    /**
+     * Tests node counting
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void node_count_03() throws IOException {
+        this.testNodeCount(1000);
+    }
+
+    /**
+     * Tests node counting
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void node_count_04() throws IOException {
+        this.testNodeCount(2500);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapReduceTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapReduceTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapReduceTest.java
new file mode 100644
index 0000000..b453bee
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapReduceTest.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Tests for the {@link TripleNodeCountMapper} used in conjunction with the
+ * {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ */
+public class QuadNodeCountMapReduceTest extends AbstractNodeTupleNodeCountReducedTests<Quad, QuadWritable> {
+
+    @Override
+    protected Mapper<LongWritable, QuadWritable, NodeWritable, LongWritable> getMapperInstance() {
+        return new QuadNodeCountMapper<LongWritable>();
+    }
+
+    @Override
+    protected Reducer<NodeWritable, LongWritable, NodeWritable, LongWritable> getReducerInstance() {
+        return new NodeCountReducer();
+    }
+
+    @Override
+    protected QuadWritable createValue(int i) {
+        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
+                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
+    }
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapperTest.java
new file mode 100644
index 0000000..869fc06
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapperTest.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Tests for the {@link QuadNodeCountMapper}
+ * 
+ * 
+ * 
+ */
+public class QuadNodeCountMapperTest extends AbstractNodeTupleNodeCountTests<Quad, QuadWritable> {
+
+    @Override
+    protected Mapper<LongWritable, QuadWritable, NodeWritable, LongWritable> getInstance() {
+        return new QuadNodeCountMapper<LongWritable>();
+    }
+
+    @Override
+    protected QuadWritable createValue(int i) {
+        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
+                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
+    }
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapReduceTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapReduceTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapReduceTest.java
new file mode 100644
index 0000000..660cfe7
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapReduceTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Tests for the {@link TripleNodeCountMapper} used in conjunction with the
+ * {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ */
+public class TripleNodeCountMapReduceTest extends AbstractNodeTupleNodeCountReducedTests<Triple, TripleWritable> {
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, NodeWritable, LongWritable> getMapperInstance() {
+        return new TripleNodeCountMapper<LongWritable>();
+    }
+    
+
+    @Override
+    protected Reducer<NodeWritable, LongWritable, NodeWritable, LongWritable> getReducerInstance() {
+        return new NodeCountReducer();
+    }
+
+    @Override
+    protected TripleWritable createValue(int i) {
+        return new TripleWritable(
+                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapperTest.java
new file mode 100644
index 0000000..fdac378
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapperTest.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.mapreduce.count;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Tests for the {@link TripleNodeCountMapper}
+ * 
+ * 
+ * 
+ */
+public class TripleNodeCountMapperTest extends AbstractNodeTupleNodeCountTests<Triple, TripleWritable> {
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, NodeWritable, LongWritable> getInstance() {
+        return new TripleNodeCountMapper<LongWritable>();
+    }
+
+    @Override
+    protected TripleWritable createValue(int i) {
+        return new TripleWritable(
+                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterTests.java
new file mode 100644
index 0000000..23c5661
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterTests.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
+import org.apache.jena.hadoop.rdf.mapreduce.filter.AbstractNodeTupleFilterMapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.junit.Test;
+
+
+/**
+ * Abstract tests for {@link AbstractNodeTupleFilterMapper} implementations
+ * which filter based on the validity of tuples
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleFilterTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        AbstractMapperTests<LongWritable, T, LongWritable, T> {
+
+    protected final void generateData(MapDriver<LongWritable, T, LongWritable, T> driver, int num) {
+        for (int i = 0; i < num; i++) {
+            LongWritable key = new LongWritable(i);
+            if (i % 2 == 0 && !this.noValidInputs()) {
+                T value = this.createValidValue(i);
+                driver.addInput(key, value);
+                if (!this.isInverted())
+                    driver.addOutput(key, value);
+            } else {
+                T value = this.createInvalidValue(i);
+                driver.addInput(key, value);
+                if (this.isInverted())
+                    driver.addOutput(key, value);
+            }
+        }
+    }
+
+    /**
+     * Method that may be overridden for testing filters where all the generated
+     * data will be rejected as invalid
+     * 
+     * @return True if there are no valid inputs, false otherwise (default)
+     */
+    protected boolean noValidInputs() {
+        return false;
+    }
+
+    /**
+     * Method that may be overridden for testing filters with inverted mode
+     * enabled i.e. where normally valid input is considered invalid and vice
+     * versa
+     * 
+     * @return True if inverted, false otherwise (default)
+     */
+    protected boolean isInverted() {
+        return false;
+    }
+
+    /**
+     * Creates an invalid value
+     * 
+     * @param i
+     *            Key
+     * @return Invalid value
+     */
+    protected abstract T createInvalidValue(int i);
+
+    /**
+     * Creates a valid value
+     * 
+     * @param i
+     *            Key
+     * @return Valid value
+     */
+    protected abstract T createValidValue(int i);
+
+    protected final void testFilterValid(int num) throws IOException {
+        MapDriver<LongWritable, T, LongWritable, T> driver = this.getMapDriver();
+        this.generateData(driver, num);
+        driver.runTest();
+    }
+
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void filter_valid_01() throws IOException {
+        this.testFilterValid(1);
+    }
+
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void filter_valid_02() throws IOException {
+        this.testFilterValid(100);
+    }
+
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void filter_valid_03() throws IOException {
+        this.testFilterValid(1000);
+    }
+
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void filter_valid_04() throws IOException {
+        this.testFilterValid(2500);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadValidityFilterTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadValidityFilterTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadValidityFilterTests.java
new file mode 100644
index 0000000..9458f83
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadValidityFilterTests.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for triple filter mappers that check triple validity
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractQuadValidityFilterTests extends AbstractNodeTupleFilterTests<Quad, QuadWritable> {
+
+    @Override
+    protected QuadWritable createValidValue(int i) {
+        return new QuadWritable(
+                new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
+                        NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
+                                XSDDatatype.XSDinteger)));
+    }
+
+    @Override
+    protected QuadWritable createInvalidValue(int i) {
+        switch (i % 8) {
+        case 0:
+            // Invalid to use Literal as Graph
+            return new QuadWritable(new Quad(NodeFactory.createLiteral("invalid"), NodeFactory.createURI("http://subjects/" + i),
+                    NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
+                            XSDDatatype.XSDinteger)));
+        case 1:
+            // Invalid to use Variable as Graph
+            return new QuadWritable(new Quad(NodeFactory.createVariable("invalid"),
+                    NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+        case 2:
+            // Invalid to use Literal as Subject
+            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createLiteral("invalid"),
+                    NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
+                            XSDDatatype.XSDinteger)));
+        case 3:
+            // Invalid to use Variable as Subject
+            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createVariable("invalid"),
+                    NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
+                            XSDDatatype.XSDinteger)));
+        case 4:
+            // Invalid to use Blank Node as Predicate
+            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
+                    NodeFactory.createAnon(), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+        case 5:
+            // Invalid to use Literal as Predicate
+            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
+                    NodeFactory.createLiteral("invalid"), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+        case 6:
+            // Invalid to use Variable as Predicate
+            return new QuadWritable(
+                    new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
+                            NodeFactory.createVariable("invalid"), NodeFactory.createLiteral(Integer.toString(i),
+                                    XSDDatatype.XSDinteger)));
+        default:
+            // Invalid to use Variable as Object
+            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
+                    NodeFactory.createURI("http://predicate"), NodeFactory.createVariable("invalid")));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleValidityFilterTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleValidityFilterTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleValidityFilterTests.java
new file mode 100644
index 0000000..3c41710
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleValidityFilterTests.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.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for triple filter mappers that check triple validity
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractTripleValidityFilterTests extends AbstractNodeTupleFilterTests<Triple, TripleWritable> {
+
+    @Override
+    protected TripleWritable createValidValue(int i) {
+        return new TripleWritable(
+                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+    @Override
+    protected TripleWritable createInvalidValue(int i) {
+        switch (i % 6) {
+        case 0:
+            // Invalid to use Literal as Subject
+            return new TripleWritable(new Triple(NodeFactory.createLiteral("invalid"), NodeFactory.createURI("http://predicate"),
+                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+        case 1:
+            // Invalid to use Variable as Subject
+            return new TripleWritable(new Triple(NodeFactory.createVariable("invalid"),
+                    NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
+                            XSDDatatype.XSDinteger)));
+        case 2:
+            // Invalid to use Blank Node as Predicate
+            return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createAnon(),
+                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+        case 3:
+            // Invalid to use Literal as Predicate
+            return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i),
+                    NodeFactory.createLiteral("invalid"), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+        case 4:
+            // Invalid to use Variable as Predicate
+            return new TripleWritable(
+                    new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createVariable("invalid"),
+                            NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+        default:
+            // Invalid to use Variable as Object
+            return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i),
+                    NodeFactory.createURI("http://predicate"), NodeFactory.createVariable("invalid")));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByNoPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByNoPredicateMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByNoPredicateMapperTest.java
new file mode 100644
index 0000000..4c9fb5a
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByNoPredicateMapperTest.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.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
+
+/**
+ * Tests for the {@link TripleFilterByPredicateUriMapper} where there are no
+ * predicates and thus all data must be invalid
+ * 
+ * 
+ * 
+ */
+public class TripleFilterByNoPredicateMapperTest extends TripleFilterByPredicateMapperTest {
+
+    private static final String[] EMPTY_PREDICATE_POOL = new String[0];
+
+    /**
+     * Gets the pool of predicates considered valid
+     * 
+     * @return Predicate pool
+     */
+    @Override
+    protected String[] getPredicatePool() {
+        return EMPTY_PREDICATE_POOL;
+    }
+
+    @Override
+    protected boolean noValidInputs() {
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByPredicateMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByPredicateMapperTest.java
new file mode 100644
index 0000000..add363e
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByPredicateMapperTest.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.mapreduce.filter;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.vocabulary.RDF;
+import com.hp.hpl.jena.vocabulary.RDFS;
+
+/**
+ * Tests for the {@link TripleFilterByPredicateUriMapper}
+ * 
+ * 
+ * 
+ */
+public class TripleFilterByPredicateMapperTest extends AbstractNodeTupleFilterTests<Triple, TripleWritable> {
+
+    private static final String[] DEFAULT_PREDICATE_POOL = new String[] { RDF.type.getURI(), RDFS.range.getURI(),
+            RDFS.domain.getURI() };
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, LongWritable, TripleWritable> getInstance() {
+        return new TripleFilterByPredicateUriMapper<LongWritable>();
+    }
+
+    @Override
+    protected void configureDriver(MapDriver<LongWritable, TripleWritable, LongWritable, TripleWritable> driver) {
+        super.configureDriver(driver);
+        driver.getContext().getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, this.getPredicatePool());
+    }
+
+    /**
+     * Gets the pool of predicates considered valid
+     * 
+     * @return Predicate pool
+     */
+    protected String[] getPredicatePool() {
+        return DEFAULT_PREDICATE_POOL;
+    }
+
+    @Override
+    protected TripleWritable createInvalidValue(int i) {
+        return new TripleWritable(
+                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+    @Override
+    protected TripleWritable createValidValue(int i) {
+        String[] predicates = this.getPredicatePool();
+        if (predicates.length == 0) return this.createInvalidValue(i);
+        return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI(predicates[i
+                % predicates.length]), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByNoPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByNoPredicateMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByNoPredicateMapperTest.java
new file mode 100644
index 0000000..fb7dd25
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByNoPredicateMapperTest.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.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
+
+/**
+ * Tests for the {@link TripleFilterByPredicateUriMapper} where there are no
+ * predicates and thus all data must be invalid
+ * 
+ * 
+ * 
+ */
+public class TripleInvertedFilterByNoPredicateMapperTest extends TripleInvertedFilterByPredicateMapperTest {
+
+    private static final String[] EMPTY_PREDICATE_POOL = new String[0];
+
+    /**
+     * Gets the pool of predicates considered valid
+     * 
+     * @return Predicate pool
+     */
+    @Override
+    protected String[] getPredicatePool() {
+        return EMPTY_PREDICATE_POOL;
+    }
+
+    @Override
+    protected boolean noValidInputs() {
+        return true;
+    }
+    
+    @Override
+    protected boolean isInverted() {
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByPredicateMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByPredicateMapperTest.java
new file mode 100644
index 0000000..b24e9c2
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByPredicateMapperTest.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.vocabulary.RDF;
+import com.hp.hpl.jena.vocabulary.RDFS;
+
+/**
+ * Tests for the {@link TripleFilterByPredicateUriMapper}
+ * 
+ * 
+ * 
+ */
+public class TripleInvertedFilterByPredicateMapperTest extends AbstractNodeTupleFilterTests<Triple, TripleWritable> {
+
+    private static final String[] DEFAULT_PREDICATE_POOL = new String[] { RDF.type.getURI(), RDFS.range.getURI(),
+            RDFS.domain.getURI() };
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, LongWritable, TripleWritable> getInstance() {
+        return new TripleFilterByPredicateUriMapper<LongWritable>();
+    }
+
+    @Override
+    protected void configureDriver(MapDriver<LongWritable, TripleWritable, LongWritable, TripleWritable> driver) {
+        super.configureDriver(driver);
+        driver.getContext().getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, this.getPredicatePool());
+        driver.getContext().getConfiguration().setBoolean(RdfMapReduceConstants.FILTER_INVERT, true);
+    }
+
+    @Override
+    protected boolean isInverted() {
+        return true;
+    }
+
+    /**
+     * Gets the pool of predicates considered valid
+     * 
+     * @return Predicate pool
+     */
+    protected String[] getPredicatePool() {
+        return DEFAULT_PREDICATE_POOL;
+    }
+
+    @Override
+    protected TripleWritable createInvalidValue(int i) {
+        return new TripleWritable(
+                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+    @Override
+    protected TripleWritable createValidValue(int i) {
+        String[] predicates = this.getPredicatePool();
+        if (predicates.length == 0)
+            return this.createInvalidValue(i);
+        return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI(predicates[i
+                % predicates.length]), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+}

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

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

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingTests.java
new file mode 100644
index 0000000..1e362d1
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingTests.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
+import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.junit.Test;
+
+
+/**
+ * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
+ * implementations
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleGroupingTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        AbstractMapperTests<LongWritable, T, NodeWritable, T> {
+
+    /**
+     * Generates data for use in tests
+     * 
+     * @param driver
+     *            Driver
+     * @param num
+     *            Number of tuples to generate
+     */
+    protected void generateData(MapDriver<LongWritable, T, NodeWritable, T> driver, int num) {
+        for (int i = 0; i < num; i++) {
+            LongWritable inputKey = new LongWritable(i);
+            T value = this.createValue(i);
+            NodeWritable outputKey = this.getOutputKey(value);
+
+            driver.addInput(inputKey, value);
+            driver.addOutput(outputKey, value);
+        }
+    }
+
+    protected abstract T createValue(int i);
+
+    protected abstract NodeWritable getOutputKey(T tuple);
+
+    protected final void testGrouping(int num) throws IOException {
+        MapDriver<LongWritable, T, NodeWritable, T> driver = this.getMapDriver();
+        this.generateData(driver, num);
+        driver.runTest();
+    }
+
+    /**
+     * Test grouping tuples by nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void grouping_01() throws IOException {
+        this.testGrouping(1);
+    }
+    
+    /**
+     * Test grouping tuples by nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void grouping_02() throws IOException {
+        this.testGrouping(100);
+    }
+    
+    /**
+     * Test grouping tuples by nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void grouping_03() throws IOException {
+        this.testGrouping(1000);
+    }
+    
+    /**
+     * Test grouping tuples by nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void grouping_04() throws IOException {
+        this.testGrouping(2500);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingTests.java
new file mode 100644
index 0000000..562512f
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingTests.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.mapreduce.group.AbstractQuadGroupingMapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for {@link AbstractQuadGroupingMapper} implementations
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractQuadGroupingTests extends AbstractNodeTupleGroupingTests<Quad, QuadWritable> {
+
+    @Override
+    protected QuadWritable createValue(int i) {
+        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
+                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingTests.java
new file mode 100644
index 0000000..8e3d33c
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingTests.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.mapreduce.group.AbstractTripleGroupingMapper;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for {@link AbstractTripleGroupingMapper} implementations
+ * 
+ *
+ */
+public abstract class AbstractTripleGroupingTests extends AbstractNodeTupleGroupingTests<Triple, TripleWritable> {
+
+    @Override
+    protected TripleWritable createValue(int i) {
+        return new TripleWritable(
+                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapperTest.java
new file mode 100644
index 0000000..370f820
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapperTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupByGraphMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Tests for the {@link QuadGroupByGraphMapper}
+ * 
+ * 
+ * 
+ */
+public class QuadGroupByGraphMapperTest extends AbstractQuadGroupingTests {
+
+    @Override
+    protected NodeWritable getOutputKey(QuadWritable tuple) {
+        return new NodeWritable(tuple.get().getGraph());
+    }
+
+    @Override
+    protected Mapper<LongWritable, QuadWritable, NodeWritable, QuadWritable> getInstance() {
+        return new QuadGroupByGraphMapper<LongWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapperTest.java
new file mode 100644
index 0000000..919696d
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapperTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupByObjectMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Tests for the {@link QuadGroupByObjectMapper}
+ * 
+ * 
+ * 
+ */
+public class QuadGroupByObjectMapperTest extends AbstractQuadGroupingTests {
+
+    @Override
+    protected NodeWritable getOutputKey(QuadWritable tuple) {
+        return new NodeWritable(tuple.get().getObject());
+    }
+
+    @Override
+    protected Mapper<LongWritable, QuadWritable, NodeWritable, QuadWritable> getInstance() {
+        return new QuadGroupByObjectMapper<LongWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapperTest.java
new file mode 100644
index 0000000..2a1b520
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapperTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupByPredicateMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Tests for the {@link QuadGroupByPredicateMapper}
+ * 
+ * 
+ * 
+ */
+public class QuadGroupByPredicateMapperTest extends AbstractQuadGroupingTests {
+
+    @Override
+    protected NodeWritable getOutputKey(QuadWritable tuple) {
+        return new NodeWritable(tuple.get().getPredicate());
+    }
+
+    @Override
+    protected Mapper<LongWritable, QuadWritable, NodeWritable, QuadWritable> getInstance() {
+        return new QuadGroupByPredicateMapper<LongWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapperTest.java
new file mode 100644
index 0000000..3b0bb1a
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapperTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupBySubjectMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Tests for the {@link QuadGroupBySubjectMapper}
+ * 
+ * 
+ * 
+ */
+public class QuadGroupBySubjectMapperTest extends AbstractQuadGroupingTests {
+
+    @Override
+    protected NodeWritable getOutputKey(QuadWritable tuple) {
+        return new NodeWritable(tuple.get().getSubject());
+    }
+
+    @Override
+    protected Mapper<LongWritable, QuadWritable, NodeWritable, QuadWritable> getInstance() {
+        return new QuadGroupBySubjectMapper<LongWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapperTest.java
new file mode 100644
index 0000000..c769bb4
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapperTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupByObjectMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Tests for the {@link TripleGroupByObjectMapper}
+ * 
+ * 
+ * 
+ */
+public class TripleGroupByObjectMapperTest extends AbstractTripleGroupingTests {
+
+    @Override
+    protected NodeWritable getOutputKey(TripleWritable tuple) {
+        return new NodeWritable(tuple.get().getObject());
+    }
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, NodeWritable, TripleWritable> getInstance() {
+        return new TripleGroupByObjectMapper<LongWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapperTest.java
new file mode 100644
index 0000000..e41cf50
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapperTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupByPredicateMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Tests for the {@link TripleGroupByPredicateMapper}
+ * 
+ * 
+ * 
+ */
+public class TripleGroupByPredicateMapperTest extends AbstractTripleGroupingTests {
+
+    @Override
+    protected NodeWritable getOutputKey(TripleWritable tuple) {
+        return new NodeWritable(tuple.get().getPredicate());
+    }
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, NodeWritable, TripleWritable> getInstance() {
+        return new TripleGroupByPredicateMapper<LongWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapperTest.java
new file mode 100644
index 0000000..64335f4
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapperTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupBySubjectMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Tests for the {@link TripleGroupBySubjectMapper}
+ * 
+ * 
+ * 
+ */
+public class TripleGroupBySubjectMapperTest extends AbstractTripleGroupingTests {
+
+    @Override
+    protected NodeWritable getOutputKey(TripleWritable tuple) {
+        return new NodeWritable(tuple.get().getSubject());
+    }
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, NodeWritable, TripleWritable> getInstance() {
+        return new TripleGroupBySubjectMapper<LongWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesTests.java
new file mode 100644
index 0000000..1de39cd
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesTests.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
+import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.junit.Test;
+
+
+/**
+ * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
+ * implementations
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleSplitToNodesTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        AbstractMapperTests<LongWritable, T, LongWritable, NodeWritable> {
+
+    /**
+     * Generates data for use in tests
+     * 
+     * @param driver
+     *            Driver
+     * @param num
+     *            Number of tuples to generate
+     */
+    protected void generateData(MapDriver<LongWritable, T, LongWritable, NodeWritable> driver, int num) {
+        for (int i = 0; i < num; i++) {
+            LongWritable key = new LongWritable(i);
+            T value = this.createValue(i);
+            NodeWritable[] nodes = this.getNodes(value);
+
+            driver.addInput(key, value);
+            for (NodeWritable n : nodes) {
+                driver.addOutput(key, n);
+            }
+        }
+    }
+
+    protected abstract T createValue(int i);
+
+    protected abstract NodeWritable[] getNodes(T tuple);
+
+    protected final void testSplitToNodes(int num) throws IOException {
+        MapDriver<LongWritable, T, LongWritable, NodeWritable> driver = this.getMapDriver();
+        this.generateData(driver, num);
+        driver.runTest();
+    }
+
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void split_to_nodes_01() throws IOException {
+        this.testSplitToNodes(1);
+    }
+    
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void split_to_nodes_02() throws IOException {
+        this.testSplitToNodes(100);
+    }
+    
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void split_to_nodes_03() throws IOException {
+        this.testSplitToNodes(1000);
+    }
+    
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void split_to_nodes_04() throws IOException {
+        this.testSplitToNodes(2500);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesTests.java
new file mode 100644
index 0000000..71136c2
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesTests.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
+import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitWithNodesMapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.junit.Test;
+
+
+/**
+ * Abstract tests for {@link AbstractNodeTupleSplitWithNodesMapper}
+ * implementations
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleSplitWithNodesTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        AbstractMapperTests<LongWritable, T, T, NodeWritable> {
+
+    /**
+     * Generates data for use in tests
+     * 
+     * @param driver
+     *            Driver
+     * @param num
+     *            Number of tuples to generate
+     */
+    protected void generateData(MapDriver<LongWritable, T, T, NodeWritable> driver, int num) {
+        for (int i = 0; i < num; i++) {
+            LongWritable key = new LongWritable(i);
+            T value = this.createValue(i);
+            NodeWritable[] nodes = this.getNodes(value);
+
+            driver.addInput(key, value);
+            for (NodeWritable n : nodes) {
+                driver.addOutput(value, n);
+            }
+        }
+    }
+
+    protected abstract T createValue(int i);
+
+    protected abstract NodeWritable[] getNodes(T tuple);
+
+    protected final void testSplitToNodes(int num) throws IOException {
+        MapDriver<LongWritable, T, T, NodeWritable> driver = this.getMapDriver();
+        this.generateData(driver, num);
+        driver.runTest();
+    }
+
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void split_to_nodes_01() throws IOException {
+        this.testSplitToNodes(1);
+    }
+    
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void split_to_nodes_02() throws IOException {
+        this.testSplitToNodes(100);
+    }
+    
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void split_to_nodes_03() throws IOException {
+        this.testSplitToNodes(1000);
+    }
+    
+    /**
+     * Test splitting tuples into their constituent nodes
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void split_to_nodes_04() throws IOException {
+        this.testSplitToNodes(2500);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitToNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitToNodesTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitToNodesTests.java
new file mode 100644
index 0000000..07192c6
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitToNodesTests.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
+ * implementations that work on Quads
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractQuadSplitToNodesTests extends AbstractNodeTupleSplitToNodesTests<Quad, QuadWritable> {
+
+    @Override
+    protected QuadWritable createValue(int i) {
+        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
+                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
+    }
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
deleted file mode 100644
index b60380d..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.thrift;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftTripleReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-public class ThriftTripleInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new ThriftTripleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
deleted file mode 100644
index 0b36e93..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.trig;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.trig.TriGReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Input format for TriG
- * 
- * 
- * 
- */
-public class TriGInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TriGReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
deleted file mode 100644
index 723c5c3..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.trix;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.trix.TriXReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-/**
- * Input format for TriX
- */
-public class TriXInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TriXReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
deleted file mode 100644
index c7771b6..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.turtle;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.turtle.TurtleReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Turtle input format
- * 
- * 
- * 
- */
-public class TurtleInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TurtleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
deleted file mode 100644
index a9e692e..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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.util;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * A block input stream which can is a wrapper around another input stream which
- * restricts reading to a specific number of bytes and can report the number of
- * bytes read
- * <p>
- * The class assumes that the underlying input stream has already been seeked to
- * the appropriate start point
- * </p>
- * 
- * 
- * 
- */
-public final class BlockInputStream extends TrackedInputStream {
-
-    private long limit = Long.MAX_VALUE;
-
-    /**
-     * Creates a new tracked input stream
-     * 
-     * @param input
-     *            Input stream to track
-     * @param limit
-     *            Maximum number of bytes to read from the stream
-     */
-    public BlockInputStream(InputStream input, long limit) {
-        super(input);
-        if (limit < 0)
-            throw new IllegalArgumentException("limit must be >= 0");
-        this.limit = limit;
-    }
-
-    @Override
-    public int read() throws IOException {
-        if (this.bytesRead >= this.limit) {
-            return -1;
-        }
-        return super.read();
-    }
-
-    @Override
-    public int available() throws IOException {
-        if (this.bytesRead >= this.limit) {
-            return 0;
-        }
-        return super.available();
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        if (len == 0) {
-            return 0;
-        } else if (this.bytesRead >= this.limit) {
-            return -1;
-        } else if (len > this.limit - this.bytesRead) {
-            len = (int) (this.limit - this.bytesRead);
-        }
-        return super.read(b, off, len);
-    }
-
-    @Override
-    public long skip(long n) throws IOException {
-        if (n == 0) {
-            return 0;
-        } else if (this.bytesRead >= this.limit) {
-            return -1;
-        } else if (n > this.limit - this.bytesRead) {
-            n = this.limit - this.bytesRead;
-        }
-        return super.skip(n);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
deleted file mode 100644
index 372b22c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.util;
-
-import java.util.UUID;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.riot.lang.LabelToNode;
-import org.apache.jena.riot.system.ErrorHandlerFactory;
-import org.apache.jena.riot.system.IRIResolver;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.system.ParserProfileBase;
-import org.apache.jena.riot.system.Prologue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * RDF IO utility functions
- * 
- * 
- * 
- */
-public class RdfIOUtils {
-    private static final Logger LOGGER = LoggerFactory.getLogger(RdfIOUtils.class);
-
-    /**
-     * Private constructor prevents instantiation
-     */
-    private RdfIOUtils() {
-    }
-
-    /**
-     * Creates a parser profile for the given job context
-     * 
-     * @param context
-     *            Context
-     * @param path
-     *            File path
-     * @return Parser profile
-     */
-    public static ParserProfile createParserProfile(JobContext context, Path path) {
-        Prologue prologue = new Prologue(null, IRIResolver.createNoResolve());
-        UUID seed = RdfIOUtils.getSeed(context, path);
-        LabelToNode labelMapping = LabelToNode.createScopeByDocumentHash(seed);
-        return new ParserProfileBase(prologue, ErrorHandlerFactory.errorHandlerStd, labelMapping);
-    }
-
-    /**
-     * Selects a seed for use in generating blank node identifiers
-     * 
-     * @param context
-     *            Job Context
-     * @param path
-     *            File path
-     * @return Seed
-     */
-    public static UUID getSeed(JobContext context, Path path) {
-        // This is to ensure that blank node allocation policy is constant when
-        // subsequent MapReduce jobs need that
-        String jobId = context.getJobID().toString();
-        if (jobId == null) {
-            jobId = String.valueOf(System.currentTimeMillis());
-            LOGGER.warn(
-                    "Job ID was not set, using current milliseconds of {}. Sequence of MapReduce jobs must carefully handle blank nodes.",
-                    jobId);
-        }
-
-        if (!context.getConfiguration().getBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, false)) {
-            // Using normal file scoped blank node allocation
-            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());
-        } else {
-            // Using globally scoped blank node allocation
-            LOGGER.warn(
-                    "Using globally scoped blank node allocation policy from Job Details (ID={}) - this is unsafe if your RDF inputs did not originate from a previous job",
-                    jobId);
-            
-            return new UUID(jobId.hashCode(), 0);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
deleted file mode 100644
index 92e2df5..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.util;
-
-import java.io.InputStream;
-
-/**
- * An input stream that tracks the number of bytes read
- * 
- * 
- * 
- */
-public abstract class TrackableInputStream extends InputStream {
-
-    /**
-     * Gets the number of bytes read
-     * 
-     * @return Number of bytes read
-     */
-    public abstract long getBytesRead();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
deleted file mode 100644
index e51a866..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.util;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * A tracked input stream which can is a wrapper around another input stream and
- * can report the number of bytes read
- * 
- * 
- * 
- */
-public class TrackedInputStream extends TrackableInputStream {
-
-    protected InputStream input;
-    protected long bytesRead = 0, lastMark;
-
-    /**
-     * Creates a new tracked input stream
-     * 
-     * @param input
-     *            Input stream to track
-     */
-    public TrackedInputStream(InputStream input) {
-        if (input == null)
-            throw new NullPointerException("Input cannot be null");
-        this.input = input;
-    }
-
-    @Override
-    public int read() throws IOException {
-        int read = this.input.read();
-        if (read >= 0)
-            this.bytesRead++;
-        return read;
-    }
-
-    @Override
-    public long getBytesRead() {
-        return this.bytesRead;
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.input.close();
-    }
-
-    @Override
-    public int available() throws IOException {
-        return this.input.available();
-    }
-
-    @Override
-    public synchronized void mark(int readlimit) {
-        this.input.mark(readlimit);
-        this.lastMark = this.bytesRead;
-    }
-
-    @Override
-    public boolean markSupported() {
-        return this.input.markSupported();
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        if (len == 0) return 0;
-        int read = this.input.read(b, off, len);
-        if (read > 0)
-            this.bytesRead += read;
-        return read;
-    }
-
-    @Override
-    public int read(byte[] b) throws IOException {
-        return this.read(b, 0, b.length);
-    }
-
-    @Override
-    public synchronized void reset() throws IOException {
-        this.input.reset();
-        this.bytesRead = this.lastMark;
-    }
-
-    @Override
-    public long skip(long n) throws IOException {
-        if (n == 0)
-            return 0;
-        long skipped = 0;
-        byte[] buffer = new byte[16];
-        int readSize = Math.min(buffer.length, n > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) n);
-        int read;
-        do {
-            if (n - skipped > readSize) {
-                read = this.input.read(buffer, 0, readSize);
-            } else {
-                read = this.input.read(buffer, 0, (int) (n - skipped));
-            }
-            if (read > 0) {
-                this.bytesRead += read;
-                skipped += read;
-            }
-        } while (skipped < n && read >= 0);
-
-        return skipped;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
deleted file mode 100644
index 845c709..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A tracked piped quads stream
- * 
- * 
- * 
- */
-public class TrackedPipedQuadsStream extends TrackedPipedRDFStream<Quad> {
-
-    /**
-     * Creates a new stream
-     * 
-     * @param sink
-     *            Sink
-     * @param input
-     *            Input stream
-     */
-    public TrackedPipedQuadsStream(PipedRDFIterator<Quad> sink, TrackableInputStream input) {
-        super(sink, input);
-    }
-
-    @Override
-    public void triple(Triple triple) {
-        // Triples are discarded
-    }
-
-    @Override
-    public void quad(Quad quad) {
-        this.receive(quad);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java
deleted file mode 100644
index 6e910be..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.util;
-
-import java.util.LinkedList;
-import java.util.Queue;
-
-import org.apache.jena.riot.lang.PipedRDFIterator;
-import org.apache.jena.riot.lang.PipedRDFStream;
-
-/**
- * A tracked piped RDF stream
- * 
- * 
- * 
- * @param <T>
- *            Type corresponding to a supported RDF primitive
- */
-public abstract class TrackedPipedRDFStream<T> extends PipedRDFStream<T> {
-
-    private TrackableInputStream input;
-    private Queue<Long> positions = new LinkedList<Long>();
-
-    protected TrackedPipedRDFStream(PipedRDFIterator<T> sink, TrackableInputStream input) {
-        super(sink);
-        this.input = input;
-    }
-
-    @Override
-    protected void receive(T t) {
-        // Track positions the input stream is at as we receive inputs
-        synchronized (this.positions) {
-            this.positions.add(this.input.getBytesRead());
-        }
-        super.receive(t);
-    }
-
-    /**
-     * Gets the next position
-     * 
-     * @return Position
-     */
-    public Long getPosition() {
-        synchronized (this.positions) {
-            return this.positions.poll();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java
deleted file mode 100644
index 2040c4f..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A tracked piped triples stream
- * 
- * 
- * 
- */
-public class TrackedPipedTriplesStream extends TrackedPipedRDFStream<Triple> {
-
-    /**
-     * Creates a tracked triples stream
-     * 
-     * @param sink
-     *            Sink
-     * @param input
-     *            Input stream
-     */
-    public TrackedPipedTriplesStream(PipedRDFIterator<Triple> sink, TrackableInputStream input) {
-        super(sink, input);
-    }
-
-    @Override
-    public void triple(Triple triple) {
-        receive(triple);
-    }
-
-    @Override
-    public void quad(Quad quad) {
-        // Quads are discarded
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
deleted file mode 100644
index 02fbf9c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedNodeTupleWriter;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-
-
-/**
- * Abstract output format for formats that use a
- * {@link AbstractBatchedNodeTupleWriter} as their writer
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TTuple>
- *            Tuple type
- * @param <TValue>
- *            Writable tuple type i.e. the value type
- */
-public abstract class AbstractBatchedNodeTupleOutputFormat<TKey, TTuple, TValue extends AbstractNodeTupleWritable<TTuple>> extends
-        AbstractNodeTupleOutputFormat<TKey, TTuple, TValue> {
-
-    @Override
-    protected RecordWriter<TKey, TValue> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
-        long batchSize = config.getLong(RdfIOConstants.OUTPUT_BATCH_SIZE, RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE);
-        return this.getRecordWriter(writer, batchSize);
-    }
-    
-    protected abstract RecordWriter<TKey, TValue> getRecordWriter(Writer writer, long batchSize);
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
deleted file mode 100644
index cfc98bd..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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.OutputStreamWriter;
-import java.io.Writer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Abstract output format which takes pairs with Node keys and arbitrary values
- * and writes them as a simple line based text file
- * 
- * 
- * 
- * @param <TValue> Value type
- */
-public abstract class AbstractNodeOutputFormat<TValue> extends FileOutputFormat<NodeWritable, TValue> {
-    
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeOutputFormat.class);
-
-    @Override
-    public RecordWriter<NodeWritable, TValue> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
-        Configuration config = context.getConfiguration();
-        boolean isCompressed = getCompressOutput(context);
-        CompressionCodec codec = null;
-        String extension = this.getFileExtension();
-        if (isCompressed) {
-            Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(context, GzipCodec.class);
-            codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, config);
-            extension += codec.getDefaultExtension();
-        }
-        Path file = getDefaultWorkFile(context, extension);
-        LOG.info("Writing output to file " + file);
-        FileSystem fs = file.getFileSystem(config);
-        if (!isCompressed) {
-            FSDataOutputStream fileOut = fs.create(file, false);
-            return this.getRecordWriter(new OutputStreamWriter(fileOut), config);
-        } else {
-            FSDataOutputStream fileOut = fs.create(file, false);
-            return this.getRecordWriter(new OutputStreamWriter(codec.createOutputStream(fileOut)), config);
-        }
-    }
-
-    /**
-     * Gets the file extension to use for output
-     * 
-     * @return File extension including the '.'
-     */
-    protected String getFileExtension() {
-        return ".nodes";
-    }
-
-    /**
-     * Gets the record writer to use
-     * 
-     * @param writer
-     *            Writer to write output to
-     * @param config
-     *            Configuration
-     * @return Record writer
-     */
-    protected abstract RecordWriter<NodeWritable, TValue> getRecordWriter(Writer writer, Configuration config);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
deleted file mode 100644
index c4a34f5..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.OutputStreamWriter;
-import java.io.Writer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract implementation of an output format for line based tuple formats
- * where the key is ignored and only the tuple values will be output
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Tuple value type
- * @param <T>
- *            Writable node tuple type
- * 
- */
-public abstract class AbstractNodeTupleOutputFormat<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        FileOutputFormat<TKey, T> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleOutputFormat.class);
-
-    @Override
-    public RecordWriter<TKey, T> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
-        Configuration config = context.getConfiguration();
-        boolean isCompressed = getCompressOutput(context);
-        CompressionCodec codec = null;
-        
-        // Build the output file path
-        String extension = this.getFileExtension();
-        if (isCompressed) {
-            // Add compression extension if applicable
-            Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(context, GzipCodec.class);
-            codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, config);
-            extension += codec.getDefaultExtension();
-        }
-        Path file = getDefaultWorkFile(context, extension);
-        LOG.info("Writing output to file " + file);
-        
-        // Open the file appropriately and create a record writer for it
-        FileSystem fs = file.getFileSystem(config);
-        if (!isCompressed) {
-            FSDataOutputStream fileOut = fs.create(file, false);
-            return this.getRecordWriter(new OutputStreamWriter(fileOut), config, file);
-        } else {
-            FSDataOutputStream fileOut = fs.create(file, false);
-            return this.getRecordWriter(new OutputStreamWriter(codec.createOutputStream(fileOut)), config, file);
-        }
-    }
-
-    /**
-     * Gets the file extension to use for output
-     * 
-     * @return File extension including the '.'
-     */
-    protected abstract String getFileExtension();
-
-    /**
-     * Gets the record writer to use
-     * 
-     * @param writer
-     *            Writer to write output to
-     * @param config
-     *            Configuration
-     * @param outputPath
-     *            Output path being written to
-     * @return Record writer
-     * @throws IOException
-     *             May be thrown if a record writer cannot be obtained for any
-     *             reason
-     */
-    protected abstract RecordWriter<TKey, T> getRecordWriter(Writer writer, Configuration config, Path outputPath)
-            throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/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
deleted file mode 100644
index 30999ae..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index cc9fe2f..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 3eaf0d7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index d9d4189..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 8f4797a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index a8cbeac..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index a8ab017..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 56935bb..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 51b9b75..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index e5fa114..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 6c9a9ea..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index bd07bff..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 73e40bc..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 6f33e29..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 0047095..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index c67b3da..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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";
-    }
-
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/pom.xml
----------------------------------------------------------------------
diff --git a/jena-elephas/pom.xml b/jena-elephas/pom.xml
new file mode 100644
index 0000000..0246c2e
--- /dev/null
+++ b/jena-elephas/pom.xml
@@ -0,0 +1,97 @@
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>jena-elephas</artifactId>
+  <version>0.9.0-SNAPSHOT</version>
+  <packaging>pom</packaging>
+
+  <parent>
+    <groupId>org.apache.jena</groupId>
+    <artifactId>jena-parent</artifactId>
+    <version>12-SNAPSHOT</version>
+    <relativePath>../jena-parent</relativePath>
+  </parent>
+
+  <name>Apache Jena - Elephas</name>
+  <description>A collection of tools for working with RDF on the Hadoop platform</description>
+
+  <modules>
+    <module>jena-elephas-io</module>
+    <module>jena-elephas-common</module>
+    <module>jena-elephas-mapreduce</module>
+    <module>jena-elephas-stats</module>
+  </modules>
+
+  <!-- Properties common across all profiles -->
+  <properties>
+    <plugin.compiler.version>2.5.1</plugin.compiler.version>
+    <arq.version>2.12.2-SNAPSHOT</arq.version>
+    <junit.version>4.11</junit.version>
+    <mrunit.version>1.0.0</mrunit.version>
+  </properties>
+
+	<!-- Profiles to allow building for different Hadoop versions -->
+  <profiles>
+    <!-- Hadoop 2.x Latest -->
+    <profile>
+      <id>hadoop_2x</id>
+      <activation>
+        <activeByDefault>true</activeByDefault>
+      </activation>
+      <properties>
+        <hadoop.version>2.6.0</hadoop.version>
+      </properties>
+    </profile>
+  </profiles>
+
+  <dependencyManagement>
+    <dependencies>
+      <!-- Hadoop Dependencies -->
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-common</artifactId>
+        <version>${hadoop.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-mapreduce-client-common</artifactId>
+        <version>${hadoop.version}</version>
+      </dependency>
+      
+      <!-- Jena Dependencies -->
+      <dependency>
+        <groupId>org.apache.jena</groupId>
+        <artifactId>jena-arq</artifactId>
+        <version>${arq.version}</version>
+      </dependency>
+
+      <!-- Test Dependencies -->
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-minicluster</artifactId>
+        <version>${hadoop.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>junit</groupId>
+        <artifactId>junit</artifactId>
+        <version>${junit.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.mrunit</groupId>
+        <artifactId>mrunit</artifactId>
+        <version>${mrunit.version}</version>
+        <classifier>hadoop2</classifier>
+      </dependency>
+    </dependencies>
+  </dependencyManagement>
+</project>

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/LICENSE
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/LICENSE b/jena-hadoop-rdf/LICENSE
deleted file mode 100644
index 68c771a..0000000
--- a/jena-hadoop-rdf/LICENSE
+++ /dev/null
@@ -1,176 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/NOTICE
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/NOTICE b/jena-hadoop-rdf/NOTICE
deleted file mode 100644
index f87b36f..0000000
--- a/jena-hadoop-rdf/NOTICE
+++ /dev/null
@@ -1,5 +0,0 @@
-Apache Jena
-Copyright 2011-2014 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/pom.xml b/jena-hadoop-rdf/hadoop-rdf-mapreduce/pom.xml
deleted file mode 100644
index b907850..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/pom.xml
+++ /dev/null
@@ -1,87 +0,0 @@
-<!--
- 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.
- -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<groupId>org.apache.jena</groupId>
-		<artifactId>jena-hadoop-rdf</artifactId>
-		<version>0.9.0-SNAPSHOT</version>
-	</parent>
-	<artifactId>jena-hadoop-rdf-mapreduce</artifactId>
-	<name>Apache Jena - RDF Tools for Hadoop - Map/Reduce</name>
-	<description>Contains some basic Map/Reduce implementations for working with RDF on Hadoop</description>
-
-	<dependencies>
-		<!-- Internal Project Dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-hadoop-rdf-common</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<!-- Hadoop Dependencies -->
-		<!-- Note these will be provided on the Hadoop cluster hence the provided 
-			scope -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-mapreduce-client-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Jena dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-arq</artifactId>
-		</dependency>
-
-		<!-- Test Dependencies -->
-		<dependency>
-			<groupId>junit</groupId>
-			<artifactId>junit</artifactId>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.mrunit</groupId>
-			<artifactId>mrunit</artifactId>
-			<scope>test</scope>
-			<classifier>hadoop2</classifier>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<plugins>
-			<!-- JAR plugin to ensure tests jar is built -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyMapper.java
deleted file mode 100644
index 306a697..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyMapper.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A mapper which discards the value replacing it with the key
- * 
- *
- * @param <TKey> Key type
- * @param <TValue> Value type
- */
-public class KeyMapper<TKey, TValue> extends Mapper<TKey, TValue, TKey, TKey> {
-    private static final Logger LOG = LoggerFactory.getLogger(KeyMapper.class);
-
-    private boolean tracing = false;
-    
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOG.isTraceEnabled();
-    }
-
-    @Override
-    protected void map(TKey key, TValue value, Context context) throws IOException,
-            InterruptedException {
-        if (this.tracing) {
-            LOG.trace("Key = {}", key);
-        }
-        context.write(key, key);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyPlusNullMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyPlusNullMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyPlusNullMapper.java
deleted file mode 100644
index a6e9a6a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyPlusNullMapper.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A mapper which discards the value replacing it with a null
- * 
- *
- * @param <TKey> Key type
- * @param <TValue> Value type
- */
-public class KeyPlusNullMapper<TKey, TValue> extends Mapper<TKey, TValue, TKey, NullWritable> {
-    private static final Logger LOG = LoggerFactory.getLogger(KeyPlusNullMapper.class);
-
-    private boolean tracing = false;
-    
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOG.isTraceEnabled();
-    }
-
-    @Override
-    protected void map(TKey key, TValue value, Context context) throws IOException,
-            InterruptedException {
-        if (this.tracing) {
-            LOG.trace("Key = {}", key);
-        }
-        context.write(key, NullWritable.get());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyReducer.java
deleted file mode 100644
index 7805f16..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyReducer.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Reducer;
-
-/**
- * A reducer that outputs a single pair consists of the key as both fields ignoring the values
- * @author rvesse
- *
- * @param <TKey> Key
- * @param <TValue> Value
- */
-public class KeyReducer<TKey, TValue> extends Reducer<TKey, TValue, TKey, TKey> {
-
-    @Override
-    protected void reduce(TKey key, Iterable<TValue> values, Context context)
-            throws IOException, InterruptedException {
-        context.write(key, key);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyMapper.java
deleted file mode 100644
index 7a48c1d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyMapper.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A mapper which discards the value, moves the key to the value position and uses a null key
- * 
- *
- * @param <TKey> Key type
- * @param <TValue> Value type
- */
-public class NullPlusKeyMapper<TKey, TValue> extends Mapper<TKey, TValue, NullWritable, TKey> {
-    private static final Logger LOG = LoggerFactory.getLogger(NullPlusKeyMapper.class);
-
-    private boolean tracing = false;
-    
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOG.isTraceEnabled();
-    }
-
-    @Override
-    protected void map(TKey key, TValue value, Context context) throws IOException,
-            InterruptedException {
-        if (this.tracing) {
-            LOG.trace("Key = {}", key);
-        }
-        context.write(NullWritable.get(), key);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyReducer.java
deleted file mode 100644
index dfc6ec1..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyReducer.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A reducer that outputs a single pair consists of a null as the key and the key as the value
- * @author rvesse
- *
- * @param <TKey> Key
- * @param <TValue> Value
- */
-public class NullPlusKeyReducer<TKey, TValue> extends Reducer<TKey, TValue, NullWritable, TKey> {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(NullPlusKeyReducer.class);
-    private boolean tracing = false;
-    
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOGGER.isTraceEnabled();
-    }
-
-    @Override
-    protected void reduce(TKey key, Iterable<TValue> values, Context context)
-            throws IOException, InterruptedException {
-        if (this.tracing) {
-            LOGGER.trace("Input Key = {}", key);
-            Iterator<TValue> iter = values.iterator();
-            while (iter.hasNext()) {
-                LOGGER.trace("Input Value = {}", iter.next());
-            }
-        }
-        context.write(NullWritable.get(), key);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueMapper.java
deleted file mode 100644
index a5ac199..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueMapper.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A mapper which discards the key replacing it with a null leaving the value as is
- * 
- *
- * @param <TKey> Key type
- * @param <TValue> Value type
- */
-public class NullPlusValueMapper<TKey, TValue> extends Mapper<TKey, TValue, NullWritable, TValue> {
-    private static final Logger LOG = LoggerFactory.getLogger(NullPlusValueMapper.class);
-
-    private boolean tracing = false;
-    
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOG.isTraceEnabled();
-    }
-
-    @Override
-    protected void map(TKey key, TValue value, Context context) throws IOException,
-            InterruptedException {
-        if (this.tracing) {
-            LOG.trace("Value = {}", value);
-        }
-        context.write(NullWritable.get(), value);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueReducer.java
deleted file mode 100644
index c6b270f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueReducer.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A reducer that outputs a pair for each value consisting of a null key and the
- * value
- * 
- * @author rvesse
- * 
- * @param <TKey>
- *            Key
- * @param <TValue>
- *            Value
- */
-public class NullPlusValueReducer<TKey, TValue> extends Reducer<TKey, TValue, NullWritable, TValue> {
-    private static final Logger LOGGER = LoggerFactory.getLogger(NullPlusValueReducer.class);
-    private boolean tracing = false;
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOGGER.isTraceEnabled();
-    }
-
-    @Override
-    protected void reduce(TKey key, Iterable<TValue> values, Context context) throws IOException, InterruptedException {
-        if (this.tracing) {
-            LOGGER.trace("Input Key = {}", key);
-        }
-        Iterator<TValue> iter = values.iterator();
-        while (iter.hasNext()) {
-            TValue value = iter.next();
-            if (tracing) {
-                LOGGER.trace("Input Value = {}", value);
-            }
-            context.write(NullWritable.get(), value);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/RdfMapReduceConstants.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/RdfMapReduceConstants.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/RdfMapReduceConstants.java
deleted file mode 100644
index 6a8cf18..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/RdfMapReduceConstants.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-/**
- * RDF Map/Reduce related constants
- * 
- * 
- * 
- */
-public class RdfMapReduceConstants {
-
-    /**
-     * Private constructor prevents instantiation
-     */
-    private RdfMapReduceConstants() {
-
-    }
-
-    /**
-     * Configuration key used to set whether the behaviour of the filter mappers
-     * is inverted. When enabled the filter mappers will invert their selection
-     * i.e. tuples that would normally be accepted will be rejected and vice
-     * versa.
-     */
-    public static final String FILTER_INVERT = "rdf.mapreduce.filter.invert";
-
-    /**
-     * Configuration key used to set a command separated list of predicate URIs
-     * to filter upon
-     */
-    public static final String FILTER_PREDICATE_URIS = "rdf.mapreduce.filter.predicate.uris";
-
-    /**
-     * Configuration key used to set a command separated list of subject URIs to
-     * filter upon
-     */
-    public static final String FILTER_SUBJECT_URIS = "rdf.mapreduce.filter.subject.uris";
-
-    /**
-     * Configuration key used to set a command separated list of object URIs to
-     * filter upon
-     */
-    public static final String FILTER_OBJECT_URIS = "rdf.mapreduce.filter.object.uris";
-
-    /**
-     * Configuration key used to set a command separated list of graph URIs to
-     * filter upon
-     */
-    public static final String FILTER_GRAPH_URIS = "rdf.mapreduce.filter.graph.uris";
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapMapper.java
deleted file mode 100644
index ef518a9..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapMapper.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A mapper which swaps the key and value around
- * 
- *
- * @param <TKey> Key type
- * @param <TValue> Value type
- */
-public class SwapMapper<TKey, TValue> extends Mapper<TKey, TValue, TValue, TKey> {
-    private static final Logger LOG = LoggerFactory.getLogger(SwapMapper.class);
-
-    private boolean tracing = false;
-    
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOG.isTraceEnabled();
-    }
-
-    @Override
-    protected void map(TKey key, TValue value, Context context) throws IOException,
-            InterruptedException {
-        if (this.tracing) {
-            LOG.trace("Key = {}", key);
-            LOG.trace("Value = {}", value);
-        }
-        context.write(value, key);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapReducer.java
deleted file mode 100644
index e7e42a0..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapReducer.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.mapreduce.Reducer;
-
-/**
- * A reducer that swaps the key and value
- * @author rvesse
- *
- * @param <TKey> Key
- * @param <TValue> Value
- */
-public class SwapReducer<TKey, TValue> extends Reducer<TKey, TValue, TValue, TKey> {
-
-    @Override
-    protected void reduce(TKey key, Iterable<TValue> values, Context context)
-            throws IOException, InterruptedException {
-        Iterator<TValue> iter = values.iterator();
-        while (iter.hasNext()) {
-            context.write(iter.next(), key);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/TextCountReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/TextCountReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/TextCountReducer.java
deleted file mode 100644
index 04b9283..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/TextCountReducer.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Reducer;
-
-/**
- * A reducer which takes text keys with a sequence of longs representing counts
- * as the values and sums the counts together into pairs consisting of a node
- * key and a count value.
- * 
- * 
- * 
- */
-public class TextCountReducer extends Reducer<Text, LongWritable, Text, LongWritable> {
-
-    @Override
-    protected void reduce(Text key, Iterable<LongWritable> values, Context context) throws IOException,
-            InterruptedException {
-        long count = 0;
-        Iterator<LongWritable> iter = values.iterator();
-        while (iter.hasNext()) {
-            count += iter.next().get();
-        }
-        context.write(key, new LongWritable(count));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueMapper.java
deleted file mode 100644
index 23ae5f0..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueMapper.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A mapper which discards the key replacing it with the value
- * 
- *
- * @param <TKey> Key type
- * @param <TValue> Value type
- */
-public class ValueMapper<TKey, TValue> extends Mapper<TKey, TValue, TValue, TValue> {
-    private static final Logger LOG = LoggerFactory.getLogger(ValueMapper.class);
-
-    private boolean tracing = false;
-    
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOG.isTraceEnabled();
-    }
-
-    @Override
-    protected void map(TKey key, TValue value, Context context) throws IOException,
-            InterruptedException {
-        if (this.tracing) {
-            LOG.trace("Value = {}", value);
-        }
-        context.write(value, value);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValuePlusNullMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValuePlusNullMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValuePlusNullMapper.java
deleted file mode 100644
index 094fb2d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValuePlusNullMapper.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A mapper which discards the key replacing it with the value and nulls out the value
- * 
- *
- * @param <TKey> Key type
- * @param <TValue> Value type
- */
-public class ValuePlusNullMapper<TKey, TValue> extends Mapper<TKey, TValue, TValue, NullWritable> {
-    private static final Logger LOG = LoggerFactory.getLogger(ValuePlusNullMapper.class);
-
-    private boolean tracing = false;
-    
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOG.isTraceEnabled();
-    }
-
-    @Override
-    protected void map(TKey key, TValue value, Context context) throws IOException,
-            InterruptedException {
-        if (this.tracing) {
-            LOG.trace("Value = {}", value);
-        }
-        context.write(value, NullWritable.get());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueReducer.java
deleted file mode 100644
index 7d25799..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueReducer.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.mapreduce.Reducer;
-
-/**
- * A reducer that outputs a pair for each value consisting of the value as both the key and value
- * @author rvesse
- *
- * @param <TKey> Key
- * @param <TValue> Value
- */
-public class ValueReducer<TKey, TValue> extends Reducer<TKey, TValue, TValue, TValue> {
-
-    @Override
-    protected void reduce(TKey key, Iterable<TValue> values, Context context)
-            throws IOException, InterruptedException {
-        Iterator<TValue> iter = values.iterator();
-        while (iter.hasNext()) {
-            TValue value = iter.next();
-            context.write(value, value);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducer.java
deleted file mode 100644
index daf61d4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducer.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
-import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Abstract reducer which takes in tuples grouped by some node and generating
- * initial characteristic sets.
- * <p>
- * This produces the characteristic sets as both the key and value so that in a
- * subsequent job the characteristic steps may be further combined together to
- * total up the usage counts appropriately.
- * </p>
- * <p>
- * It is important to note that the output from this mapper can be very large
- * and since it typically needs to be written to HDFS before being processed by
- * further jobs it is strongly recommended that you use appropriate output
- * compression
- * </p>
- * 
- * 
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractCharacteristicSetGeneratingReducer<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        Reducer<NodeWritable, T, CharacteristicSetWritable, NullWritable> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractCharacteristicSetGeneratingReducer.class);
-
-    private boolean tracing = false;
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOG.isTraceEnabled();
-    }
-
-    @Override
-    protected void reduce(NodeWritable key, Iterable<T> values, Context context) throws IOException, InterruptedException {
-        Map<NodeWritable, CharacteristicWritable> characteristics = new TreeMap<NodeWritable, CharacteristicWritable>();
-
-        // Firstly need to find individual characteristics
-        Iterator<T> iter = values.iterator();
-        while (iter.hasNext()) {
-            T tuple = iter.next();
-            NodeWritable predicate = this.getPredicate(tuple);
-
-            if (characteristics.containsKey(predicate)) {
-                characteristics.get(predicate).increment();
-            } else {
-                characteristics.put(predicate, new CharacteristicWritable(predicate.get()));
-            }
-        }
-
-        // Then we need to produce all the possible characteristic sets based on
-        // this information
-        List<CharacteristicWritable> cs = new ArrayList<CharacteristicWritable>(characteristics.values());
-        if (cs.size() == 0)
-            return;
-        for (int i = 1; i <= cs.size(); i++) {
-            this.outputSets(cs, i, context);
-        }
-    }
-
-    /**
-     * Output all sets of a given size
-     * 
-     * @param cs
-     *            Characteristics
-     * @param perSet
-     *            Set size
-     * @param context
-     *            Context to output sets to
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected void outputSets(List<CharacteristicWritable> cs, int perSet, Context context) throws IOException,
-            InterruptedException {
-        if (perSet == 1) {
-            for (CharacteristicWritable c : cs) {
-                CharacteristicSetWritable set = new CharacteristicSetWritable(c);
-                context.write(set, NullWritable.get());
-                if (this.tracing) {
-                    LOG.trace("Key = {}", set);
-                }
-            }
-        } else if (perSet == cs.size()) {
-            CharacteristicSetWritable set = new CharacteristicSetWritable();
-            for (CharacteristicWritable c : cs) {
-                set.add(c);
-            }
-            context.write(set, NullWritable.get());
-            if (this.tracing) {
-                LOG.trace("Key = {}", set);
-            }
-        } else {
-            CharacteristicWritable[] members = new CharacteristicWritable[perSet];
-            this.combinations(cs, perSet, 0, members, context);
-        }
-    }
-
-    /**
-     * Calculate all available combinations of N elements from the given
-     * characteristics
-     * 
-     * @param cs
-     *            Characteristics
-     * @param len
-     *            Desired number of elements
-     * @param startPosition
-     *            Start position
-     * @param result
-     *            Result array to fill
-     * @param context
-     *            Context to write completed combinations to
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected final void combinations(List<CharacteristicWritable> cs, int len, int startPosition,
-            CharacteristicWritable[] result, Context context) throws IOException, InterruptedException {
-        if (len == 0) {
-            CharacteristicSetWritable set = new CharacteristicSetWritable(result);
-            context.write(set, NullWritable.get());
-            if (this.tracing) {
-                LOG.trace("Key = {}", set);
-            }
-            return;
-        }
-        for (int i = startPosition; i <= cs.size() - len; i++) {
-            result[result.length - len] = cs.get(i);
-            combinations(cs, len - 1, i + 1, result, context);
-        }
-    }
-
-    /**
-     * Gets the predicate for the tuple
-     * 
-     * @param tuple
-     *            Tuple
-     * @return
-     */
-    protected abstract NodeWritable getPredicate(T tuple);
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducer.java
deleted file mode 100644
index e70698a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducer.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Reducer which takes in characteristic sets and sums up all their usage counts
- * 
- * 
- */
-public class CharacteristicSetReducer extends
-        Reducer<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(CharacteristicSetReducer.class);
-    private boolean tracing = false;
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.tracing = LOG.isTraceEnabled();
-    }
-
-    @Override
-    protected void reduce(CharacteristicSetWritable key, Iterable<CharacteristicSetWritable> values, Context context)
-            throws IOException, InterruptedException {
-        Iterator<CharacteristicSetWritable> iter = values.iterator();
-        CharacteristicSetWritable output = new CharacteristicSetWritable(0);
-
-        if (this.tracing) {
-            LOG.trace("Key = {}", key);
-        }
-
-        while (iter.hasNext()) {
-            CharacteristicSetWritable set = iter.next();
-            if (this.tracing) {
-                LOG.trace("Value = {}", set);
-            }
-            output.add(set);
-        }
-
-        context.write(output, NullWritable.get());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/QuadCharacteristicSetGeneratingReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/QuadCharacteristicSetGeneratingReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/QuadCharacteristicSetGeneratingReducer.java
deleted file mode 100644
index d11cd56..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/QuadCharacteristicSetGeneratingReducer.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A reducer which converts quads grouped by some node into characteristic sets
- * 
- * 
- * 
- */
-public class QuadCharacteristicSetGeneratingReducer extends AbstractCharacteristicSetGeneratingReducer<Quad, QuadWritable> {
-
-    @Override
-    protected NodeWritable getPredicate(QuadWritable tuple) {
-        return new NodeWritable(tuple.get().getPredicate());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducer.java
deleted file mode 100644
index 6515c91..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducer.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A reducer which converts triples grouped by some node into characteristic
- * sets
- * 
- * 
- * 
- */
-public class TripleCharacteristicSetGeneratingReducer extends AbstractCharacteristicSetGeneratingReducer<Triple, TripleWritable> {
-
-    @Override
-    protected NodeWritable getPredicate(TripleWritable tuple) {
-        return new NodeWritable(tuple.get().getPredicate());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java
deleted file mode 100644
index 7c56d1c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-
-/**
- * Abstract mapper class for mappers which split node tuple values into pairs of
- * node keys with a long value of 1. Can be used in conjunction with a
- * {@link NodeCountReducer} to count the usages of each unique node.
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleNodeCountMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        Mapper<TKey, T, NodeWritable, LongWritable> {
-    
-    private LongWritable initialCount = new LongWritable(1);
-
-    @Override
-    protected void map(TKey key, T value, Context context) throws IOException,
-            InterruptedException {
-        NodeWritable[] ns = this.getNodes(value);
-        for (NodeWritable n : ns) {
-            context.write(n, this.initialCount);
-        }
-    }
-
-    /**
-     * Gets the nodes of the tuple which are to be counted
-     * 
-     * @param tuple
-     *            Tuple
-     * @return Nodes
-     */
-    protected abstract NodeWritable[] getNodes(T tuple);
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java
deleted file mode 100644
index a5460f6..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-
-/**
- * A reducer which takes node keys with a sequence of longs representing counts
- * as the values and sums the counts together into pairs consisting of a node
- * key and a count value.
- * 
- * 
- * 
- */
-public class NodeCountReducer extends Reducer<NodeWritable, LongWritable, NodeWritable, LongWritable> {
-
-    @Override
-    protected void reduce(NodeWritable key, Iterable<LongWritable> values, Context context) throws IOException,
-            InterruptedException {
-        long count = 0;
-        Iterator<LongWritable> iter = values.iterator();
-        while (iter.hasNext()) {
-            count += iter.next().get();
-        }
-        context.write(key, new LongWritable(count));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java
deleted file mode 100644
index 5c2485b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A mapper for counting node usages within quads designed primarily for use in
- * conjunction with {@link NodeCountReducer}
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadNodeCountMapper<TKey> extends AbstractNodeTupleNodeCountMapper<TKey, Quad, QuadWritable> {
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        Quad q = tuple.get();
-        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
-                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java
deleted file mode 100644
index 4108841..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A mapper for counting node usages within triples designed primarily for use
- * in conjunction with {@link NodeCountReducer}
- * 
- * 
- * 
- * @param <TKey> Key type
- */
-public class TripleNodeCountMapper<TKey> extends AbstractNodeTupleNodeCountMapper<TKey, Triple, TripleWritable> {
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        Triple t = tuple.get();
-        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
-                new NodeWritable(t.getObject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java
deleted file mode 100644
index 7ea7ead..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.datatypes;
-
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * A mapper for counting data type usages within quads designed primarily for
- * use in conjunction with {@link NodeCountReducer}
- * <p>
- * This mapper extracts the data types for typed literal objects and converts
- * them into nodes so they can be counted
- * </p>
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadDataTypeCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
-
-    private static final NodeWritable[] EMPTY = new NodeWritable[0];
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        Node object = tuple.get().getObject();
-        if (!object.isLiteral())
-            return EMPTY;
-        String dtUri = object.getLiteralDatatypeURI();
-        if (dtUri == null)
-            return EMPTY;
-        return new NodeWritable[] { new NodeWritable(NodeFactory.createURI(dtUri)) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java
deleted file mode 100644
index 2c294be..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.datatypes;
-
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * A mapper for counting data type usages within triples designed primarily for
- * use in conjunction with {@link NodeCountReducer}
- * <p>
- * This mapper extracts the data types for typed literal objects and converts
- * them into nodes so they can be counted
- * </p>
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class TripleDataTypeCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
-
-    private static final NodeWritable[] EMPTY = new NodeWritable[0];
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        Node object = tuple.get().getObject();
-        if (!object.isLiteral())
-            return EMPTY;
-        String dtUri = object.getLiteralDatatypeURI();
-        if (dtUri == null)
-            return EMPTY;
-        return new NodeWritable[] { new NodeWritable(NodeFactory.createURI(dtUri)) };
-    }
-}


[15/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
deleted file mode 100644
index 146c836..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.turtle.TurtleInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.turtle.TurtleOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link TurtleInputFormat}
- */
-public class TurtleBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.TURTLE;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".ttl";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new TurtleInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new TurtleOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new TurtleInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
deleted file mode 100644
index 1f18a95..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-
-
-/**
- * 
- * 
- * @param <TValue>
- * @param <T>
- */
-public abstract class AbstractCompressedNodeTupleInputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        AbstractNodeTupleInputFormatTests<TValue, T> {
-    
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
-        return config;
-    }
-
-    @Override
-    protected OutputStream getOutputStream(File f) throws IOException {
-        CompressionCodec codec = this.getCompressionCodec();
-        if (codec instanceof Configurable) {
-            ((Configurable) codec).setConf(this.prepareConfiguration());
-        }
-        FileOutputStream fileOutput = new FileOutputStream(f, false);
-        return codec.createOutputStream(fileOutput);
-    }
-
-    /**
-     * Gets the compression codec to use
-     * 
-     * @return Compression codec
-     */
-    protected abstract CompressionCodec getCompressionCodec();
-
-    /**
-     * Indicates whether inputs can be split, defaults to false for compressed
-     * input tests
-     */
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
deleted file mode 100644
index 312aae7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for Quad input formats
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedQuadsInputFormatTests extends
-        AbstractCompressedNodeTupleInputFormatTests<Quad, QuadWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected void generateTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateBadTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write("<http://broken\n".getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
-        boolean bad = false;
-        for (int i = 0; i < num; i++, bad = !bad) {
-            if (bad) {
-                output.write("<http://broken\n".getBytes(utf8));
-            } else {
-                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
-            }
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
deleted file mode 100644
index f0f0caf..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for Triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedTriplesInputFormatTests extends
-        AbstractCompressedNodeTupleInputFormatTests<Triple, TripleWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected void generateTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateBadTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write("<http://broken\n".getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
-        boolean bad = false;
-        for (int i = 0; i < num; i++, bad = !bad) {
-            if (bad) {
-                output.write("<http://broken\n".getBytes(utf8));
-            } else {
-                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
-            }
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
deleted file mode 100644
index be2b1d7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * 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;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
-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.query.Dataset;
-import com.hp.hpl.jena.query.DatasetFactory;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ModelFactory;
-import com.hp.hpl.jena.rdf.model.Property;
-import com.hp.hpl.jena.rdf.model.Resource;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for compressed whole file quad formats
- * 
- * 
- */
-public abstract class AbstractCompressedWholeFileQuadInputFormatTests extends
-        AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
-        return config;
-    }
-
-    @Override
-    protected OutputStream getOutputStream(File f) throws IOException {
-        CompressionCodec codec = this.getCompressionCodec();
-        if (codec instanceof Configurable) {
-            ((Configurable) codec).setConf(this.prepareConfiguration());
-        }
-        FileOutputStream fileOutput = new FileOutputStream(f, false);
-        return codec.createOutputStream(fileOutput);
-    }
-
-    /**
-     * Gets the compression codec to use
-     * 
-     * @return Compression codec
-     */
-    protected abstract CompressionCodec getCompressionCodec();
-
-    /**
-     * Indicates whether inputs can be split, defaults to false for compressed
-     * input tests
-     */
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-
-    private void writeTuples(Dataset ds, OutputStream output) {
-        RDFDataMgr.write(output, ds, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
-    }
-
-    /**
-     * Gets the RDF language to write out generated tuples in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-    private void writeGoodTuples(OutputStream output, int num) throws IOException {
-        Dataset ds = DatasetFactory.createMem();
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num; i++) {
-            if (i % 100 == 0) {
-                ds.addNamedModel("http://example.org/graphs/" + (i / 100), m);
-                m = ModelFactory.createDefaultModel();
-            }
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        if (!m.isEmpty()) {
-            ds.addNamedModel("http://example.org/graphs/extra", m);
-        }
-        this.writeTuples(ds, output);
-    }
-
-    @Override
-    protected final void generateTuples(OutputStream output, int num) throws IOException {
-        this.writeGoodTuples(output, num);
-        output.close();
-    }
-
-    @Override
-    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
-        // Write good data
-        this.writeGoodTuples(output, num / 2);
-
-        // Write junk data
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num / 2; i++) {
-            output.write(junk);
-        }
-
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
deleted file mode 100644
index 56dd8ca..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
-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.Triple;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ModelFactory;
-import com.hp.hpl.jena.rdf.model.Property;
-import com.hp.hpl.jena.rdf.model.Resource;
-
-/**
- * Abstract tests for compressed whole file triple formats
- * 
- * 
- */
-public abstract class AbstractCompressedWholeFileTripleInputFormatTests extends
-        AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
-
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
-        return config;
-    }
-
-    @Override
-    protected OutputStream getOutputStream(File f) throws IOException {
-        CompressionCodec codec = this.getCompressionCodec();
-        if (codec instanceof Configurable) {
-            ((Configurable) codec).setConf(this.prepareConfiguration());
-        }
-        FileOutputStream fileOutput = new FileOutputStream(f, false);
-        return codec.createOutputStream(fileOutput);
-    }
-
-    /**
-     * Gets the compression codec to use
-     * 
-     * @return Compression codec
-     */
-    protected abstract CompressionCodec getCompressionCodec();
-
-    /**
-     * Indicates whether inputs can be split, defaults to false for compressed
-     * input tests
-     */
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-
-    private void writeTuples(Model m, OutputStream output) {
-        RDFDataMgr.write(output, m, this.getRdfLanguage());
-    }
-
-    /**
-     * Gets the RDF language to write out generated tuples in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-    @Override
-    protected final void generateTuples(OutputStream output, int num) throws IOException {
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num; i++) {
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        this.writeTuples(m, output);
-        output.close();
-    }
-
-    @Override
-    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
-        // Write good data
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num / 2; i++) {
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        this.writeTuples(m, output);
-
-        // Write junk data
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num / 2; i++) {
-            output.write(junk);
-        }
-
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/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
deleted file mode 100644
index d118f29..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDQuadInputFormat;
-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/a6c0fefc/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
deleted file mode 100644
index acb9e08..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDTripleInputFormat;
-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/a6c0fefc/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
deleted file mode 100644
index e5e7066..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/a6c0fefc/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
deleted file mode 100644
index 8d2e122..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/a6c0fefc/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
deleted file mode 100644
index 292b17f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/a6c0fefc/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
deleted file mode 100644
index e5edd6a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/a6c0fefc/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
deleted file mode 100644
index 0a4a240..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/a6c0fefc/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
deleted file mode 100644
index 8a6324d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
deleted file mode 100644
index 2b8447b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.nquads;
-
-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.compressed.AbstractCompressedQuadsInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.nquads.NQuadsInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Abstract compressed NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedNQuadsInputFormatTests extends AbstractCompressedQuadsInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedNQuadsInputFormatTests(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 InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new NQuadsInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
deleted file mode 100644
index 326258a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.nquads;
-
-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.nquads.WholeFileNQuadsInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed whole file NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedWholeFileNQuadsInputFormatTests extends
-        AbstractCompressedWholeFileQuadInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedWholeFileNQuadsInputFormatTests(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.NQUADS;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new WholeFileNQuadsInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
deleted file mode 100644
index 62dc9ce..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-
-/**
- * Tests for BZipped NQuads input
- * 
- * 
- * 
- */
-public class BZipppedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZipppedNQuadsInputTest() {
-        super(".nq.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
deleted file mode 100644
index e3d670c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped NQuads input
- * 
- * 
- * 
- */
-public class BZipppedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZipppedWholeFileNQuadsInputTest() {
-        super(".nq.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
deleted file mode 100644
index 5fec23e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for deflated NQuads input
- * 
- * 
- * 
- */
-public class DeflatedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedNQuadsInputTest() {
-        super(".nq.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
deleted file mode 100644
index a31472d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for deflated NQuads input
- * 
- * 
- * 
- */
-public class DeflatedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedWholeFileNQuadsInputTest() {
-        super(".nq.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
deleted file mode 100644
index 2d40dec..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped NQuads input
- * 
- * 
- * 
- */
-public class GZippedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedNQuadsInputTest() {
-        super(".nq.gz", new GzipCodec());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
deleted file mode 100644
index 0f73eb7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped NQuads input
- * 
- * 
- * 
- */
-public class GZippedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedWholeFileNQuadsInputTest() {
-        super(".nq.gz", new GzipCodec());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
deleted file mode 100644
index 611d862..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.ntriples;
-
-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.ntriples.BlockedNTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Abstract compressed blocked NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedBlockedNTriplesInputFormatTests extends
-        AbstractCompressedWholeFileNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedBlockedNTriplesInputFormatTests(String ext, CompressionCodec codec) {
-        super(ext, codec);
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new BlockedNTriplesInputFormat();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
deleted file mode 100644
index 4d9aeba..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.ntriples;
-
-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.compressed.AbstractCompressedTriplesInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Abstract compressed NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedNTriplesInputFormatTests extends AbstractCompressedTriplesInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedNTriplesInputFormatTests(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 InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new NTriplesInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
deleted file mode 100644
index 7d4b510..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.ntriples;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.ntriples.WholeFileNTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed whole file NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedWholeFileNTriplesInputFormatTests extends
-        AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedWholeFileNTriplesInputFormatTests(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.NTRIPLES;
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new WholeFileNTriplesInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
deleted file mode 100644
index de45f17..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped blocked NTriples input
- * 
- * 
- * 
- */
-public class BZippedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedBlockedNTriplesInput() {
-        super(".nt.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
deleted file mode 100644
index fb1ab8b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-
-/**
- * Tests for BZipped NTriples input
- * 
- * 
- * 
- */
-public class BZippedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedNTriplesInputTest() {
-        super(".nt.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
deleted file mode 100644
index d50b6eb..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-
-/**
- * Tests for BZipped NTriples input
- * 
- * 
- * 
- */
-public class BZippedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedWholeFileNTriplesInputTest() {
-        super(".nt.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
deleted file mode 100644
index 9780707..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for deflated blocked NTriples input
- * 
- * 
- * 
- */
-public class DeflatedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedBlockedNTriplesInput() {
-        super(".nt.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
deleted file mode 100644
index b6dd1e9..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-
-/**
- * Tests for deflated NTriples input
- * 
- * 
- * 
- */
-public class DeflatedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedNTriplesInputTest() {
-        super(".nt.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
deleted file mode 100644
index e1e57f4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-
-/**
- * Tests for deflated NTriples input
- * 
- * 
- * 
- */
-public class DeflatedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedWholeFileNTriplesInputTest() {
-        super(".nt.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
deleted file mode 100644
index 012fd58..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped blocked NTriples input
- * 
- * 
- * 
- */
-public class GZippedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedBlockedNTriplesInput() {
-        super(".nt.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedNTriplesInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedNTriplesInputTest.java
deleted file mode 100644
index 3cb4ee0..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedNTriplesInputTest.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-
-
-
-/**
- * Tests for GZipped NTriples input
- * 
- * 
- * 
- */
-public class GZippedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedNTriplesInputTest() {
-        super(".nt.gz", new GzipCodec());
-    }
-
-}


[09/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
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/input/readers/TriplesOrQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
new file mode 100644
index 0000000..b071f67
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
@@ -0,0 +1,72 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+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.graph.Node;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A record reader that reads RDF from any triples/quads format. Triples are
+ * converted into quads in the default graph. This behaviour can be changed by
+ * deriving from this class and overriding the {@link #getGraphNode()} method
+ * 
+ * 
+ * 
+ */
+@SuppressWarnings("javadoc")
+public class TriplesOrQuadsReader extends AbstractRdfReader<Quad, QuadWritable> {
+
+    @Override
+    protected RecordReader<LongWritable, QuadWritable> selectRecordReader(Lang lang) throws IOException {
+        if (!RDFLanguages.isQuads(lang) && !RDFLanguages.isTriples(lang))
+            throw new IOException(lang.getLabel() + " is not a RDF triples/quads format");
+
+        if (HadoopRdfIORegistry.hasQuadReader(lang)) {
+            // Supports quads directly
+            return HadoopRdfIORegistry.createQuadReader(lang);
+        } else {
+            // Try to create a triples reader and wrap upwards into quads
+            // This will throw an error if a triple reader is not available
+            return new TriplesToQuadsReader(HadoopRdfIORegistry.createTripleReader(lang));
+        }
+    }
+
+    /**
+     * Gets the graph node which represents the graph into which triples will be
+     * indicated to belong to when they are converting into quads.
+     * <p>
+     * Defaults to {@link Quad#defaultGraphNodeGenerated} which represents the
+     * default graph
+     * </p>
+     * 
+     * @return Graph node
+     */
+    protected Node getGraphNode() {
+        return Quad.defaultGraphNodeGenerated;
+    }
+}

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/input/readers/TriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
new file mode 100644
index 0000000..96c4a65
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+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;
+
+/**
+ * A record reader that reads triples from any RDF triples format
+ */
+public class TriplesReader extends AbstractRdfReader<Triple, TripleWritable> {
+
+    @Override
+    protected RecordReader<LongWritable, TripleWritable> selectRecordReader(Lang lang) throws IOException {
+        if (!RDFLanguages.isTriples(lang))
+            throw new IOException(
+                    lang.getLabel()
+                            + " is not a RDF triples format, perhaps you wanted QuadsInputFormat or TriplesOrQuadsInputFormat instead?");
+
+        // This will throw an appropriate error if the language does not support
+        // triples
+        return HadoopRdfIORegistry.createTripleReader(lang);
+    }
+
+}

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/input/readers/TriplesToQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
new file mode 100644
index 0000000..a388f0e
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
@@ -0,0 +1,102 @@
+/*
+ * 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.readers;
+
+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.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A record reader that converts triples into quads by wrapping a
+ * {@code RecordReader<LongWritable, TripleWritable>} implementation
+ * 
+ * 
+ * 
+ */
+public class TriplesToQuadsReader extends RecordReader<LongWritable, QuadWritable> {
+
+    private final RecordReader<LongWritable, TripleWritable> reader;
+    private Node graph;
+
+    /**
+     * Creates a new reader
+     * 
+     * @param reader
+     *            Triple reader
+     */
+    public TriplesToQuadsReader(RecordReader<LongWritable, TripleWritable> reader) {
+        this(reader, Quad.defaultGraphNodeGenerated);
+    }
+
+    /**
+     * Creates a new reader
+     * 
+     * @param reader
+     *            Triple reader
+     * @param graphNode
+     *            Graph node
+     */
+    public TriplesToQuadsReader(RecordReader<LongWritable, TripleWritable> reader, Node graphNode) {
+        if (reader == null)
+            throw new NullPointerException("reader cannot be null");
+        if (graphNode == null)
+            throw new NullPointerException("Graph node cannot be null");
+        this.reader = reader;
+        this.graph = graphNode;
+    }
+
+    @Override
+    public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
+        this.reader.initialize(split, context);
+    }
+
+    @Override
+    public final boolean nextKeyValue() throws IOException, InterruptedException {
+        return this.reader.nextKeyValue();
+    }
+
+    @Override
+    public final LongWritable getCurrentKey() throws IOException, InterruptedException {
+        return this.reader.getCurrentKey();
+    }
+
+    @Override
+    public final QuadWritable getCurrentValue() throws IOException, InterruptedException {
+        TripleWritable t = this.reader.getCurrentValue();
+        return new QuadWritable(new Quad(this.graph, t.get()));
+    }
+
+    @Override
+    public final float getProgress() throws IOException, InterruptedException {
+        return this.reader.getProgress();
+    }
+
+    @Override
+    public final void close() throws IOException {
+        this.reader.close();
+    }
+}

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/input/readers/jsonld/JsonLDQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
new file mode 100644
index 0000000..1b3f467
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.readers.jsonld;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
new file mode 100644
index 0000000..7cdea9e
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
@@ -0,0 +1,30 @@
+/*
+ * 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.readers.jsonld;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
new file mode 100644
index 0000000..cef8ef1
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.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.input.readers.nquads;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractBlockBasedQuadReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record record for NQuads
+ * <p>
+ * This is a hybrid of the {@link NQuadsReader} and the
+ * {@link WholeFileNQuadsReader} in that it does not process individual lines
+ * rather it processes the inputs in blocks of lines parsing the whole block
+ * rather than individual lines. This provides a compromise between the higher
+ * parser setup of creating more parsers and the benefit of being able to split
+ * input files over multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class BlockedNQuadsReader extends AbstractBlockBasedQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+}

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/input/readers/nquads/NQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java
new file mode 100644
index 0000000..e00e318
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.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.readers.nquads;
+
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractLineBasedQuadReader;
+import org.apache.jena.riot.lang.LangNQuads;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.tokens.Tokenizer;
+import org.apache.jena.riot.tokens.TokenizerFactory;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A record reader for NQuads
+ * 
+ * 
+ * 
+ */
+public class NQuadsReader extends AbstractLineBasedQuadReader {
+
+    @Override
+    protected Tokenizer getTokenizer(String line) {
+        return TokenizerFactory.makeTokenizerString(line);
+    }
+
+    @Override
+    protected Iterator<Quad> getQuadsIterator(Tokenizer tokenizer, ParserProfile profile) {
+        return new LangNQuads(tokenizer, profile, null);
+    }
+
+}

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/input/readers/nquads/WholeFileNQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
new file mode 100644
index 0000000..96e6f80
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.readers.nquads;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record record for NQuads
+ * <p>
+ * Unlike the {@link NQuadsReader} this processes files as a whole rather than
+ * individual lines. This has the advantage of less parser setup overhead but
+ * the disadvantage that the input cannot be split between multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class WholeFileNQuadsReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+}

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/input/readers/ntriples/BlockedNTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java
new file mode 100644
index 0000000..7268d5a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.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.input.readers.ntriples;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractBlockBasedTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record record for NTriples
+ * <p>
+ * This is a hybrid of the {@link NTriplesReader} and the
+ * {@link WholeFileNTriplesReader} in that it does not process individual lines
+ * rather it processes the inputs in blocks of lines parsing the whole block
+ * rather than individual lines. This provides a compromise between the higher
+ * parser setup of creating more parsers and the benefit of being able to split
+ * input files over multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class BlockedNTriplesReader extends AbstractBlockBasedTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+}

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/input/readers/ntriples/NTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
new file mode 100644
index 0000000..bfc8503
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.readers.ntriples;
+
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractLineBasedTripleReader;
+import org.apache.jena.riot.lang.LangNTriples;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.tokens.Tokenizer;
+import org.apache.jena.riot.tokens.TokenizerFactory;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A record reader for NTriples
+ * 
+ * 
+ * 
+ */
+public class NTriplesReader extends AbstractLineBasedTripleReader {
+
+    @Override
+    protected Iterator<Triple> getTriplesIterator(Tokenizer tokenizer, ParserProfile profile) {
+        return new LangNTriples(tokenizer, profile, null);
+    }
+
+    @Override
+    protected Tokenizer getTokenizer(String line) {
+        return TokenizerFactory.makeTokenizerString(line);
+    }
+}

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/input/readers/ntriples/WholeFileNTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
new file mode 100644
index 0000000..c200d93
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.readers.ntriples;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record record for NTriples
+ * <p>
+ * Unlike the {@link NTriplesReader} this processes files as a whole rather than
+ * individual lines. This has the advantage of less parser setup overhead but
+ * the disadvantage that the input cannot be split between multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class WholeFileNTriplesReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+}

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/input/readers/rdfjson/RdfJsonReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
new file mode 100644
index 0000000..009024b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.rdfjson;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for RDF/JSON files
+ * 
+ * 
+ * 
+ */
+public class RdfJsonReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFJSON;
+    }
+
+}

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/input/readers/rdfxml/RdfXmlReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
new file mode 100644
index 0000000..9c374c6
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.rdfxml;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for RDF/XML files
+ * 
+ * 
+ * 
+ */
+public class RdfXmlReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFXML;
+    }
+
+}

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/input/readers/thrift/ThriftQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
new file mode 100644
index 0000000..084b1ec
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.readers.thrift;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class ThriftQuadReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+}

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/input/readers/thrift/ThriftTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
new file mode 100644
index 0000000..713bfa7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
@@ -0,0 +1,30 @@
+/*
+ * 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.readers.thrift;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class ThriftTripleReader extends AbstractWholeFileTripleReader {
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+}

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/input/readers/trig/TriGReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
new file mode 100644
index 0000000..b1b0c3c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.trig;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for TriG files
+ * 
+ * 
+ * 
+ */
+public class TriGReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.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/input/readers/trix/TriXReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
new file mode 100644
index 0000000..6873c64
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.trix;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for TriX files
+ * 
+ * 
+ * 
+ */
+public class TriXReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.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/input/readers/turtle/TurtleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
new file mode 100644
index 0000000..b3fb377
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.turtle;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for Turtle files
+ * 
+ * 
+ * 
+ */
+public class TurtleReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TURTLE;
+    }
+
+}

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/input/thrift/ThriftQuadInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
new file mode 100644
index 0000000..f75542a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.thrift;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftQuadReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+public class ThriftQuadInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new ThriftQuadReader();
+    }
+
+}

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/input/thrift/ThriftTripleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
new file mode 100644
index 0000000..b60380d
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.thrift;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftTripleReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+public class ThriftTripleInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new ThriftTripleReader();
+    }
+
+}

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/input/trig/TriGInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
new file mode 100644
index 0000000..0b36e93
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.trig;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.trig.TriGReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Input format for TriG
+ * 
+ * 
+ * 
+ */
+public class TriGInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TriGReader();
+    }
+
+}

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/input/trix/TriXInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
new file mode 100644
index 0000000..723c5c3
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.trix;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.trix.TriXReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+/**
+ * Input format for TriX
+ */
+public class TriXInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TriXReader();
+    }
+
+}

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/input/turtle/TurtleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
new file mode 100644
index 0000000..c7771b6
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.turtle;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.turtle.TurtleReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Turtle input format
+ * 
+ * 
+ * 
+ */
+public class TurtleInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TurtleReader();
+    }
+
+}

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/input/util/BlockInputStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
new file mode 100644
index 0000000..a9e692e
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
@@ -0,0 +1,94 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A block input stream which can is a wrapper around another input stream which
+ * restricts reading to a specific number of bytes and can report the number of
+ * bytes read
+ * <p>
+ * The class assumes that the underlying input stream has already been seeked to
+ * the appropriate start point
+ * </p>
+ * 
+ * 
+ * 
+ */
+public final class BlockInputStream extends TrackedInputStream {
+
+    private long limit = Long.MAX_VALUE;
+
+    /**
+     * Creates a new tracked input stream
+     * 
+     * @param input
+     *            Input stream to track
+     * @param limit
+     *            Maximum number of bytes to read from the stream
+     */
+    public BlockInputStream(InputStream input, long limit) {
+        super(input);
+        if (limit < 0)
+            throw new IllegalArgumentException("limit must be >= 0");
+        this.limit = limit;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (this.bytesRead >= this.limit) {
+            return -1;
+        }
+        return super.read();
+    }
+
+    @Override
+    public int available() throws IOException {
+        if (this.bytesRead >= this.limit) {
+            return 0;
+        }
+        return super.available();
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (len == 0) {
+            return 0;
+        } else if (this.bytesRead >= this.limit) {
+            return -1;
+        } else if (len > this.limit - this.bytesRead) {
+            len = (int) (this.limit - this.bytesRead);
+        }
+        return super.read(b, off, len);
+    }
+
+    @Override
+    public long skip(long n) throws IOException {
+        if (n == 0) {
+            return 0;
+        } else if (this.bytesRead >= this.limit) {
+            return -1;
+        } else if (n > this.limit - this.bytesRead) {
+            n = this.limit - this.bytesRead;
+        }
+        return super.skip(n);
+    }
+}

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/input/util/RdfIOUtils.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
new file mode 100644
index 0000000..372b22c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
@@ -0,0 +1,101 @@
+/*
+ * 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.util;
+
+import java.util.UUID;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.riot.lang.LabelToNode;
+import org.apache.jena.riot.system.ErrorHandlerFactory;
+import org.apache.jena.riot.system.IRIResolver;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.system.ParserProfileBase;
+import org.apache.jena.riot.system.Prologue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * RDF IO utility functions
+ * 
+ * 
+ * 
+ */
+public class RdfIOUtils {
+    private static final Logger LOGGER = LoggerFactory.getLogger(RdfIOUtils.class);
+
+    /**
+     * Private constructor prevents instantiation
+     */
+    private RdfIOUtils() {
+    }
+
+    /**
+     * Creates a parser profile for the given job context
+     * 
+     * @param context
+     *            Context
+     * @param path
+     *            File path
+     * @return Parser profile
+     */
+    public static ParserProfile createParserProfile(JobContext context, Path path) {
+        Prologue prologue = new Prologue(null, IRIResolver.createNoResolve());
+        UUID seed = RdfIOUtils.getSeed(context, path);
+        LabelToNode labelMapping = LabelToNode.createScopeByDocumentHash(seed);
+        return new ParserProfileBase(prologue, ErrorHandlerFactory.errorHandlerStd, labelMapping);
+    }
+
+    /**
+     * Selects a seed for use in generating blank node identifiers
+     * 
+     * @param context
+     *            Job Context
+     * @param path
+     *            File path
+     * @return Seed
+     */
+    public static UUID getSeed(JobContext context, Path path) {
+        // This is to ensure that blank node allocation policy is constant when
+        // subsequent MapReduce jobs need that
+        String jobId = context.getJobID().toString();
+        if (jobId == null) {
+            jobId = String.valueOf(System.currentTimeMillis());
+            LOGGER.warn(
+                    "Job ID was not set, using current milliseconds of {}. Sequence of MapReduce jobs must carefully handle blank nodes.",
+                    jobId);
+        }
+
+        if (!context.getConfiguration().getBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, false)) {
+            // Using normal file scoped blank node allocation
+            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());
+        } else {
+            // Using globally scoped blank node allocation
+            LOGGER.warn(
+                    "Using globally scoped blank node allocation policy from Job Details (ID={}) - this is unsafe if your RDF inputs did not originate from a previous job",
+                    jobId);
+            
+            return new UUID(jobId.hashCode(), 0);
+        }
+    }
+}

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/input/util/TrackableInputStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
new file mode 100644
index 0000000..92e2df5
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
@@ -0,0 +1,38 @@
+/*
+ * 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.util;
+
+import java.io.InputStream;
+
+/**
+ * An input stream that tracks the number of bytes read
+ * 
+ * 
+ * 
+ */
+public abstract class TrackableInputStream extends InputStream {
+
+    /**
+     * Gets the number of bytes read
+     * 
+     * @return Number of bytes read
+     */
+    public abstract long getBytesRead();
+
+}
\ 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/input/util/TrackedInputStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
new file mode 100644
index 0000000..e51a866
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
@@ -0,0 +1,124 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A tracked input stream which can is a wrapper around another input stream and
+ * can report the number of bytes read
+ * 
+ * 
+ * 
+ */
+public class TrackedInputStream extends TrackableInputStream {
+
+    protected InputStream input;
+    protected long bytesRead = 0, lastMark;
+
+    /**
+     * Creates a new tracked input stream
+     * 
+     * @param input
+     *            Input stream to track
+     */
+    public TrackedInputStream(InputStream input) {
+        if (input == null)
+            throw new NullPointerException("Input cannot be null");
+        this.input = input;
+    }
+
+    @Override
+    public int read() throws IOException {
+        int read = this.input.read();
+        if (read >= 0)
+            this.bytesRead++;
+        return read;
+    }
+
+    @Override
+    public long getBytesRead() {
+        return this.bytesRead;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.input.close();
+    }
+
+    @Override
+    public int available() throws IOException {
+        return this.input.available();
+    }
+
+    @Override
+    public synchronized void mark(int readlimit) {
+        this.input.mark(readlimit);
+        this.lastMark = this.bytesRead;
+    }
+
+    @Override
+    public boolean markSupported() {
+        return this.input.markSupported();
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (len == 0) return 0;
+        int read = this.input.read(b, off, len);
+        if (read > 0)
+            this.bytesRead += read;
+        return read;
+    }
+
+    @Override
+    public int read(byte[] b) throws IOException {
+        return this.read(b, 0, b.length);
+    }
+
+    @Override
+    public synchronized void reset() throws IOException {
+        this.input.reset();
+        this.bytesRead = this.lastMark;
+    }
+
+    @Override
+    public long skip(long n) throws IOException {
+        if (n == 0)
+            return 0;
+        long skipped = 0;
+        byte[] buffer = new byte[16];
+        int readSize = Math.min(buffer.length, n > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) n);
+        int read;
+        do {
+            if (n - skipped > readSize) {
+                read = this.input.read(buffer, 0, readSize);
+            } else {
+                read = this.input.read(buffer, 0, (int) (n - skipped));
+            }
+            if (read > 0) {
+                this.bytesRead += read;
+                skipped += read;
+            }
+        } while (skipped < n && read >= 0);
+
+        return skipped;
+    }
+}

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/input/util/TrackedPipedQuadsStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
new file mode 100644
index 0000000..845c709
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
@@ -0,0 +1,55 @@
+/*
+ * 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.util;
+
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A tracked piped quads stream
+ * 
+ * 
+ * 
+ */
+public class TrackedPipedQuadsStream extends TrackedPipedRDFStream<Quad> {
+
+    /**
+     * Creates a new stream
+     * 
+     * @param sink
+     *            Sink
+     * @param input
+     *            Input stream
+     */
+    public TrackedPipedQuadsStream(PipedRDFIterator<Quad> sink, TrackableInputStream input) {
+        super(sink, input);
+    }
+
+    @Override
+    public void triple(Triple triple) {
+        // Triples are discarded
+    }
+
+    @Override
+    public void quad(Quad quad) {
+        this.receive(quad);
+    }
+}

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/input/util/TrackedPipedRDFStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java
new file mode 100644
index 0000000..6e910be
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.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.input.util;
+
+import java.util.LinkedList;
+import java.util.Queue;
+
+import org.apache.jena.riot.lang.PipedRDFIterator;
+import org.apache.jena.riot.lang.PipedRDFStream;
+
+/**
+ * A tracked piped RDF stream
+ * 
+ * 
+ * 
+ * @param <T>
+ *            Type corresponding to a supported RDF primitive
+ */
+public abstract class TrackedPipedRDFStream<T> extends PipedRDFStream<T> {
+
+    private TrackableInputStream input;
+    private Queue<Long> positions = new LinkedList<Long>();
+
+    protected TrackedPipedRDFStream(PipedRDFIterator<T> sink, TrackableInputStream input) {
+        super(sink);
+        this.input = input;
+    }
+
+    @Override
+    protected void receive(T t) {
+        // Track positions the input stream is at as we receive inputs
+        synchronized (this.positions) {
+            this.positions.add(this.input.getBytesRead());
+        }
+        super.receive(t);
+    }
+
+    /**
+     * Gets the next position
+     * 
+     * @return Position
+     */
+    public Long getPosition() {
+        synchronized (this.positions) {
+            return this.positions.poll();
+        }
+    }
+}

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/input/util/TrackedPipedTriplesStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java
new file mode 100644
index 0000000..2040c4f
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.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.input.util;
+
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A tracked piped triples stream
+ * 
+ * 
+ * 
+ */
+public class TrackedPipedTriplesStream extends TrackedPipedRDFStream<Triple> {
+
+    /**
+     * Creates a tracked triples stream
+     * 
+     * @param sink
+     *            Sink
+     * @param input
+     *            Input stream
+     */
+    public TrackedPipedTriplesStream(PipedRDFIterator<Triple> sink, TrackableInputStream input) {
+        super(sink, input);
+    }
+
+    @Override
+    public void triple(Triple triple) {
+        receive(triple);
+    }
+
+    @Override
+    public void quad(Quad quad) {
+        // Quads are discarded
+    }
+
+}

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/AbstractBatchedNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
new file mode 100644
index 0000000..02fbf9c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
@@ -0,0 +1,55 @@
+/*
+ * 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.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedNodeTupleWriter;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+
+
+/**
+ * Abstract output format for formats that use a
+ * {@link AbstractBatchedNodeTupleWriter} as their writer
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TTuple>
+ *            Tuple type
+ * @param <TValue>
+ *            Writable tuple type i.e. the value type
+ */
+public abstract class AbstractBatchedNodeTupleOutputFormat<TKey, TTuple, TValue extends AbstractNodeTupleWritable<TTuple>> extends
+        AbstractNodeTupleOutputFormat<TKey, TTuple, TValue> {
+
+    @Override
+    protected RecordWriter<TKey, TValue> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
+        long batchSize = config.getLong(RdfIOConstants.OUTPUT_BATCH_SIZE, RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE);
+        return this.getRecordWriter(writer, batchSize);
+    }
+    
+    protected abstract RecordWriter<TKey, TValue> getRecordWriter(Writer writer, long batchSize);
+
+}

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/AbstractNodeOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
new file mode 100644
index 0000000..cfc98bd
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
@@ -0,0 +1,94 @@
+/*
+ * 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.OutputStreamWriter;
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Abstract output format which takes pairs with Node keys and arbitrary values
+ * and writes them as a simple line based text file
+ * 
+ * 
+ * 
+ * @param <TValue> Value type
+ */
+public abstract class AbstractNodeOutputFormat<TValue> extends FileOutputFormat<NodeWritable, TValue> {
+    
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeOutputFormat.class);
+
+    @Override
+    public RecordWriter<NodeWritable, TValue> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+        Configuration config = context.getConfiguration();
+        boolean isCompressed = getCompressOutput(context);
+        CompressionCodec codec = null;
+        String extension = this.getFileExtension();
+        if (isCompressed) {
+            Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(context, GzipCodec.class);
+            codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, config);
+            extension += codec.getDefaultExtension();
+        }
+        Path file = getDefaultWorkFile(context, extension);
+        LOG.info("Writing output to file " + file);
+        FileSystem fs = file.getFileSystem(config);
+        if (!isCompressed) {
+            FSDataOutputStream fileOut = fs.create(file, false);
+            return this.getRecordWriter(new OutputStreamWriter(fileOut), config);
+        } else {
+            FSDataOutputStream fileOut = fs.create(file, false);
+            return this.getRecordWriter(new OutputStreamWriter(codec.createOutputStream(fileOut)), config);
+        }
+    }
+
+    /**
+     * Gets the file extension to use for output
+     * 
+     * @return File extension including the '.'
+     */
+    protected String getFileExtension() {
+        return ".nodes";
+    }
+
+    /**
+     * Gets the record writer to use
+     * 
+     * @param writer
+     *            Writer to write output to
+     * @param config
+     *            Configuration
+     * @return Record writer
+     */
+    protected abstract RecordWriter<NodeWritable, TValue> getRecordWriter(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/AbstractNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
new file mode 100644
index 0000000..c4a34f5
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
@@ -0,0 +1,109 @@
+/*
+ * 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.OutputStreamWriter;
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract implementation of an output format for line based tuple formats
+ * where the key is ignored and only the tuple values will be output
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple value type
+ * @param <T>
+ *            Writable node tuple type
+ * 
+ */
+public abstract class AbstractNodeTupleOutputFormat<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        FileOutputFormat<TKey, T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleOutputFormat.class);
+
+    @Override
+    public RecordWriter<TKey, T> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+        Configuration config = context.getConfiguration();
+        boolean isCompressed = getCompressOutput(context);
+        CompressionCodec codec = null;
+        
+        // Build the output file path
+        String extension = this.getFileExtension();
+        if (isCompressed) {
+            // Add compression extension if applicable
+            Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(context, GzipCodec.class);
+            codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, config);
+            extension += codec.getDefaultExtension();
+        }
+        Path file = getDefaultWorkFile(context, extension);
+        LOG.info("Writing output to file " + file);
+        
+        // Open the file appropriately and create a record writer for it
+        FileSystem fs = file.getFileSystem(config);
+        if (!isCompressed) {
+            FSDataOutputStream fileOut = fs.create(file, false);
+            return this.getRecordWriter(new OutputStreamWriter(fileOut), config, file);
+        } else {
+            FSDataOutputStream fileOut = fs.create(file, false);
+            return this.getRecordWriter(new OutputStreamWriter(codec.createOutputStream(fileOut)), config, file);
+        }
+    }
+
+    /**
+     * Gets the file extension to use for output
+     * 
+     * @return File extension including the '.'
+     */
+    protected abstract String getFileExtension();
+
+    /**
+     * Gets the record writer to use
+     * 
+     * @param writer
+     *            Writer to write output to
+     * @param config
+     *            Configuration
+     * @param outputPath
+     *            Output path being written to
+     * @return Record writer
+     * @throws IOException
+     *             May be thrown if a record writer cannot be obtained for any
+     *             reason
+     */
+    protected abstract RecordWriter<TKey, T> getRecordWriter(Writer writer, Configuration config, Path outputPath)
+            throws IOException;
+
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/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
deleted file mode 100644
index c7564ac..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index c0202d8..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index c4eafd8..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 13c2799..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 881cf15..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 89b8f4b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 9ecef61..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 161c067..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index 9d8eeb2..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index aa178b2..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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/49c4cffe/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
deleted file mode 100644
index d48546b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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);
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java
deleted file mode 100644
index 5fc0024..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.Lang;
-import org.apache.jena.riot.RDFDataMgr;
-import org.apache.jena.riot.RDFWriterRegistry;
-
-import com.hp.hpl.jena.sparql.core.DatasetGraph;
-import com.hp.hpl.jena.sparql.core.DatasetGraphFactory;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract record writer for whole file triple formats
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public abstract class AbstractWholeFileQuadWriter<TKey> extends AbstractWholeFileNodeTupleWriter<TKey, Quad, QuadWritable> {
-
-    private DatasetGraph g = DatasetGraphFactory.createMem();
-
-    protected AbstractWholeFileQuadWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected final void add(QuadWritable value) {
-        this.g.add(value.get());
-    }
-
-    @SuppressWarnings("deprecation")
-    @Override
-    protected void writeOutput(Writer writer) {
-        RDFDataMgr.write(writer, this.g, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
-    }
-
-    /**
-     * Gets the RDF language to write the output in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
deleted file mode 100644
index bb26093..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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;
-
-/**
- * An abstract record writer for whole file triple formats
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public abstract class AbstractWholeFileTripleWriter<TKey> extends AbstractWholeFileNodeTupleWriter<TKey, Triple, TripleWritable> {
-
-    private Graph g = GraphFactory.createDefaultGraph();
-
-    protected AbstractWholeFileTripleWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected final void add(TripleWritable value) {
-        this.g.add(value.get());
-    }
-
-    @SuppressWarnings("deprecation")
-    @Override
-    protected final void writeOutput(Writer writer) {
-        RDFDataMgr.write(writer, this.g, this.getRdfLanguage());
-    }
-
-    /**
-     * Gets the RDF language to write the output in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
deleted file mode 100644
index e932e1f..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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 org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-/**
- * A record writer that converts quads into triples by stripping off the graph
- * field
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadsToTriplesWriter<TKey> extends RecordWriter<TKey, QuadWritable> {
-
-    private RecordWriter<TKey, TripleWritable> writer;
-
-    /**
-     * Creates a new writer
-     * 
-     * @param tripleWriter
-     *            Triple writer to use
-     */
-    public QuadsToTriplesWriter(RecordWriter<TKey, TripleWritable> tripleWriter) {
-        if (tripleWriter == null)
-            throw new NullPointerException("tripleWriter cannot be null");
-        this.writer = tripleWriter;
-    }
-
-    @Override
-    public void write(TKey key, QuadWritable value) throws IOException, InterruptedException {
-        this.writer.write(key, new TripleWritable(value.get().asTriple()));
-    }
-
-    @Override
-    public void close(TaskAttemptContext context) throws IOException, InterruptedException {
-        this.writer.close(context);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java
deleted file mode 100644
index e9156d2..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.system.StreamRDF;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A writer for {@link StreamRDF} based quad writers
- * 
- * @param <TKey>
- *            Key type
- */
-public class StreamRdfQuadWriter<TKey> extends
-		AbstractStreamRdfNodeTupleWriter<TKey, Quad, QuadWritable> {
-
-	public StreamRdfQuadWriter(StreamRDF stream, Writer writer) {
-		super(stream, writer);
-	}
-
-	@Override
-	protected void sendOutput(TKey key, QuadWritable value, StreamRDF stream) {
-		stream.quad(value.get());
-	}
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java
deleted file mode 100644
index 174edf3..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.system.StreamRDF;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A writer for {@link StreamRDF} based triple writers
- * 
- * @param <TKey>
- *            Key type
- */
-public class StreamRdfTripleWriter<TKey> extends AbstractStreamRdfNodeTupleWriter<TKey, Triple, TripleWritable> {
-
-	public StreamRdfTripleWriter(StreamRDF stream, Writer writer) {
-		super(stream, writer);
-	}
-
-	@Override
-	protected void sendOutput(TKey key, TripleWritable value, StreamRDF stream) {
-		stream.triple(value.get());
-	}
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
deleted file mode 100644
index 1b4b62f..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.jsonld;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileQuadWriter;
-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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
deleted file mode 100644
index 8d2079d..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.jsonld;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
-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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
deleted file mode 100644
index 8df606d..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.nquads;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractLineBasedQuadWriter;
-import org.apache.jena.riot.out.CharSpace;
-import org.apache.jena.riot.out.NodeFormatterNT;
-
-/**
- * A record writer for NQuads
- * 
- * 
- * 
- * @param <TKey>
- */
-public class NQuadsWriter<TKey> extends AbstractLineBasedQuadWriter<TKey> {
-
-    /**
-     * Creates a new writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public NQuadsWriter(Writer writer) {
-        super(writer, new NodeFormatterNT());
-    }
-
-    /**
-     * Creates a new writer using the given character space
-     * 
-     * @param writer
-     *            Writer
-     * @param charSpace
-     *            Character space
-     */
-    public NQuadsWriter(Writer writer, CharSpace charSpace) {
-        super(writer, new NodeFormatterNT(charSpace));
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
deleted file mode 100644
index 6413f21..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.ntriples;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractNodeWriter;
-import org.apache.jena.riot.out.CharSpace;
-import org.apache.jena.riot.out.NodeFormatterNT;
-
-/**
- * A NTriples based node writer
- * 
- * 
- * 
- * @param <TValue>
- *            Value type
- */
-public class NTriplesNodeWriter<TValue> extends AbstractNodeWriter<TValue> {
-
-    /**
-     * Creates a new writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public NTriplesNodeWriter(Writer writer) {
-        super(writer);
-    }
-
-    /**
-     * Creates a new writer
-     * 
-     * @param writer
-     *            Writer
-     * @param charSpace
-     *            Character space to use
-     */
-    public NTriplesNodeWriter(Writer writer, CharSpace charSpace) {
-        super(writer, new NodeFormatterNT(charSpace));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java
deleted file mode 100644
index 2b3c0b4..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.ntriples;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractLineBasedTripleWriter;
-import org.apache.jena.riot.out.CharSpace;
-import org.apache.jena.riot.out.NodeFormatterNT;
-
-/**
- * A record writer for NTriples
- * 
- * 
- * @param <TKey>
- *            Key type
- * 
- */
-public class NTriplesWriter<TKey> extends AbstractLineBasedTripleWriter<TKey> {
-
-    /**
-     * Creates a new writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public NTriplesWriter(Writer writer) {
-        super(writer, new NodeFormatterNT());
-    }
-
-    /**
-     * Creates a new writer using the given character space
-     * 
-     * @param writer
-     *            Writer
-     * @param charSpace
-     *            Character space
-     */
-    public NTriplesWriter(Writer writer, CharSpace charSpace) {
-        super(writer, new NodeFormatterNT(charSpace));
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java
deleted file mode 100644
index 30bd4fa..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.rdfjson;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record writer for RDF/JSON
- * 
- * 
- * @param <TKey>
- *            Key type
- * 
- */
-public class RdfJsonWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
-
-    /**
-     * Creates a new record writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public RdfJsonWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFJSON;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java
deleted file mode 100644
index f202914..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.rdfxml;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record writer for RDF/XML
- * 
- * 
- * @param <TKey>
- *            Key type
- * 
- */
-public class RdfXmlWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
-
-    /**
-     * Creates a new record writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public RdfXmlWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFXML;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
deleted file mode 100644
index 599dba9..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.thrift;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileQuadWriter;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-public class ThriftQuadWriter<TKey> extends AbstractWholeFileQuadWriter<TKey> {
-
-    public ThriftQuadWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
deleted file mode 100644
index 0089459..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.thrift;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-public class ThriftTripleWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
-
-    public ThriftTripleWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java
deleted file mode 100644
index 4c0c75f..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.trig;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedQuadWriter;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record writer for TriG that uses the batched approach, note that this
- * approach will produce invalid data when blank nodes span batches
- *  
- * @param <TKey>
- *            Key type
- */
-public class BatchedTriGWriter<TKey> extends AbstractBatchedQuadWriter<TKey> {
-
-	/**
-	 * Creates a new record writer
-	 * 
-	 * @param writer
-	 *            Writer
-	 * @param batchSize
-	 *            Batch size
-	 */
-	public BatchedTriGWriter(Writer writer, long batchSize) {
-		super(writer, batchSize);
-	}
-
-	@Override
-	protected Lang getRdfLanguage() {
-		return Lang.TRIG;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java
deleted file mode 100644
index 86ab2f9..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.turtle;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedTripleWriter;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record writer for Turtle that uses the batched approach, note that this
- * approach will produce invalid data when blank nodes span batches
- * 
- * 
- * 
- * @param <TKey>
- */
-public class BatchedTurtleWriter<TKey> extends
-		AbstractBatchedTripleWriter<TKey> {
-
-	/**
-	 * Creates a new record writer
-	 * 
-	 * @param writer
-	 *            Writer
-	 * @param batchSize
-	 *            Batch size
-	 */
-	public BatchedTurtleWriter(Writer writer, long batchSize) {
-		super(writer, batchSize);
-	}
-
-	@Override
-	protected Lang getRdfLanguage() {
-		return Lang.TURTLE;
-	}
-
-}


[04/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-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/jena-elephas-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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
new file mode 100644
index 0000000..2b8447b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.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.input.compressed.nquads;
+
+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.compressed.AbstractCompressedQuadsInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.nquads.NQuadsInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Abstract compressed NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedNQuadsInputFormatTests extends AbstractCompressedQuadsInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedNQuadsInputFormatTests(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 InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new NQuadsInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
new file mode 100644
index 0000000..326258a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
@@ -0,0 +1,75 @@
+/*
+ * 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.nquads;
+
+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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.nquads.WholeFileNQuadsInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed whole file NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedWholeFileNQuadsInputFormatTests extends
+        AbstractCompressedWholeFileQuadInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedWholeFileNQuadsInputFormatTests(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.NQUADS;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new WholeFileNQuadsInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
new file mode 100644
index 0000000..62dc9ce
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+
+/**
+ * Tests for BZipped NQuads input
+ * 
+ * 
+ * 
+ */
+public class BZipppedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZipppedNQuadsInputTest() {
+        super(".nq.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
new file mode 100644
index 0000000..e3d670c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped NQuads input
+ * 
+ * 
+ * 
+ */
+public class BZipppedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZipppedWholeFileNQuadsInputTest() {
+        super(".nq.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
new file mode 100644
index 0000000..5fec23e
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for deflated NQuads input
+ * 
+ * 
+ * 
+ */
+public class DeflatedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedNQuadsInputTest() {
+        super(".nq.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
new file mode 100644
index 0000000..a31472d
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for deflated NQuads input
+ * 
+ * 
+ * 
+ */
+public class DeflatedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedWholeFileNQuadsInputTest() {
+        super(".nq.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
new file mode 100644
index 0000000..2d40dec
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped NQuads input
+ * 
+ * 
+ * 
+ */
+public class GZippedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedNQuadsInputTest() {
+        super(".nq.gz", new GzipCodec());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
new file mode 100644
index 0000000..0f73eb7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped NQuads input
+ * 
+ * 
+ * 
+ */
+public class GZippedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedWholeFileNQuadsInputTest() {
+        super(".nq.gz", new GzipCodec());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
new file mode 100644
index 0000000..611d862
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
@@ -0,0 +1,53 @@
+/*
+ * 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.ntriples;
+
+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.ntriples.BlockedNTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Abstract compressed blocked NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedBlockedNTriplesInputFormatTests extends
+        AbstractCompressedWholeFileNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedBlockedNTriplesInputFormatTests(String ext, CompressionCodec codec) {
+        super(ext, codec);
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new BlockedNTriplesInputFormat();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
new file mode 100644
index 0000000..4d9aeba
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.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.input.compressed.ntriples;
+
+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.compressed.AbstractCompressedTriplesInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Abstract compressed NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedNTriplesInputFormatTests extends AbstractCompressedTriplesInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedNTriplesInputFormatTests(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 InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new NTriplesInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
new file mode 100644
index 0000000..7d4b510
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
@@ -0,0 +1,75 @@
+/*
+ * 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.ntriples;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.ntriples.WholeFileNTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed whole file NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedWholeFileNTriplesInputFormatTests extends
+        AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedWholeFileNTriplesInputFormatTests(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.NTRIPLES;
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new WholeFileNTriplesInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
new file mode 100644
index 0000000..de45f17
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
@@ -0,0 +1,37 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped blocked NTriples input
+ * 
+ * 
+ * 
+ */
+public class BZippedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedBlockedNTriplesInput() {
+        super(".nt.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
new file mode 100644
index 0000000..fb1ab8b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+
+/**
+ * Tests for BZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class BZippedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedNTriplesInputTest() {
+        super(".nt.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
new file mode 100644
index 0000000..d50b6eb
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+
+/**
+ * Tests for BZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class BZippedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedWholeFileNTriplesInputTest() {
+        super(".nt.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
new file mode 100644
index 0000000..9780707
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
@@ -0,0 +1,37 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for deflated blocked NTriples input
+ * 
+ * 
+ * 
+ */
+public class DeflatedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedBlockedNTriplesInput() {
+        super(".nt.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
new file mode 100644
index 0000000..b6dd1e9
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+
+/**
+ * Tests for deflated NTriples input
+ * 
+ * 
+ * 
+ */
+public class DeflatedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedNTriplesInputTest() {
+        super(".nt.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
new file mode 100644
index 0000000..e1e57f4
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+
+/**
+ * Tests for deflated NTriples input
+ * 
+ * 
+ * 
+ */
+public class DeflatedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedWholeFileNTriplesInputTest() {
+        super(".nt.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
new file mode 100644
index 0000000..012fd58
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
@@ -0,0 +1,37 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped blocked NTriples input
+ * 
+ * 
+ * 
+ */
+public class GZippedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedBlockedNTriplesInput() {
+        super(".nt.gz", new GzipCodec());
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
new file mode 100644
index 0000000..b7acc08
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class GZippedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedWholeFileNTriplesInputTest() {
+        super(".nt.gz", new GzipCodec());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
new file mode 100644
index 0000000..3b2546d
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.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.rdfjson;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed RDF/JSON input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedRdfJsonInputFormatTests extends AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedRdfJsonInputFormatTests(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.RDFJSON;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new RdfJsonInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
new file mode 100644
index 0000000..6e58d4b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfjson;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped RDF/JSON input
+ * 
+ * 
+ * 
+ */
+public class BZippedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedRdfJsonInputTest() {
+        super(".rj.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
new file mode 100644
index 0000000..8b7b044
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfjson;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated RDF/JSON input
+ * 
+ * 
+ * 
+ */
+public class DeflatedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedRdfJsonInputTest() {
+        super(".rj.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
new file mode 100644
index 0000000..66996dd
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfjson;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped RDF/JSON input
+ * 
+ * 
+ * 
+ */
+public class GZippedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedRdfJsonInputTest() {
+        super(".rj.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
new file mode 100644
index 0000000..a6d1e24
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rdfxml;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.rdfxml.RdfXmlInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed RDF/XML input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedRdfXmlInputFormatTests extends
+ AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedRdfXmlInputFormatTests(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.RDFXML;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new RdfXmlInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
new file mode 100644
index 0000000..30a6c39
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfxml;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped RDF/XML input
+ * 
+ * 
+ * 
+ */
+public class BZippedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedRdfXmlInputTest() {
+        super(".rdf.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
new file mode 100644
index 0000000..a3d747c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfxml;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated RDF/XML input
+ * 
+ * 
+ * 
+ */
+public class DeflatedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedRdfXmlInputTest() {
+        super(".rdf.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
new file mode 100644
index 0000000..748785c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfxml;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped RDF/XML input
+ * 
+ * 
+ * 
+ */
+public class GZippedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedRdfXmlInputTest() {
+        super(".rdf.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
new file mode 100644
index 0000000..78affb2
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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.thrift;
+
+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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftQuadInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+/**
+ * Abstract compressed Thrift quad input tests
+ */
+public abstract class AbstractCompressedThriftQuadInputFormatTests extends
+        AbstractCompressedWholeFileQuadInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedThriftQuadInputFormatTests(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 RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new ThriftQuadInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
new file mode 100644
index 0000000..f837b89
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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.thrift;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftTripleInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+/**
+ * Abstract compressed Thrift triple input tests
+ */
+public abstract class AbstractCompressedThriftTripleInputFormatTests extends
+        AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedThriftTripleInputFormatTests(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 RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new ThriftTripleInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
new file mode 100644
index 0000000..320d278
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped Thrift input
+ */
+public class BZippedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedThriftQuadInputTest() {
+        super(".trdf.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
new file mode 100644
index 0000000..bb2d65b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped Thrift input
+ */
+public class BZippedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedThriftTripleInputTest() {
+        super(".trdf.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
new file mode 100644
index 0000000..6872583
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated Thrift input
+ */
+public class DeflatedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedThriftQuadInputTest() {
+        super(".trdf.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
new file mode 100644
index 0000000..e76d2d7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated Thrift input
+ */
+public class DeflatedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedThriftTripleInputTest() {
+        super(".trdf.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
new file mode 100644
index 0000000..6590f22
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped Thrift input
+ */
+public class GZippedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedThriftQuadInputTest() {
+        super(".trdf.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
new file mode 100644
index 0000000..1ce74f4
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped Thrift input
+ */
+public class GZippedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedThriftTripleInputTest() {
+        super(".trdf.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
new file mode 100644
index 0000000..2975f29
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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.trig;
+
+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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.trig.TriGInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed TriG input tests
+ */
+public abstract class AbstractCompressedTriGInputFormatTests extends
+        AbstractCompressedWholeFileQuadInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedTriGInputFormatTests(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.TRIG;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new TriGInputFormat();
+    }
+
+}
\ No newline at end of file


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
new file mode 100644
index 0000000..bfc8503
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.readers.ntriples;
+
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractLineBasedTripleReader;
+import org.apache.jena.riot.lang.LangNTriples;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.tokens.Tokenizer;
+import org.apache.jena.riot.tokens.TokenizerFactory;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A record reader for NTriples
+ * 
+ * 
+ * 
+ */
+public class NTriplesReader extends AbstractLineBasedTripleReader {
+
+    @Override
+    protected Iterator<Triple> getTriplesIterator(Tokenizer tokenizer, ParserProfile profile) {
+        return new LangNTriples(tokenizer, profile, null);
+    }
+
+    @Override
+    protected Tokenizer getTokenizer(String line) {
+        return TokenizerFactory.makeTokenizerString(line);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
new file mode 100644
index 0000000..c200d93
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.readers.ntriples;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record record for NTriples
+ * <p>
+ * Unlike the {@link NTriplesReader} this processes files as a whole rather than
+ * individual lines. This has the advantage of less parser setup overhead but
+ * the disadvantage that the input cannot be split between multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class WholeFileNTriplesReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
new file mode 100644
index 0000000..009024b
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.rdfjson;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for RDF/JSON files
+ * 
+ * 
+ * 
+ */
+public class RdfJsonReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFJSON;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
new file mode 100644
index 0000000..9c374c6
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.rdfxml;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for RDF/XML files
+ * 
+ * 
+ * 
+ */
+public class RdfXmlReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFXML;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
new file mode 100644
index 0000000..084b1ec
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.readers.thrift;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class ThriftQuadReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
new file mode 100644
index 0000000..713bfa7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
@@ -0,0 +1,30 @@
+/*
+ * 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.readers.thrift;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class ThriftTripleReader extends AbstractWholeFileTripleReader {
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
new file mode 100644
index 0000000..b1b0c3c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.trig;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for TriG files
+ * 
+ * 
+ * 
+ */
+public class TriGReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIG;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
new file mode 100644
index 0000000..6873c64
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.trix;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for TriX files
+ * 
+ * 
+ * 
+ */
+public class TriXReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIX;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
new file mode 100644
index 0000000..b3fb377
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
@@ -0,0 +1,37 @@
+/*
+ * 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.readers.turtle;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record reader for Turtle files
+ * 
+ * 
+ * 
+ */
+public class TurtleReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TURTLE;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
new file mode 100644
index 0000000..f75542a
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.thrift;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftQuadReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+public class ThriftQuadInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new ThriftQuadReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
new file mode 100644
index 0000000..b60380d
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.thrift;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftTripleReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+public class ThriftTripleInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new ThriftTripleReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
new file mode 100644
index 0000000..0b36e93
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.trig;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.trig.TriGReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Input format for TriG
+ * 
+ * 
+ * 
+ */
+public class TriGInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TriGReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
new file mode 100644
index 0000000..723c5c3
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.trix;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.trix.TriXReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+/**
+ * Input format for TriX
+ */
+public class TriXInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TriXReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
new file mode 100644
index 0000000..c7771b6
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.turtle;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.turtle.TurtleReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Turtle input format
+ * 
+ * 
+ * 
+ */
+public class TurtleInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TurtleReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
new file mode 100644
index 0000000..a9e692e
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
@@ -0,0 +1,94 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A block input stream which can is a wrapper around another input stream which
+ * restricts reading to a specific number of bytes and can report the number of
+ * bytes read
+ * <p>
+ * The class assumes that the underlying input stream has already been seeked to
+ * the appropriate start point
+ * </p>
+ * 
+ * 
+ * 
+ */
+public final class BlockInputStream extends TrackedInputStream {
+
+    private long limit = Long.MAX_VALUE;
+
+    /**
+     * Creates a new tracked input stream
+     * 
+     * @param input
+     *            Input stream to track
+     * @param limit
+     *            Maximum number of bytes to read from the stream
+     */
+    public BlockInputStream(InputStream input, long limit) {
+        super(input);
+        if (limit < 0)
+            throw new IllegalArgumentException("limit must be >= 0");
+        this.limit = limit;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (this.bytesRead >= this.limit) {
+            return -1;
+        }
+        return super.read();
+    }
+
+    @Override
+    public int available() throws IOException {
+        if (this.bytesRead >= this.limit) {
+            return 0;
+        }
+        return super.available();
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (len == 0) {
+            return 0;
+        } else if (this.bytesRead >= this.limit) {
+            return -1;
+        } else if (len > this.limit - this.bytesRead) {
+            len = (int) (this.limit - this.bytesRead);
+        }
+        return super.read(b, off, len);
+    }
+
+    @Override
+    public long skip(long n) throws IOException {
+        if (n == 0) {
+            return 0;
+        } else if (this.bytesRead >= this.limit) {
+            return -1;
+        } else if (n > this.limit - this.bytesRead) {
+            n = this.limit - this.bytesRead;
+        }
+        return super.skip(n);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
new file mode 100644
index 0000000..372b22c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
@@ -0,0 +1,101 @@
+/*
+ * 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.util;
+
+import java.util.UUID;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.riot.lang.LabelToNode;
+import org.apache.jena.riot.system.ErrorHandlerFactory;
+import org.apache.jena.riot.system.IRIResolver;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.system.ParserProfileBase;
+import org.apache.jena.riot.system.Prologue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * RDF IO utility functions
+ * 
+ * 
+ * 
+ */
+public class RdfIOUtils {
+    private static final Logger LOGGER = LoggerFactory.getLogger(RdfIOUtils.class);
+
+    /**
+     * Private constructor prevents instantiation
+     */
+    private RdfIOUtils() {
+    }
+
+    /**
+     * Creates a parser profile for the given job context
+     * 
+     * @param context
+     *            Context
+     * @param path
+     *            File path
+     * @return Parser profile
+     */
+    public static ParserProfile createParserProfile(JobContext context, Path path) {
+        Prologue prologue = new Prologue(null, IRIResolver.createNoResolve());
+        UUID seed = RdfIOUtils.getSeed(context, path);
+        LabelToNode labelMapping = LabelToNode.createScopeByDocumentHash(seed);
+        return new ParserProfileBase(prologue, ErrorHandlerFactory.errorHandlerStd, labelMapping);
+    }
+
+    /**
+     * Selects a seed for use in generating blank node identifiers
+     * 
+     * @param context
+     *            Job Context
+     * @param path
+     *            File path
+     * @return Seed
+     */
+    public static UUID getSeed(JobContext context, Path path) {
+        // This is to ensure that blank node allocation policy is constant when
+        // subsequent MapReduce jobs need that
+        String jobId = context.getJobID().toString();
+        if (jobId == null) {
+            jobId = String.valueOf(System.currentTimeMillis());
+            LOGGER.warn(
+                    "Job ID was not set, using current milliseconds of {}. Sequence of MapReduce jobs must carefully handle blank nodes.",
+                    jobId);
+        }
+
+        if (!context.getConfiguration().getBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, false)) {
+            // Using normal file scoped blank node allocation
+            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());
+        } else {
+            // Using globally scoped blank node allocation
+            LOGGER.warn(
+                    "Using globally scoped blank node allocation policy from Job Details (ID={}) - this is unsafe if your RDF inputs did not originate from a previous job",
+                    jobId);
+            
+            return new UUID(jobId.hashCode(), 0);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
new file mode 100644
index 0000000..92e2df5
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
@@ -0,0 +1,38 @@
+/*
+ * 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.util;
+
+import java.io.InputStream;
+
+/**
+ * An input stream that tracks the number of bytes read
+ * 
+ * 
+ * 
+ */
+public abstract class TrackableInputStream extends InputStream {
+
+    /**
+     * Gets the number of bytes read
+     * 
+     * @return Number of bytes read
+     */
+    public abstract long getBytesRead();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
new file mode 100644
index 0000000..e51a866
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
@@ -0,0 +1,124 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A tracked input stream which can is a wrapper around another input stream and
+ * can report the number of bytes read
+ * 
+ * 
+ * 
+ */
+public class TrackedInputStream extends TrackableInputStream {
+
+    protected InputStream input;
+    protected long bytesRead = 0, lastMark;
+
+    /**
+     * Creates a new tracked input stream
+     * 
+     * @param input
+     *            Input stream to track
+     */
+    public TrackedInputStream(InputStream input) {
+        if (input == null)
+            throw new NullPointerException("Input cannot be null");
+        this.input = input;
+    }
+
+    @Override
+    public int read() throws IOException {
+        int read = this.input.read();
+        if (read >= 0)
+            this.bytesRead++;
+        return read;
+    }
+
+    @Override
+    public long getBytesRead() {
+        return this.bytesRead;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.input.close();
+    }
+
+    @Override
+    public int available() throws IOException {
+        return this.input.available();
+    }
+
+    @Override
+    public synchronized void mark(int readlimit) {
+        this.input.mark(readlimit);
+        this.lastMark = this.bytesRead;
+    }
+
+    @Override
+    public boolean markSupported() {
+        return this.input.markSupported();
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (len == 0) return 0;
+        int read = this.input.read(b, off, len);
+        if (read > 0)
+            this.bytesRead += read;
+        return read;
+    }
+
+    @Override
+    public int read(byte[] b) throws IOException {
+        return this.read(b, 0, b.length);
+    }
+
+    @Override
+    public synchronized void reset() throws IOException {
+        this.input.reset();
+        this.bytesRead = this.lastMark;
+    }
+
+    @Override
+    public long skip(long n) throws IOException {
+        if (n == 0)
+            return 0;
+        long skipped = 0;
+        byte[] buffer = new byte[16];
+        int readSize = Math.min(buffer.length, n > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) n);
+        int read;
+        do {
+            if (n - skipped > readSize) {
+                read = this.input.read(buffer, 0, readSize);
+            } else {
+                read = this.input.read(buffer, 0, (int) (n - skipped));
+            }
+            if (read > 0) {
+                this.bytesRead += read;
+                skipped += read;
+            }
+        } while (skipped < n && read >= 0);
+
+        return skipped;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
new file mode 100644
index 0000000..845c709
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
@@ -0,0 +1,55 @@
+/*
+ * 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.util;
+
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A tracked piped quads stream
+ * 
+ * 
+ * 
+ */
+public class TrackedPipedQuadsStream extends TrackedPipedRDFStream<Quad> {
+
+    /**
+     * Creates a new stream
+     * 
+     * @param sink
+     *            Sink
+     * @param input
+     *            Input stream
+     */
+    public TrackedPipedQuadsStream(PipedRDFIterator<Quad> sink, TrackableInputStream input) {
+        super(sink, input);
+    }
+
+    @Override
+    public void triple(Triple triple) {
+        // Triples are discarded
+    }
+
+    @Override
+    public void quad(Quad quad) {
+        this.receive(quad);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java
new file mode 100644
index 0000000..6e910be
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.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.input.util;
+
+import java.util.LinkedList;
+import java.util.Queue;
+
+import org.apache.jena.riot.lang.PipedRDFIterator;
+import org.apache.jena.riot.lang.PipedRDFStream;
+
+/**
+ * A tracked piped RDF stream
+ * 
+ * 
+ * 
+ * @param <T>
+ *            Type corresponding to a supported RDF primitive
+ */
+public abstract class TrackedPipedRDFStream<T> extends PipedRDFStream<T> {
+
+    private TrackableInputStream input;
+    private Queue<Long> positions = new LinkedList<Long>();
+
+    protected TrackedPipedRDFStream(PipedRDFIterator<T> sink, TrackableInputStream input) {
+        super(sink);
+        this.input = input;
+    }
+
+    @Override
+    protected void receive(T t) {
+        // Track positions the input stream is at as we receive inputs
+        synchronized (this.positions) {
+            this.positions.add(this.input.getBytesRead());
+        }
+        super.receive(t);
+    }
+
+    /**
+     * Gets the next position
+     * 
+     * @return Position
+     */
+    public Long getPosition() {
+        synchronized (this.positions) {
+            return this.positions.poll();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java
new file mode 100644
index 0000000..2040c4f
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.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.input.util;
+
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A tracked piped triples stream
+ * 
+ * 
+ * 
+ */
+public class TrackedPipedTriplesStream extends TrackedPipedRDFStream<Triple> {
+
+    /**
+     * Creates a tracked triples stream
+     * 
+     * @param sink
+     *            Sink
+     * @param input
+     *            Input stream
+     */
+    public TrackedPipedTriplesStream(PipedRDFIterator<Triple> sink, TrackableInputStream input) {
+        super(sink, input);
+    }
+
+    @Override
+    public void triple(Triple triple) {
+        receive(triple);
+    }
+
+    @Override
+    public void quad(Quad quad) {
+        // Quads are discarded
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
new file mode 100644
index 0000000..02fbf9c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
@@ -0,0 +1,55 @@
+/*
+ * 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.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedNodeTupleWriter;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+
+
+/**
+ * Abstract output format for formats that use a
+ * {@link AbstractBatchedNodeTupleWriter} as their writer
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TTuple>
+ *            Tuple type
+ * @param <TValue>
+ *            Writable tuple type i.e. the value type
+ */
+public abstract class AbstractBatchedNodeTupleOutputFormat<TKey, TTuple, TValue extends AbstractNodeTupleWritable<TTuple>> extends
+        AbstractNodeTupleOutputFormat<TKey, TTuple, TValue> {
+
+    @Override
+    protected RecordWriter<TKey, TValue> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
+        long batchSize = config.getLong(RdfIOConstants.OUTPUT_BATCH_SIZE, RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE);
+        return this.getRecordWriter(writer, batchSize);
+    }
+    
+    protected abstract RecordWriter<TKey, TValue> getRecordWriter(Writer writer, long batchSize);
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
new file mode 100644
index 0000000..cfc98bd
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
@@ -0,0 +1,94 @@
+/*
+ * 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.OutputStreamWriter;
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Abstract output format which takes pairs with Node keys and arbitrary values
+ * and writes them as a simple line based text file
+ * 
+ * 
+ * 
+ * @param <TValue> Value type
+ */
+public abstract class AbstractNodeOutputFormat<TValue> extends FileOutputFormat<NodeWritable, TValue> {
+    
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeOutputFormat.class);
+
+    @Override
+    public RecordWriter<NodeWritable, TValue> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+        Configuration config = context.getConfiguration();
+        boolean isCompressed = getCompressOutput(context);
+        CompressionCodec codec = null;
+        String extension = this.getFileExtension();
+        if (isCompressed) {
+            Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(context, GzipCodec.class);
+            codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, config);
+            extension += codec.getDefaultExtension();
+        }
+        Path file = getDefaultWorkFile(context, extension);
+        LOG.info("Writing output to file " + file);
+        FileSystem fs = file.getFileSystem(config);
+        if (!isCompressed) {
+            FSDataOutputStream fileOut = fs.create(file, false);
+            return this.getRecordWriter(new OutputStreamWriter(fileOut), config);
+        } else {
+            FSDataOutputStream fileOut = fs.create(file, false);
+            return this.getRecordWriter(new OutputStreamWriter(codec.createOutputStream(fileOut)), config);
+        }
+    }
+
+    /**
+     * Gets the file extension to use for output
+     * 
+     * @return File extension including the '.'
+     */
+    protected String getFileExtension() {
+        return ".nodes";
+    }
+
+    /**
+     * Gets the record writer to use
+     * 
+     * @param writer
+     *            Writer to write output to
+     * @param config
+     *            Configuration
+     * @return Record writer
+     */
+    protected abstract RecordWriter<NodeWritable, TValue> getRecordWriter(Writer writer, Configuration config);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
new file mode 100644
index 0000000..c4a34f5
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
@@ -0,0 +1,109 @@
+/*
+ * 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.OutputStreamWriter;
+import java.io.Writer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract implementation of an output format for line based tuple formats
+ * where the key is ignored and only the tuple values will be output
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple value type
+ * @param <T>
+ *            Writable node tuple type
+ * 
+ */
+public abstract class AbstractNodeTupleOutputFormat<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        FileOutputFormat<TKey, T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleOutputFormat.class);
+
+    @Override
+    public RecordWriter<TKey, T> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+        Configuration config = context.getConfiguration();
+        boolean isCompressed = getCompressOutput(context);
+        CompressionCodec codec = null;
+        
+        // Build the output file path
+        String extension = this.getFileExtension();
+        if (isCompressed) {
+            // Add compression extension if applicable
+            Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(context, GzipCodec.class);
+            codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, config);
+            extension += codec.getDefaultExtension();
+        }
+        Path file = getDefaultWorkFile(context, extension);
+        LOG.info("Writing output to file " + file);
+        
+        // Open the file appropriately and create a record writer for it
+        FileSystem fs = file.getFileSystem(config);
+        if (!isCompressed) {
+            FSDataOutputStream fileOut = fs.create(file, false);
+            return this.getRecordWriter(new OutputStreamWriter(fileOut), config, file);
+        } else {
+            FSDataOutputStream fileOut = fs.create(file, false);
+            return this.getRecordWriter(new OutputStreamWriter(codec.createOutputStream(fileOut)), config, file);
+        }
+    }
+
+    /**
+     * Gets the file extension to use for output
+     * 
+     * @return File extension including the '.'
+     */
+    protected abstract String getFileExtension();
+
+    /**
+     * Gets the record writer to use
+     * 
+     * @param writer
+     *            Writer to write output to
+     * @param config
+     *            Configuration
+     * @param outputPath
+     *            Output path being written to
+     * @return Record writer
+     * @throws IOException
+     *             May be thrown if a record writer cannot be obtained for any
+     *             reason
+     */
+    protected abstract RecordWriter<TKey, T> getRecordWriter(Writer writer, Configuration config, Path outputPath)
+            throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java b/jena-elephas/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-elephas/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);
+    }
+
+}


[07/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
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/AbstractWholeFileQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java
new file mode 100644
index 0000000..5fc0024
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.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.writers;
+
+import java.io.Writer;
+
+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.sparql.core.DatasetGraph;
+import com.hp.hpl.jena.sparql.core.DatasetGraphFactory;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract record writer for whole file triple formats
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractWholeFileQuadWriter<TKey> extends AbstractWholeFileNodeTupleWriter<TKey, Quad, QuadWritable> {
+
+    private DatasetGraph g = DatasetGraphFactory.createMem();
+
+    protected AbstractWholeFileQuadWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected final void add(QuadWritable value) {
+        this.g.add(value.get());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected void writeOutput(Writer writer) {
+        RDFDataMgr.write(writer, this.g, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
+    }
+
+    /**
+     * Gets the RDF language to write the output in
+     * 
+     * @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/AbstractWholeFileTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
new file mode 100644
index 0000000..bb26093
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+/**
+ * An abstract record writer for whole file triple formats
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractWholeFileTripleWriter<TKey> extends AbstractWholeFileNodeTupleWriter<TKey, Triple, TripleWritable> {
+
+    private Graph g = GraphFactory.createDefaultGraph();
+
+    protected AbstractWholeFileTripleWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected final void add(TripleWritable value) {
+        this.g.add(value.get());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected final void writeOutput(Writer writer) {
+        RDFDataMgr.write(writer, this.g, this.getRdfLanguage());
+    }
+
+    /**
+     * Gets the RDF language to write the output in
+     * 
+     * @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/QuadsToTriplesWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
new file mode 100644
index 0000000..e932e1f
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
@@ -0,0 +1,59 @@
+/*
+ * 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 org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+/**
+ * A record writer that converts quads into triples by stripping off the graph
+ * field
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadsToTriplesWriter<TKey> extends RecordWriter<TKey, QuadWritable> {
+
+    private RecordWriter<TKey, TripleWritable> writer;
+
+    /**
+     * Creates a new writer
+     * 
+     * @param tripleWriter
+     *            Triple writer to use
+     */
+    public QuadsToTriplesWriter(RecordWriter<TKey, TripleWritable> tripleWriter) {
+        if (tripleWriter == null)
+            throw new NullPointerException("tripleWriter cannot be null");
+        this.writer = tripleWriter;
+    }
+
+    @Override
+    public void write(TKey key, QuadWritable value) throws IOException, InterruptedException {
+        this.writer.write(key, new TripleWritable(value.get().asTriple()));
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+        this.writer.close(context);
+    }
+}

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/StreamRdfQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java
new file mode 100644
index 0000000..e9156d2
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.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.writers;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.system.StreamRDF;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A writer for {@link StreamRDF} based quad writers
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class StreamRdfQuadWriter<TKey> extends
+		AbstractStreamRdfNodeTupleWriter<TKey, Quad, QuadWritable> {
+
+	public StreamRdfQuadWriter(StreamRDF stream, Writer writer) {
+		super(stream, writer);
+	}
+
+	@Override
+	protected void sendOutput(TKey key, QuadWritable value, StreamRDF stream) {
+		stream.quad(value.get());
+	}
+}

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/StreamRdfTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java
new file mode 100644
index 0000000..174edf3
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.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.writers;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.system.StreamRDF;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A writer for {@link StreamRDF} based triple writers
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class StreamRdfTripleWriter<TKey> extends AbstractStreamRdfNodeTupleWriter<TKey, Triple, TripleWritable> {
+
+	public StreamRdfTripleWriter(StreamRDF stream, Writer writer) {
+		super(stream, writer);
+	}
+
+	@Override
+	protected void sendOutput(TKey key, TripleWritable value, StreamRDF stream) {
+		stream.triple(value.get());
+	}
+}

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/jsonld/JsonLDQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
new file mode 100644
index 0000000..1b4b62f
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
@@ -0,0 +1,38 @@
+/*
+ * 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.jsonld;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileQuadWriter;
+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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
new file mode 100644
index 0000000..8d2079d
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
@@ -0,0 +1,38 @@
+/*
+ * 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.jsonld;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
+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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
new file mode 100644
index 0000000..8df606d
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.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.writers.nquads;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractLineBasedQuadWriter;
+import org.apache.jena.riot.out.CharSpace;
+import org.apache.jena.riot.out.NodeFormatterNT;
+
+/**
+ * A record writer for NQuads
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class NQuadsWriter<TKey> extends AbstractLineBasedQuadWriter<TKey> {
+
+    /**
+     * Creates a new writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public NQuadsWriter(Writer writer) {
+        super(writer, new NodeFormatterNT());
+    }
+
+    /**
+     * Creates a new writer using the given character space
+     * 
+     * @param writer
+     *            Writer
+     * @param charSpace
+     *            Character space
+     */
+    public NQuadsWriter(Writer writer, CharSpace charSpace) {
+        super(writer, new NodeFormatterNT(charSpace));
+    }
+}

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/ntriples/NTriplesNodeWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
new file mode 100644
index 0000000..6413f21
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
@@ -0,0 +1,59 @@
+/*
+ * 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.ntriples;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractNodeWriter;
+import org.apache.jena.riot.out.CharSpace;
+import org.apache.jena.riot.out.NodeFormatterNT;
+
+/**
+ * A NTriples based node writer
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Value type
+ */
+public class NTriplesNodeWriter<TValue> extends AbstractNodeWriter<TValue> {
+
+    /**
+     * Creates a new writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public NTriplesNodeWriter(Writer writer) {
+        super(writer);
+    }
+
+    /**
+     * Creates a new writer
+     * 
+     * @param writer
+     *            Writer
+     * @param charSpace
+     *            Character space to use
+     */
+    public NTriplesNodeWriter(Writer writer, CharSpace charSpace) {
+        super(writer, new NodeFormatterNT(charSpace));
+    }
+
+}

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/ntriples/NTriplesWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java
new file mode 100644
index 0000000..2b3c0b4
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.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.writers.ntriples;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractLineBasedTripleWriter;
+import org.apache.jena.riot.out.CharSpace;
+import org.apache.jena.riot.out.NodeFormatterNT;
+
+/**
+ * A record writer for NTriples
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * 
+ */
+public class NTriplesWriter<TKey> extends AbstractLineBasedTripleWriter<TKey> {
+
+    /**
+     * Creates a new writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public NTriplesWriter(Writer writer) {
+        super(writer, new NodeFormatterNT());
+    }
+
+    /**
+     * Creates a new writer using the given character space
+     * 
+     * @param writer
+     *            Writer
+     * @param charSpace
+     *            Character space
+     */
+    public NTriplesWriter(Writer writer, CharSpace charSpace) {
+        super(writer, new NodeFormatterNT(charSpace));
+    }
+}

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/rdfjson/RdfJsonWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java
new file mode 100644
index 0000000..30bd4fa
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.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.writers.rdfjson;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record writer for RDF/JSON
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * 
+ */
+public class RdfJsonWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
+
+    /**
+     * Creates a new record writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public RdfJsonWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFJSON;
+    }
+
+}

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/rdfxml/RdfXmlWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java
new file mode 100644
index 0000000..f202914
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.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.writers.rdfxml;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record writer for RDF/XML
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * 
+ */
+public class RdfXmlWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
+
+    /**
+     * Creates a new record writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public RdfXmlWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFXML;
+    }
+
+}

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/thrift/ThriftQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
new file mode 100644
index 0000000..599dba9
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
@@ -0,0 +1,38 @@
+/*
+ * 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.thrift;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileQuadWriter;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class ThriftQuadWriter<TKey> extends AbstractWholeFileQuadWriter<TKey> {
+
+    public ThriftQuadWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+}

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/thrift/ThriftTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
new file mode 100644
index 0000000..0089459
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
@@ -0,0 +1,38 @@
+/*
+ * 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.thrift;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class ThriftTripleWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
+
+    public ThriftTripleWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+}

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/trig/BatchedTriGWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java
new file mode 100644
index 0000000..4c0c75f
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.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.writers.trig;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedQuadWriter;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record writer for TriG that uses the batched approach, note that this
+ * approach will produce invalid data when blank nodes span batches
+ *  
+ * @param <TKey>
+ *            Key type
+ */
+public class BatchedTriGWriter<TKey> extends AbstractBatchedQuadWriter<TKey> {
+
+	/**
+	 * Creates a new record writer
+	 * 
+	 * @param writer
+	 *            Writer
+	 * @param batchSize
+	 *            Batch size
+	 */
+	public BatchedTriGWriter(Writer writer, long batchSize) {
+		super(writer, batchSize);
+	}
+
+	@Override
+	protected Lang getRdfLanguage() {
+		return Lang.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/writers/turtle/BatchedTurtleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java
new file mode 100644
index 0000000..86ab2f9
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.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.writers.turtle;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedTripleWriter;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record writer for Turtle that uses the batched approach, note that this
+ * approach will produce invalid data when blank nodes span batches
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class BatchedTurtleWriter<TKey> extends
+		AbstractBatchedTripleWriter<TKey> {
+
+	/**
+	 * Creates a new record writer
+	 * 
+	 * @param writer
+	 *            Writer
+	 * @param batchSize
+	 *            Batch size
+	 */
+	public BatchedTurtleWriter(Writer writer, long batchSize) {
+		super(writer, batchSize);
+	}
+
+	@Override
+	protected Lang getRdfLanguage() {
+		return Lang.TURTLE;
+	}
+
+}

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/registry/HadoopRdfIORegistry.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
new file mode 100644
index 0000000..7b04ef4
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
@@ -0,0 +1,310 @@
+/*
+ * 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.registry;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A registry which is used by various classes to dynamically select record
+ * readers and writers based on a provided {@link Lang}
+ * <p>
+ * Readers and writers are dynamically discovered using the Java
+ * {@link ServiceLoader} mechanism. This will look for files under
+ * {@code META-INF/services} named
+ * {@code org.apache.jena.hadoop.rdf.io.registry.ReaderFactory} and
+ * {@code org.apache.jena.hadoop.rdf.io.registry.WriterFactory}. This follows
+ * the standard {@linkplain ServiceLoader} format of provided one class name per
+ * line which implements the relevant interface.
+ * </p>
+ * 
+ */
+public class HadoopRdfIORegistry {
+
+    private static Map<Lang, ReaderFactory> readerFactories = new HashMap<>();
+    private static Map<Lang, WriterFactory> writerFactories = new HashMap<>();
+    private static boolean init = false;
+
+    static {
+        init();
+    }
+
+    private static synchronized void init() {
+        if (init)
+            return;
+
+        // Dynamically load and register reader factories
+        ServiceLoader<ReaderFactory> readerFactoryLoader = ServiceLoader.load(ReaderFactory.class);
+        Iterator<ReaderFactory> readerFactoryIterator = readerFactoryLoader.iterator();
+        while (readerFactoryIterator.hasNext()) {
+            ReaderFactory f = readerFactoryIterator.next();
+            addReaderFactory(f);
+        }
+
+        // Dynamically load and register writer factories
+        ServiceLoader<WriterFactory> writerFactoryLoader = ServiceLoader.load(WriterFactory.class);
+        Iterator<WriterFactory> writerFactoryIterator = writerFactoryLoader.iterator();
+        while (writerFactoryIterator.hasNext()) {
+            WriterFactory f = writerFactoryIterator.next();
+            addWriterFactory(f);
+        }
+
+        init = true;
+    }
+
+    /**
+     * Resets the registry to the default configuration
+     */
+    public static synchronized void reset() {
+        if (!init)
+            return;
+
+        init = false;
+        init();
+    }
+
+    /**
+     * Registers the reader factory for all the languages it declares itself as
+     * supporting
+     * 
+     * @param f
+     *            Reader factory
+     */
+    public static void addReaderFactory(ReaderFactory f) {
+        if (f == null)
+            throw new NullPointerException("Factory cannot be null");
+
+        readerFactories.put(f.getPrimaryLanguage(), f);
+        for (Lang altLang : f.getAlternativeLanguages()) {
+            readerFactories.put(altLang, f);
+        }
+    }
+
+    /**
+     * Registers the writer factory for all the languages it declares itself as
+     * supporting
+     * 
+     * @param f
+     *            Writer factory
+     */
+    public static void addWriterFactory(WriterFactory f) {
+        if (f == null)
+            throw new NullPointerException("Factory cannot be null");
+
+        writerFactories.put(f.getPrimaryLanguage(), f);
+        for (Lang altLang : f.getAlternativeLanguages()) {
+            writerFactories.put(altLang, f);
+        }
+    }
+
+    /**
+     * Gets whether there is a quad reader available for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return True if available, false otherwise
+     */
+    public static boolean hasQuadReader(Lang lang) {
+        if (lang == null)
+            return false;
+
+        ReaderFactory f = readerFactories.get(lang);
+        if (f == null)
+            return false;
+        return f.canReadQuads();
+    }
+
+    /**
+     * Gets whether there is a triple reader available for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return True if available, false otherwise
+     */
+    public static boolean hasTriplesReader(Lang lang) {
+        if (lang == null)
+            return false;
+
+        ReaderFactory f = readerFactories.get(lang);
+        if (f == null)
+            return false;
+        return f.canReadTriples();
+    }
+
+    /**
+     * Tries to create a quad reader for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return Quad reader if one is available
+     * @throws IOException
+     *             Thrown if a quad reader is not available or the given
+     *             language does not support quads
+     */
+    public static RecordReader<LongWritable, QuadWritable> createQuadReader(Lang lang) throws IOException {
+        if (lang == null)
+            throw new IOException("Cannot create a quad reader for an undefined language");
+
+        ReaderFactory f = readerFactories.get(lang);
+        if (f == null)
+            throw new IOException("No factory registered for language " + lang.getName());
+        if (!f.canReadQuads())
+            throw new IOException(lang.getName() + " does not support reading quads");
+
+        RecordReader<LongWritable, QuadWritable> reader = f.createQuadReader();
+        if (reader == null)
+            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples reader");
+        return reader;
+    }
+
+    /**
+     * Tries to create a triple reader for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return Triple reader if one is available
+     * @throws IOException
+     *             Thrown if a triple reader is not available or the given
+     *             language does not support triple
+     */
+    public static RecordReader<LongWritable, TripleWritable> createTripleReader(Lang lang) throws IOException {
+        if (lang == null)
+            throw new IOException("Cannot create a triple reader for an undefined language");
+
+        ReaderFactory f = readerFactories.get(lang);
+        if (f == null)
+            throw new IOException("No factory registered for language " + lang.getName());
+        if (!f.canReadTriples())
+            throw new IOException(lang.getName() + " does not support reading triples");
+
+        RecordReader<LongWritable, TripleWritable> reader = f.createTripleReader();
+        if (reader == null)
+            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples reader");
+        return reader;
+    }
+
+    /**
+     * Gets whether there is a quad writer available for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return True if available, false otherwise
+     */
+    public static boolean hasQuadWriter(Lang lang) {
+        if (lang == null)
+            return false;
+
+        WriterFactory f = writerFactories.get(lang);
+        if (f == null)
+            return false;
+        return f.canWriteQuads();
+    }
+
+    /**
+     * Gets whether there is a triple writer available for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return True if available, false otherwise
+     */
+    public static boolean hasTriplesWriter(Lang lang) {
+        if (lang == null)
+            return false;
+
+        WriterFactory f = writerFactories.get(lang);
+        if (f == null)
+            return false;
+        return f.canWriteTriples();
+    }
+
+    /**
+     * Tries to create a quad writer for the given language
+     * 
+     * @param lang
+     *            Language
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * 
+     * @return Quad writer if one is available
+     * @throws IOException
+     *             Thrown if a quad writer is not available or the given
+     *             language does not support quads
+     */
+    public static <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Lang lang, Writer writer,
+            Configuration config) throws IOException {
+        if (lang == null)
+            throw new IOException("Cannot create a quad writer for an undefined language");
+
+        WriterFactory f = writerFactories.get(lang);
+        if (f == null)
+            throw new IOException("No factory registered for language " + lang.getName());
+        if (!f.canWriteQuads())
+            throw new IOException(lang.getName() + " does not support writeing quads");
+
+        RecordWriter<TKey, QuadWritable> rwriter = f.<TKey> createQuadWriter(writer, config);
+        if (rwriter == null)
+            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples writer");
+        return rwriter;
+    }
+
+    /**
+     * Tries to create a triple writer for the given language
+     * 
+     * @param lang
+     *            Language
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * @return Triple writer if one is available
+     * @throws IOException
+     *             Thrown if a triple writer is not available or the given
+     *             language does not support triple
+     */
+    public static <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Lang lang, Writer writer,
+            Configuration config) throws IOException {
+        if (lang == null)
+            throw new IOException("Cannot create a triple writer for an undefined language");
+
+        WriterFactory f = writerFactories.get(lang);
+        if (f == null)
+            throw new IOException("No factory registered for language " + lang.getName());
+        if (!f.canWriteTriples())
+            throw new IOException(lang.getName() + " does not support writing triples");
+
+        RecordWriter<TKey, TripleWritable> rwriter = f.<TKey> createTripleWriter(writer, config);
+        if (rwriter == null)
+            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples writer");
+        return rwriter;
+    }
+}

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/registry/ReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
new file mode 100644
index 0000000..e1c98c7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.registry;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Interface for reader factories
+ * 
+ */
+public interface ReaderFactory {
+
+    /**
+     * Gets the primary language this factory produces readers for
+     * 
+     * @return Primary language
+     */
+    public abstract Lang getPrimaryLanguage();
+
+    /**
+     * Gets the alternative languages this factory can produce readers for
+     * 
+     * @return Alternative languages
+     */
+    public abstract Collection<Lang> getAlternativeLanguages();
+
+    /**
+     * Gets whether this factory can produce readers that are capable of reading
+     * quads
+     * 
+     * @return True if quads can be read, false if not
+     */
+    public abstract boolean canReadQuads();
+
+    /**
+     * Gets whether this factory can produce readers that are capable of reading
+     * triples
+     * 
+     * @return True if triples can be read, false if not
+     */
+    public abstract boolean canReadTriples();
+
+    /**
+     * Creates a quad reader
+     * 
+     * @return Quad reader
+     * @throws IOException
+     *             May be thrown if a quad reader cannot be created
+     */
+    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
+
+    /**
+     * Creates a triples reader
+     * 
+     * @return Triples reader
+     * @throws IOException
+     *             May be thrown if a triple reader cannot be created
+     */
+    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
+}

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/registry/WriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java
new file mode 100644
index 0000000..db5635f
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.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.registry;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Interface for writer factories
+ * 
+ */
+public interface WriterFactory {
+
+    /**
+     * Gets the primary language this factory produces writers for
+     * 
+     * @return Primary language
+     */
+    public abstract Lang getPrimaryLanguage();
+
+    /**
+     * Gets the alternative languages this factory can produce writers for
+     * 
+     * @return Alternative languages
+     */
+    public abstract Collection<Lang> getAlternativeLanguages();
+
+    /**
+     * Gets whether this factory can produce writers that are capable of reading
+     * quads
+     * 
+     * @return True if quads can be read, false if not
+     */
+    public abstract boolean canWriteQuads();
+
+    /**
+     * Gets whether this factory can produce writers that are capable of reading
+     * triples
+     * 
+     * @return True if triples can be read, false if not
+     */
+    public abstract boolean canWriteTriples();
+
+    /**
+     * Creates a quad writer
+     * 
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * 
+     * @return Quad writer
+     * @throws IOException
+     *             May be thrown if a quad writer cannot be created
+     */
+    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException;
+
+    /**
+     * Creates a triples writer
+     * 
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * 
+     * @return Triples writer
+     * @throws IOException
+     *             May be thrown if a triple writer cannot be created
+     */
+    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException;
+}

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/registry/readers/AbstractQuadsOnlyReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
new file mode 100644
index 0000000..7fe15a9
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.registry.readers;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract reader factory for languages that only support quads
+ */
+public abstract class AbstractQuadsOnlyReaderFactory implements ReaderFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
+
+    public AbstractQuadsOnlyReaderFactory(Lang lang) {
+        this(lang, (Collection<Lang>)null);
+    }
+    
+    public AbstractQuadsOnlyReaderFactory(Lang lang, Lang...altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractQuadsOnlyReaderFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+    
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canReadQuads() {
+        return true;
+    }
+
+    @Override
+    public final boolean canReadTriples() {
+        return false;
+    }
+
+    @Override
+    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
+
+    @Override
+    public final RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        throw new IOException(this.lang.getName() + " does not support reading triples");
+    }
+
+}

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/registry/readers/AbstractReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java
new file mode 100644
index 0000000..60e45af
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.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.registry.readers;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract reader factory for languages that support triples and quads
+ */
+public abstract class AbstractReaderFactory implements ReaderFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
+
+    public AbstractReaderFactory(Lang lang) {
+        this(lang, (Collection<Lang>)null);
+    }
+    
+    public AbstractReaderFactory(Lang lang, Lang...altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractReaderFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+    
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canReadQuads() {
+        return true;
+    }
+
+    @Override
+    public final boolean canReadTriples() {
+        return true;
+    }
+
+    @Override
+    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
+
+    @Override
+    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
+
+}

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/registry/readers/AbstractTriplesOnlyReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
new file mode 100644
index 0000000..7fb8131
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.registry.readers;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract reader factory for languages that only support triples
+ */
+public abstract class AbstractTriplesOnlyReaderFactory implements ReaderFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
+
+    public AbstractTriplesOnlyReaderFactory(Lang lang) {
+        this(lang, (Collection<Lang>)null);
+    }
+    
+    public AbstractTriplesOnlyReaderFactory(Lang lang, Lang...altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractTriplesOnlyReaderFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+    
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+    
+    @Override
+    public final boolean canReadQuads() {
+        return false;
+    }
+
+    @Override
+    public final boolean canReadTriples() {
+        return true;
+    }
+
+    @Override
+    public final RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        throw new IOException(this.lang.getName() + " does not support reading quads");
+    }
+
+    @Override
+    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
+
+}

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/registry/readers/JsonLDReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java
new file mode 100644
index 0000000..6b064a4
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.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.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.JsonLDQuadReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.JsonLDTripleReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class JsonLDReaderFactory extends AbstractReaderFactory {
+    
+    public JsonLDReaderFactory() {
+        super(Lang.JSONLD);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new JsonLDQuadReader();
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new JsonLDTripleReader();
+    }
+
+}

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/registry/readers/NQuadsReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
new file mode 100644
index 0000000..2296296
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.nquads.WholeFileNQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class NQuadsReaderFactory extends AbstractQuadsOnlyReaderFactory {
+    
+    public NQuadsReaderFactory() {
+        super(Lang.NQUADS, Lang.NQ);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new WholeFileNQuadsReader();
+    }
+
+}

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/registry/readers/NTriplesReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
new file mode 100644
index 0000000..a98a1ae
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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.registry.readers;
+
+import java.io.IOException;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.WholeFileNTriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+public class NTriplesReaderFactory extends AbstractTriplesOnlyReaderFactory {
+
+    public NTriplesReaderFactory() {
+        super(Lang.NTRIPLES, Lang.NT);
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new WholeFileNTriplesReader();
+    }
+}

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/registry/readers/RdfJsonReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
new file mode 100644
index 0000000..ccf5feb
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.rdfjson.RdfJsonReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class RdfJsonReaderFactory extends AbstractTriplesOnlyReaderFactory {
+
+    public RdfJsonReaderFactory() {
+        super(Lang.RDFJSON);
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new RdfJsonReader();
+    }
+}

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/registry/readers/RdfXmlReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
new file mode 100644
index 0000000..1aa88d7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.rdfxml.RdfXmlReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+public class RdfXmlReaderFactory extends AbstractTriplesOnlyReaderFactory {
+
+    public RdfXmlReaderFactory() {
+        super(Lang.RDFXML);
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new RdfXmlReader();
+    }
+
+}

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/registry/readers/ThriftReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java
new file mode 100644
index 0000000..25e8234
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.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.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftQuadReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftTripleReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.RDFLanguages;
+
+/**
+ *
+ */
+public class ThriftReaderFactory extends AbstractReaderFactory {
+    
+    public ThriftReaderFactory() {
+        super(RDFLanguages.THRIFT);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new ThriftQuadReader();
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new ThriftTripleReader();
+    }
+
+}

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/registry/readers/TriGReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
new file mode 100644
index 0000000..83ea818
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.trig.TriGReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class TriGReaderFactory extends AbstractQuadsOnlyReaderFactory {
+
+    public TriGReaderFactory() {
+        super(Lang.TRIG);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new TriGReader();
+    }
+
+}

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/registry/readers/TriXReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
new file mode 100644
index 0000000..cb8795c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.trix.TriXReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class TriXReaderFactory extends AbstractQuadsOnlyReaderFactory {
+
+    public TriXReaderFactory() {
+        super(Lang.TRIX);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new TriXReader();
+    }
+}

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/registry/readers/TurtleReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
new file mode 100644
index 0000000..7800376
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.turtle.TurtleReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+public class TurtleReaderFactory extends AbstractTriplesOnlyReaderFactory {
+    
+    public TurtleReaderFactory() {
+        super(Lang.TURTLE, Lang.TTL, Lang.N3);
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new TurtleReader();
+    }
+
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java b/jena-elephas/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-elephas/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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java b/jena-elephas/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-elephas/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);
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java
new file mode 100644
index 0000000..5fc0024
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.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.writers;
+
+import java.io.Writer;
+
+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.sparql.core.DatasetGraph;
+import com.hp.hpl.jena.sparql.core.DatasetGraphFactory;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract record writer for whole file triple formats
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractWholeFileQuadWriter<TKey> extends AbstractWholeFileNodeTupleWriter<TKey, Quad, QuadWritable> {
+
+    private DatasetGraph g = DatasetGraphFactory.createMem();
+
+    protected AbstractWholeFileQuadWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected final void add(QuadWritable value) {
+        this.g.add(value.get());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected void writeOutput(Writer writer) {
+        RDFDataMgr.write(writer, this.g, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
+    }
+
+    /**
+     * Gets the RDF language to write the output in
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
new file mode 100644
index 0000000..bb26093
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+/**
+ * An abstract record writer for whole file triple formats
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractWholeFileTripleWriter<TKey> extends AbstractWholeFileNodeTupleWriter<TKey, Triple, TripleWritable> {
+
+    private Graph g = GraphFactory.createDefaultGraph();
+
+    protected AbstractWholeFileTripleWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected final void add(TripleWritable value) {
+        this.g.add(value.get());
+    }
+
+    @SuppressWarnings("deprecation")
+    @Override
+    protected final void writeOutput(Writer writer) {
+        RDFDataMgr.write(writer, this.g, this.getRdfLanguage());
+    }
+
+    /**
+     * Gets the RDF language to write the output in
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
new file mode 100644
index 0000000..e932e1f
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
@@ -0,0 +1,59 @@
+/*
+ * 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 org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+/**
+ * A record writer that converts quads into triples by stripping off the graph
+ * field
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadsToTriplesWriter<TKey> extends RecordWriter<TKey, QuadWritable> {
+
+    private RecordWriter<TKey, TripleWritable> writer;
+
+    /**
+     * Creates a new writer
+     * 
+     * @param tripleWriter
+     *            Triple writer to use
+     */
+    public QuadsToTriplesWriter(RecordWriter<TKey, TripleWritable> tripleWriter) {
+        if (tripleWriter == null)
+            throw new NullPointerException("tripleWriter cannot be null");
+        this.writer = tripleWriter;
+    }
+
+    @Override
+    public void write(TKey key, QuadWritable value) throws IOException, InterruptedException {
+        this.writer.write(key, new TripleWritable(value.get().asTriple()));
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+        this.writer.close(context);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java
new file mode 100644
index 0000000..e9156d2
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.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.writers;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.system.StreamRDF;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A writer for {@link StreamRDF} based quad writers
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class StreamRdfQuadWriter<TKey> extends
+		AbstractStreamRdfNodeTupleWriter<TKey, Quad, QuadWritable> {
+
+	public StreamRdfQuadWriter(StreamRDF stream, Writer writer) {
+		super(stream, writer);
+	}
+
+	@Override
+	protected void sendOutput(TKey key, QuadWritable value, StreamRDF stream) {
+		stream.quad(value.get());
+	}
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java
new file mode 100644
index 0000000..174edf3
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.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.writers;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.system.StreamRDF;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A writer for {@link StreamRDF} based triple writers
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class StreamRdfTripleWriter<TKey> extends AbstractStreamRdfNodeTupleWriter<TKey, Triple, TripleWritable> {
+
+	public StreamRdfTripleWriter(StreamRDF stream, Writer writer) {
+		super(stream, writer);
+	}
+
+	@Override
+	protected void sendOutput(TKey key, TripleWritable value, StreamRDF stream) {
+		stream.triple(value.get());
+	}
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
new file mode 100644
index 0000000..1b4b62f
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
@@ -0,0 +1,38 @@
+/*
+ * 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.jsonld;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileQuadWriter;
+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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
new file mode 100644
index 0000000..8d2079d
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
@@ -0,0 +1,38 @@
+/*
+ * 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.jsonld;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
+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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
new file mode 100644
index 0000000..8df606d
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.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.writers.nquads;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractLineBasedQuadWriter;
+import org.apache.jena.riot.out.CharSpace;
+import org.apache.jena.riot.out.NodeFormatterNT;
+
+/**
+ * A record writer for NQuads
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class NQuadsWriter<TKey> extends AbstractLineBasedQuadWriter<TKey> {
+
+    /**
+     * Creates a new writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public NQuadsWriter(Writer writer) {
+        super(writer, new NodeFormatterNT());
+    }
+
+    /**
+     * Creates a new writer using the given character space
+     * 
+     * @param writer
+     *            Writer
+     * @param charSpace
+     *            Character space
+     */
+    public NQuadsWriter(Writer writer, CharSpace charSpace) {
+        super(writer, new NodeFormatterNT(charSpace));
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
new file mode 100644
index 0000000..6413f21
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
@@ -0,0 +1,59 @@
+/*
+ * 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.ntriples;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractNodeWriter;
+import org.apache.jena.riot.out.CharSpace;
+import org.apache.jena.riot.out.NodeFormatterNT;
+
+/**
+ * A NTriples based node writer
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Value type
+ */
+public class NTriplesNodeWriter<TValue> extends AbstractNodeWriter<TValue> {
+
+    /**
+     * Creates a new writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public NTriplesNodeWriter(Writer writer) {
+        super(writer);
+    }
+
+    /**
+     * Creates a new writer
+     * 
+     * @param writer
+     *            Writer
+     * @param charSpace
+     *            Character space to use
+     */
+    public NTriplesNodeWriter(Writer writer, CharSpace charSpace) {
+        super(writer, new NodeFormatterNT(charSpace));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java
new file mode 100644
index 0000000..2b3c0b4
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.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.writers.ntriples;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractLineBasedTripleWriter;
+import org.apache.jena.riot.out.CharSpace;
+import org.apache.jena.riot.out.NodeFormatterNT;
+
+/**
+ * A record writer for NTriples
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * 
+ */
+public class NTriplesWriter<TKey> extends AbstractLineBasedTripleWriter<TKey> {
+
+    /**
+     * Creates a new writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public NTriplesWriter(Writer writer) {
+        super(writer, new NodeFormatterNT());
+    }
+
+    /**
+     * Creates a new writer using the given character space
+     * 
+     * @param writer
+     *            Writer
+     * @param charSpace
+     *            Character space
+     */
+    public NTriplesWriter(Writer writer, CharSpace charSpace) {
+        super(writer, new NodeFormatterNT(charSpace));
+    }
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java
new file mode 100644
index 0000000..6d8b08a
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.StreamRdfQuadWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
+
+/**
+ *
+ */
+public class TriGWriterFactory extends AbstractQuadsOnlyWriterFactory {
+    
+    public TriGWriterFactory() {
+        super(Lang.TRIG);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfQuadWriter<TKey>(new WriterStreamRDFBlocks(writer), writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
new file mode 100644
index 0000000..0e1b7b2
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.writers;
+
+import java.io.IOException;
+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.writers.StreamRdfQuadWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.writer.StreamWriterTriX;
+
+/**
+ *
+ */
+public class TriXWriterFactory extends AbstractQuadsOnlyWriterFactory {
+    
+    public TriXWriterFactory() {
+        super(Lang.TRIX);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfQuadWriter<>(new StreamWriterTriX(new WriterOutputStream(writer, Charset.forName("utf-8"))), writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java
new file mode 100644
index 0000000..c837f12
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.StreamRdfTripleWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
+
+/**
+ *
+ */
+public class TurtleWriterFactory extends AbstractTriplesOnlyWriterFactory {
+    
+    public TurtleWriterFactory() {
+        super(Lang.TURTLE, Lang.TTL, Lang.N3);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfTripleWriter<>(new WriterStreamRDFBlocks(writer), writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory b/jena-elephas/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
new file mode 100644
index 0000000..ec0e48a
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
@@ -0,0 +1,10 @@
+# Default Reader Factory implementations
+org.apache.jena.hadoop.rdf.io.registry.readers.JsonLDReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.NQuadsReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.NTriplesReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.RdfJsonReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.RdfXmlReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.ThriftReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.TriGReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.TriXReaderFactory
+org.apache.jena.hadoop.rdf.io.registry.readers.TurtleReaderFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory b/jena-elephas/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
new file mode 100644
index 0000000..164880d
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
@@ -0,0 +1,10 @@
+# Default Writer Factory implementations
+org.apache.jena.hadoop.rdf.io.registry.writers.JsonLDWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.NQuadsWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.NTriplesWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.RdfJsonWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.RdfXmlWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.ThriftWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.TriGWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.TriXWriterFactory
+org.apache.jena.hadoop.rdf.io.registry.writers.TurtleWriterFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
new file mode 100644
index 0000000..5762fb7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.log4j.Logger;
+
+
+/**
+ * A test mapper which takes in line based RDF triple input and just produces triples
+ * 
+ *
+ */
+public class RdfTriplesInputTestMapper extends Mapper<LongWritable, TripleWritable, NullWritable, TripleWritable> {
+    
+    private static final Logger LOG = Logger.getLogger(RdfTriplesInputTestMapper.class);
+
+    @Override
+    protected void map(LongWritable key, TripleWritable value, Context context)
+            throws IOException, InterruptedException {
+        LOG.info("Line " + key.toString() + " => " + value.toString());
+        context.write(NullWritable.get(), value);
+    }
+
+    
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
new file mode 100644
index 0000000..1cda0bd
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * Abstract tests for blocked triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractBlockedQuadInputFormatTests extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected boolean canSplitInputs() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
new file mode 100644
index 0000000..2e1e865
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * Abstract tests for blocked triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractBlockedTripleInputFormatTests extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected boolean canSplitInputs() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
new file mode 100644
index 0000000..e22650f
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
@@ -0,0 +1,612 @@
+/*
+ * 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 java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract node tuple input format tests
+ * 
+ * 
+ * 
+ * @param <TValue>
+ * @param <T>
+ */
+public abstract class AbstractNodeTupleInputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleInputFormatTests.class);
+
+    protected static final int EMPTY_SIZE = 0, SMALL_SIZE = 100, LARGE_SIZE = 10000, BAD_SIZE = 100, MIXED_SIZE = 100;
+    protected static final String EMPTY = "empty";
+    protected static final String SMALL = "small";
+    protected static final String LARGE = "large";
+    protected static final String BAD = "bad";
+    protected static final String MIXED = "mixed";
+
+    /**
+     * Temporary folder for the tests
+     */
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+
+    protected File empty, small, large, bad, mixed;
+
+    /**
+     * Prepares the inputs for the tests
+     * 
+     * @throws IOException
+     */
+    @Before
+    public void beforeTest() throws IOException {
+        this.prepareInputs();
+    }
+
+    /**
+     * Cleans up the inputs after each test
+     */
+    @After
+    public void afterTest() {
+        // Should be unnecessary since JUnit will clean up the temporary folder
+        // anyway but best to do this regardless
+        if (empty != null)
+            empty.delete();
+        if (small != null)
+            small.delete();
+        if (large != null)
+            large.delete();
+        if (bad != null)
+            bad.delete();
+        if (mixed != null)
+            mixed.delete();
+    }
+
+    /**
+     * Prepares a fresh configuration
+     * 
+     * @return Configuration
+     */
+    protected Configuration prepareConfiguration() {
+        Configuration config = new Configuration(true);
+        // Nothing else to do
+        return config;
+    }
+
+    /**
+     * Prepares the inputs
+     * 
+     * @throws IOException
+     */
+    protected void prepareInputs() throws IOException {
+        String ext = this.getFileExtension();
+        empty = folder.newFile(EMPTY + ext);
+        this.generateTuples(empty, EMPTY_SIZE);
+        small = folder.newFile(SMALL + ext);
+        this.generateTuples(small, SMALL_SIZE);
+        large = folder.newFile(LARGE + ext);
+        this.generateTuples(large, LARGE_SIZE);
+        bad = folder.newFile(BAD + ext);
+        this.generateBadTuples(bad, BAD_SIZE);
+        mixed = folder.newFile(MIXED + ext);
+        this.generateMixedTuples(mixed, MIXED_SIZE);
+    }
+
+    /**
+     * Gets the extra file extension to add to the filenames
+     * 
+     * @return File extension
+     */
+    protected abstract String getFileExtension();
+
+    /**
+     * Generates tuples used for tests
+     * 
+     * @param f
+     *            File
+     * @param num
+     *            Number of tuples to generate
+     * @throws IOException
+     */
+    protected final void generateTuples(File f, int num) throws IOException {
+        this.generateTuples(this.getOutputStream(f), num);
+    }
+
+    /**
+     * Gets the output stream to use for generating tuples
+     * 
+     * @param f
+     *            File
+     * @return Output Stream
+     * @throws IOException
+     */
+    protected OutputStream getOutputStream(File f) throws IOException {
+        return new FileOutputStream(f, false);
+    }
+
+    /**
+     * Generates tuples used for tests
+     * 
+     * @param output
+     *            Output Stream to write to
+     * @param num
+     *            Number of tuples to generate
+     * @throws IOException
+     */
+    protected abstract void generateTuples(OutputStream output, int num) throws IOException;
+
+    /**
+     * Generates bad tuples used for tests
+     * 
+     * @param f
+     *            File
+     * @param num
+     *            Number of bad tuples to generate
+     * @throws IOException
+     */
+    protected final void generateBadTuples(File f, int num) throws IOException {
+        this.generateBadTuples(this.getOutputStream(f), num);
+    }
+
+    /**
+     * Generates bad tuples used for tests
+     * 
+     * @param output
+     *            Output Stream to write to
+     * @param num
+     *            Number of bad tuples to generate
+     * @throws IOException
+     */
+    protected abstract void generateBadTuples(OutputStream output, int num) throws IOException;
+
+    /**
+     * Generates a mixture of good and bad tuples used for tests
+     * 
+     * @param f
+     *            File
+     * @param num
+     *            Number of tuples to generate, they should be a 50/50 mix of
+     *            good and bad tuples
+     * @throws IOException
+     */
+    protected final void generateMixedTuples(File f, int num) throws IOException {
+        this.generateMixedTuples(this.getOutputStream(f), num);
+    }
+
+    /**
+     * Generates a mixture of good and bad tuples used for tests
+     * 
+     * @param output
+     *            Output Stream to write to
+     * @param num
+     *            Number of tuples to generate, they should be a 50/50 mix of
+     *            good and bad tuples
+     * @throws IOException
+     */
+    protected abstract void generateMixedTuples(OutputStream output, int num) throws IOException;
+
+    /**
+     * Adds an input path to the job configuration
+     * 
+     * @param f
+     *            File
+     * @param config
+     *            Configuration
+     * @param job
+     *            Job
+     * @throws IOException
+     */
+    protected void addInputPath(File f, Configuration config, Job job) throws IOException {
+        FileSystem fs = FileSystem.getLocal(config);
+        Path inputPath = fs.makeQualified(new Path(f.getAbsolutePath()));
+        FileInputFormat.addInputPath(job, inputPath);
+    }
+
+    protected final int countTuples(RecordReader<LongWritable, T> reader) throws IOException, InterruptedException {
+        int count = 0;
+
+        // Check initial progress
+        LOG.info(String.format("Initial Reported Progress %f", reader.getProgress()));
+        float progress = reader.getProgress();
+        if (Float.compare(0.0f, progress) == 0) {
+            Assert.assertEquals(0.0d, reader.getProgress(), 0.0d);
+        } else if (Float.compare(1.0f, progress) == 0) {
+            // If reader is reported 1.0 straight away then we expect there to
+            // be no key values
+            Assert.assertEquals(1.0d, reader.getProgress(), 0.0d);
+            Assert.assertFalse(reader.nextKeyValue());
+        } else {
+            Assert.fail(String.format(
+                    "Expected progress of 0.0 or 1.0 before reader has been accessed for first time but got %f",
+                    progress));
+        }
+
+        // Count tuples
+        boolean debug = LOG.isDebugEnabled();
+        while (reader.nextKeyValue()) {
+            count++;
+            progress = reader.getProgress();
+            if (debug)
+                LOG.debug(String.format("Current Reported Progress %f", progress));
+            Assert.assertTrue(String.format("Progress should be in the range 0.0 < p <= 1.0 but got %f", progress),
+                    progress > 0.0f && progress <= 1.0f);
+        }
+        reader.close();
+        LOG.info(String.format("Got %d tuples from this record reader", count));
+
+        // Check final progress
+        LOG.info(String.format("Final Reported Progress %f", reader.getProgress()));
+        Assert.assertEquals(1.0d, reader.getProgress(), 0.0d);
+
+        return count;
+    }
+
+    protected final void checkTuples(RecordReader<LongWritable, T> reader, int expected) throws IOException,
+            InterruptedException {
+        Assert.assertEquals(expected, this.countTuples(reader));
+    }
+
+    /**
+     * Runs a test with a single input
+     * 
+     * @param input
+     *            Input
+     * @param expectedTuples
+     *            Expected tuples
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected final void testSingleInput(File input, int expectedSplits, int expectedTuples) throws IOException,
+            InterruptedException {
+        // Prepare configuration
+        Configuration config = this.prepareConfiguration();
+        this.testSingleInput(config, input, expectedSplits, expectedTuples);
+    }
+
+    /**
+     * Runs a test with a single input
+     * 
+     * @param config
+     *            Configuration
+     * @param input
+     *            Input
+     * @param expectedTuples
+     *            Expected tuples
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected final void testSingleInput(Configuration config, File input, int expectedSplits, int expectedTuples)
+            throws IOException, InterruptedException {
+        // Set up fake job
+        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
+        Job job = Job.getInstance(config);
+        job.setInputFormatClass(inputFormat.getClass());
+        this.addInputPath(input, job.getConfiguration(), job);
+        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+        Assert.assertEquals(1, FileInputFormat.getInputPaths(context).length);
+        NLineInputFormat.setNumLinesPerSplit(job, LARGE_SIZE);
+
+        // Check splits
+        List<InputSplit> splits = inputFormat.getSplits(context);
+        Assert.assertEquals(expectedSplits, splits.size());
+
+        // Check tuples
+        for (InputSplit split : splits) {
+            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
+            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
+            reader.initialize(split, taskContext);
+            this.checkTuples(reader, expectedTuples);
+        }
+    }
+
+    protected abstract InputFormat<LongWritable, T> getInputFormat();
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_01() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(empty, this.canSplitInputs() ? 0 : 1, EMPTY_SIZE);
+    }
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_02() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(small, 1, SMALL_SIZE);
+    }
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_03() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(large, 1, LARGE_SIZE);
+    }
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_04() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(bad, 1, 0);
+    }
+
+    /**
+     * Basic tuples input test
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void single_input_05() throws IOException, InterruptedException, ClassNotFoundException {
+        testSingleInput(mixed, 1, MIXED_SIZE / 2);
+    }
+
+    /**
+     * Tests behaviour when ignoring bad tuples is disabled
+     * 
+     * @throws InterruptedException
+     * @throws IOException
+     */
+    @Test(expected = IOException.class)
+    public final void fail_on_bad_input_01() throws IOException, InterruptedException {
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        Assert.assertFalse(config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true));
+        testSingleInput(config, bad, 1, 0);
+    }
+
+    /**
+     * Tests behaviour when ignoring bad tuples is disabled
+     * 
+     * @throws InterruptedException
+     * @throws IOException
+     */
+    @Test(expected = IOException.class)
+    public final void fail_on_bad_input_02() throws IOException, InterruptedException {
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        Assert.assertFalse(config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true));
+        testSingleInput(config, mixed, 1, MIXED_SIZE / 2);
+    }
+
+    /**
+     * Runs a multiple input test
+     * 
+     * @param inputs
+     *            Inputs
+     * @param expectedSplits
+     *            Number of splits expected
+     * @param expectedTuples
+     *            Number of tuples expected
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected final void testMultipleInputs(File[] inputs, int expectedSplits, int expectedTuples) throws IOException,
+            InterruptedException {
+        // Prepare configuration and inputs
+        Configuration config = this.prepareConfiguration();
+
+        // Set up fake job
+        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
+        Job job = Job.getInstance(config);
+        job.setInputFormatClass(inputFormat.getClass());
+        for (File input : inputs) {
+            this.addInputPath(input, job.getConfiguration(), job);
+        }
+        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+        Assert.assertEquals(inputs.length, FileInputFormat.getInputPaths(context).length);
+        NLineInputFormat.setNumLinesPerSplit(job, expectedTuples);
+
+        // Check splits
+        List<InputSplit> splits = inputFormat.getSplits(context);
+        Assert.assertEquals(expectedSplits, splits.size());
+
+        // Check tuples
+        int count = 0;
+        for (InputSplit split : splits) {
+            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
+            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
+            reader.initialize(split, taskContext);
+            count += this.countTuples(reader);
+        }
+        Assert.assertEquals(expectedTuples, count);
+    }
+
+    /**
+     * tuples test with multiple inputs
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void multiple_inputs_01() throws IOException, InterruptedException, ClassNotFoundException {
+        testMultipleInputs(new File[] { empty, small, large }, this.canSplitInputs() ? 2 : 3, EMPTY_SIZE + SMALL_SIZE
+                + LARGE_SIZE);
+    }
+
+    /**
+     * tuples test with multiple inputs
+     * 
+     * @throws IOException
+     * @throws ClassNotFoundException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void multiple_inputs_02() throws IOException, InterruptedException, ClassNotFoundException {
+        testMultipleInputs(new File[] { folder.getRoot() }, this.canSplitInputs() ? 4 : 5, EMPTY_SIZE + SMALL_SIZE
+                + LARGE_SIZE + (MIXED_SIZE / 2));
+    }
+
+    protected final void testSplitInputs(Configuration config, File[] inputs, int expectedSplits, int expectedTuples)
+            throws IOException, InterruptedException {
+        // Set up fake job
+        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
+        Job job = Job.getInstance(config);
+        job.setInputFormatClass(inputFormat.getClass());
+        for (File input : inputs) {
+            this.addInputPath(input, job.getConfiguration(), job);
+        }
+        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+        Assert.assertEquals(inputs.length, FileInputFormat.getInputPaths(context).length);
+
+        // Check splits
+        List<InputSplit> splits = inputFormat.getSplits(context);
+        Assert.assertEquals(expectedSplits, splits.size());
+
+        // Check tuples
+        int count = 0;
+        for (InputSplit split : splits) {
+            // Validate split
+            Assert.assertTrue(this.isValidSplit(split, config));
+
+            // Read split
+            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
+            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
+            reader.initialize(split, taskContext);
+            count += this.countTuples(reader);
+        }
+        Assert.assertEquals(expectedTuples, count);
+    }
+
+    /**
+     * Determines whether an input split is valid
+     * 
+     * @param split
+     *            Input split
+     * @return True if a valid split, false otherwise
+     * @throws IOException
+     */
+    protected boolean isValidSplit(InputSplit split, Configuration config) throws IOException {
+        return split instanceof FileSplit;
+    }
+
+    /**
+     * Indicates whether inputs can be split, defaults to true
+     * 
+     * @return Whether inputs can be split
+     */
+    protected boolean canSplitInputs() {
+        return true;
+    }
+
+    /**
+     * Tests for input splitting
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public final void split_input_01() throws IOException, InterruptedException, ClassNotFoundException {
+        Assume.assumeTrue(this.canSplitInputs());
+
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
+        this.testSplitInputs(config, new File[] { small }, 100, SMALL_SIZE);
+    }
+
+    /**
+     * Tests for input splitting
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public final void split_input_02() throws IOException, InterruptedException, ClassNotFoundException {
+        Assume.assumeTrue(this.canSplitInputs());
+
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        config.setLong(NLineInputFormat.LINES_PER_MAP, 10);
+        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
+        this.testSplitInputs(config, new File[] { small }, 10, SMALL_SIZE);
+    }
+
+    /**
+     * Tests for input splitting
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public final void split_input_03() throws IOException, InterruptedException, ClassNotFoundException {
+        Assume.assumeTrue(this.canSplitInputs());
+
+        Configuration config = this.prepareConfiguration();
+        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
+        config.setLong(NLineInputFormat.LINES_PER_MAP, 100);
+        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
+        this.testSplitInputs(config, new File[] { large }, 100, LARGE_SIZE);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
new file mode 100644
index 0000000..78d7f33
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
@@ -0,0 +1,70 @@
+/*
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for Quad input formats
+ * 
+ *
+ */
+public abstract class AbstractQuadsInputFormatTests extends AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected void generateTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+    
+    @Override
+    protected void generateBadTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write("<http://broken\n".getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
+        boolean bad = false;
+        for (int i = 0; i < num; i++, bad = !bad) {
+            if (bad) {
+                output.write("<http://broken\n".getBytes(utf8));
+            } else {
+                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
+            }
+        }
+        output.flush();
+        output.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
new file mode 100644
index 0000000..65a9889
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for Triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractTriplesInputFormatTests extends AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected void generateTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "<http://broken\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
+        boolean bad = false;
+        for (int i = 0; i < num; i++, bad = !bad) {
+            if (bad) {
+                output.write("<http://broken\n".getBytes(utf8));
+            } else {
+                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
+            }
+        }
+        output.flush();
+        output.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
new file mode 100644
index 0000000..0b6cfde
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
@@ -0,0 +1,115 @@
+/*
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+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.query.Dataset;
+import com.hp.hpl.jena.query.DatasetFactory;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Property;
+import com.hp.hpl.jena.rdf.model.Resource;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for Quad input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractWholeFileQuadInputFormatTests extends AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+
+    private void writeTuples(Dataset ds, OutputStream output) {
+        RDFDataMgr.write(output, ds, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
+    }
+
+    /**
+     * Gets the RDF language to write out generate tuples in
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+    private void writeGoodTuples(OutputStream output, int num) throws IOException {
+        Dataset ds = DatasetFactory.createMem();
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num; i++) {
+            if (i % 100 == 0) {
+                ds.addNamedModel("http://example.org/graphs/" + (i / 100), m);
+                m = ModelFactory.createDefaultModel();
+            }
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        if (!m.isEmpty()) {
+            ds.addNamedModel("http://example.org/graphs/extra", m);
+        }
+        this.writeTuples(ds, output);
+    }
+
+    @Override
+    protected final void generateTuples(OutputStream output, int num) throws IOException {
+        this.writeGoodTuples(output, num);
+        output.close();
+    }
+
+    @Override
+    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
+        // Write good data
+        this.writeGoodTuples(output, num / 2);
+
+        // Write junk data
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num / 2; i++) {
+            output.write(junk);
+        }
+
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
new file mode 100644
index 0000000..b68d662
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
@@ -0,0 +1,108 @@
+/*
+ * 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 java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+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.Triple;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Property;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * Abstract tests for Triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractWholeFileTripleInputFormatTests extends AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+    
+    private void writeTuples(Model m, OutputStream output) {
+        RDFDataMgr.write(output, m, this.getRdfLanguage());
+    }
+        
+    /**
+     * Gets the RDF language to write out generate tuples in
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+    
+    @Override
+    protected final void generateTuples(OutputStream output, int num) throws IOException {
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num; i++) {
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        this.writeTuples(m, output);
+        output.close();
+    }
+    
+    @Override
+    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
+        // Write good data
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num / 2; i++) {
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        this.writeTuples(m, output);
+        
+        // Write junk data
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num / 2; i++) {
+            output.write(junk);
+        }
+        
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
new file mode 100644
index 0000000..4bb0939
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
@@ -0,0 +1,636 @@
+/*
+ * 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.bnodes;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.FileAttribute;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.log4j.BasicConfigurator;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * Test case that embodies the scenario described in JENA-820
+ */
+@SuppressWarnings("unused")
+public abstract class AbstractBlankNodeTests<T, TValue extends AbstractNodeTupleWritable<T>> {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractBlankNodeTests.class);
+
+    @BeforeClass
+    public static void setup() {
+        // Enable if you need to diagnose test failures
+        // Useful since it includes printing the file names of the temporary
+        // files being used
+        // BasicConfigurator.resetConfiguration();
+        // BasicConfigurator.configure();
+    }
+
+    /**
+     * Gets the extension for the initial input files
+     * 
+     * @return Extension including the {@code .}
+     */
+    protected abstract String getInitialInputExtension();
+
+    /**
+     * Creates a tuple
+     * 
+     * @param s
+     *            Subject
+     * @param p
+     *            Predicate
+     * @param o
+     *            Object
+     * @return Tuple
+     */
+    protected abstract T createTuple(Node s, Node p, Node o);
+
+    /**
+     * Writes out the given tuples to the given file
+     * 
+     * @param f
+     *            File
+     * @param tuples
+     *            Tuples
+     * @throws FileNotFoundException
+     */
+    protected abstract void writeTuples(File f, List<T> tuples) throws FileNotFoundException;
+
+    /**
+     * Creates the input format for reading the initial inputs
+     * 
+     * @return Input format
+     */
+    protected abstract InputFormat<LongWritable, TValue> createInitialInputFormat();
+
+    /**
+     * Creates the output format for writing the intermediate output
+     * 
+     * @return Output format
+     */
+    protected abstract OutputFormat<LongWritable, TValue> createIntermediateOutputFormat();
+
+    /**
+     * Creates the input format for reading the intermediate outputs back in
+     * 
+     * @return Input format
+     */
+    protected abstract InputFormat<LongWritable, TValue> createIntermediateInputFormat();
+
+    /**
+     * Gets the subject of the tuple
+     * 
+     * @param value
+     *            Tuple
+     * @return Subject
+     */
+    protected abstract Node getSubject(T value);
+
+    /**
+     * Gets whether the format being tested respects the RIOT
+     * {@link ParserProfile}
+     * 
+     * @return True if parser profile is respected, false otherwise
+     */
+    protected boolean respectsParserProfile() {
+        return true;
+    }
+
+    /**
+     * Gets whether the format being tested preserves blank node identity
+     * 
+     * @return True if identity is presereved, false otherwise
+     */
+    protected boolean preservesBlankNodeIdentity() {
+        return false;
+    }
+
+    /**
+     * Test that starts with two blank nodes with the same identity in a single
+     * file, splits them over two files and checks that we can workaround
+     * JENA-820 successfully by setting the
+     * {@link RdfIOConstants#GLOBAL_BNODE_IDENTITY} flag for our subsequent job
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void blank_node_divergence_01() throws IOException, InterruptedException {
+        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
+        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
+        
+        // Temporary files
+        File a = File.createTempFile("bnode_divergence", getInitialInputExtension());
+        File intermediateOutputDir = Files.createTempDirectory("bnode_divergence", new FileAttribute[0]).toFile();
+
+        try {
+            // Prepare the input data
+            // Two mentions of the same blank node in the same file
+            List<T> tuples = new ArrayList<>();
+            Node bnode = NodeFactory.createAnon();
+            Node pred = NodeFactory.createURI("http://example.org/predicate");
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
+            writeTuples(a, tuples);
+
+            // Set up fake job which will process the file as a single split
+            Configuration config = new Configuration(true);
+            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
+            Job job = Job.getInstance(config);
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()));
+            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            List<InputSplit> splits = inputFormat.getSplits(context);
+            Assert.assertEquals(1, splits.size());
+
+            for (InputSplit split : splits) {
+                // Initialize the input reading
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        createAttemptID(1, 1, 1));
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                // Copy the input to the output - each triple goes to a separate
+                // output file
+                // This is how we force multiple files to be produced
+                int taskID = 1;
+                while (reader.nextKeyValue()) {
+                    // Prepare the output writing
+                    OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
+                    TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                            createAttemptID(1, ++taskID, 1));
+                    RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
+
+                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
+                    writer.close(outputTaskContext);
+                }
+            }
+
+            // Promote outputs from temporary status
+            promoteInputs(intermediateOutputDir);
+
+            // Now we need to create a subsequent job that reads the
+            // intermediate outputs
+            // As described in JENA-820 at this point the blank nodes are
+            // consistent, however when we read them from different files they
+            // by default get treated as different nodes and so the blank nodes
+            // diverge which is incorrect and undesirable behaviour in
+            // multi-stage pipelines
+            System.out.println(intermediateOutputDir.getAbsolutePath());
+            job = Job.getInstance(config);
+            inputFormat = createIntermediateInputFormat();
+            job.setInputFormatClass(inputFormat.getClass());
+            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
+
+            // Enabling this flag works around the JENA-820 issue
+            job.getConfiguration().setBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, true);
+            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            splits = inputFormat.getSplits(context);
+            Assert.assertEquals(2, splits.size());
+
+            // Expect to end up with a single blank node
+            Set<Node> nodes = new HashSet<Node>();
+            for (InputSplit split : splits) {
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        new TaskAttemptID());
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                while (reader.nextKeyValue()) {
+                    nodes.add(getSubject(reader.getCurrentValue().get()));
+                }
+            }
+            // Nodes should not have diverged
+            Assert.assertEquals(1, nodes.size());
+
+        } finally {
+            a.delete();
+            deleteDirectory(intermediateOutputDir);
+        }
+    }
+
+    /**
+     * Test that starts with two blank nodes with the same identity in a single
+     * file, splits them over two files and shows that they diverge in the
+     * subsequent job when the JENA-820 workaround is not enabled
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void blank_node_divergence_02() throws IOException, InterruptedException {
+        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
+        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
+        
+        // Temporary files
+        File a = File.createTempFile("bnode_divergence", getInitialInputExtension());
+        File intermediateOutputDir = Files.createTempDirectory("bnode_divergence", new FileAttribute[0]).toFile();
+
+        try {
+            // Prepare the input data
+            // Two mentions of the same blank node in the same file
+            List<T> tuples = new ArrayList<>();
+            Node bnode = NodeFactory.createAnon();
+            Node pred = NodeFactory.createURI("http://example.org/predicate");
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
+            writeTuples(a, tuples);
+
+            // Set up fake job which will process the file as a single split
+            Configuration config = new Configuration(true);
+            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
+            Job job = Job.getInstance(config);
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()));
+            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            List<InputSplit> splits = inputFormat.getSplits(context);
+            Assert.assertEquals(1, splits.size());
+
+            for (InputSplit split : splits) {
+                // Initialize the input reading
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        createAttemptID(1, 1, 1));
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                // Copy the input to the output - each triple goes to a separate
+                // output file
+                // This is how we force multiple files to be produced
+                int taskID = 1;
+                while (reader.nextKeyValue()) {
+                    // Prepare the output writing
+                    OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
+                    TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                            createAttemptID(1, ++taskID, 1));
+                    RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
+
+                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
+                    writer.close(outputTaskContext);
+                }
+            }
+
+            // Promote outputs from temporary status
+            promoteInputs(intermediateOutputDir);
+
+            // Now we need to create a subsequent job that reads the
+            // intermediate outputs
+            // As described in JENA-820 at this point the blank nodes are
+            // consistent, however when we read them from different files they
+            // by default get treated as different nodes and so the blank nodes
+            // diverge which is incorrect and undesirable behaviour in
+            // multi-stage pipelines. However it is the default behaviour
+            // because when we start from external inputs we want them to be
+            // file scoped.
+            System.out.println(intermediateOutputDir.getAbsolutePath());
+            job = Job.getInstance(config);
+            inputFormat = createIntermediateInputFormat();
+            job.setInputFormatClass(inputFormat.getClass());
+            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
+
+            // Make sure JENA-820 flag is disabled
+            job.getConfiguration().setBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, false);
+            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            splits = inputFormat.getSplits(context);
+            Assert.assertEquals(2, splits.size());
+
+            // Expect to end up with a single blank node
+            Set<Node> nodes = new HashSet<Node>();
+            for (InputSplit split : splits) {
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        new TaskAttemptID());
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                while (reader.nextKeyValue()) {
+                    nodes.add(getSubject(reader.getCurrentValue().get()));
+                }
+            }
+            // Nodes should have diverged
+            Assert.assertEquals(2, nodes.size());
+
+        } finally {
+            a.delete();
+            deleteDirectory(intermediateOutputDir);
+        }
+    }
+
+    /**
+     * Test that starts with two blank nodes in two different files and checks
+     * that writing them to a single file does not conflate them
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void blank_node_identity_01() throws IOException, InterruptedException {
+        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
+        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
+        
+        // Temporary files
+        File a = File.createTempFile("bnode_identity", getInitialInputExtension());
+        File b = File.createTempFile("bnode_identity", getInitialInputExtension());
+        File intermediateOutputDir = Files.createTempDirectory("bnode_identity", new FileAttribute[0]).toFile();
+
+        try {
+            // Prepare the input data
+            // Different blank nodes in different files
+            List<T> tuples = new ArrayList<>();
+            Node bnode1 = NodeFactory.createAnon();
+            Node bnode2 = NodeFactory.createAnon();
+            Node pred = NodeFactory.createURI("http://example.org/predicate");
+
+            tuples.add(createTuple(bnode1, pred, NodeFactory.createLiteral("first")));
+            writeTuples(a, tuples);
+
+            tuples.clear();
+            tuples.add(createTuple(bnode2, pred, NodeFactory.createLiteral("second")));
+            writeTuples(b, tuples);
+
+            // Set up fake job which will process the two files
+            Configuration config = new Configuration(true);
+            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
+            Job job = Job.getInstance(config);
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()), new Path(b.getAbsolutePath()));
+            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            List<InputSplit> splits = inputFormat.getSplits(context);
+            Assert.assertEquals(2, splits.size());
+
+            // Prepare the output writing - putting all output to a single file
+            OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
+            TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(), createAttemptID(
+                    1, 2, 1));
+            RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
+
+            for (InputSplit split : splits) {
+                // Initialize the input reading
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        createAttemptID(1, 1, 1));
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                // Copy the input to the output - all triples go to a single
+                // output
+                while (reader.nextKeyValue()) {
+                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
+                }
+            }
+            writer.close(outputTaskContext);
+
+            // Promote outputs from temporary status
+            promoteInputs(intermediateOutputDir);
+
+            // Now we need to create a subsequent job that reads the
+            // intermediate outputs
+            // The Blank nodes should have been given separate identities so we
+            // should not be conflating them, this is the opposite problem to
+            // that described in JENA-820
+            System.out.println(intermediateOutputDir.getAbsolutePath());
+            job = Job.getInstance(config);
+            inputFormat = createIntermediateInputFormat();
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            splits = inputFormat.getSplits(context);
+            Assert.assertEquals(1, splits.size());
+
+            // Expect to end up with a single blank node
+            Set<Node> nodes = new HashSet<Node>();
+            for (InputSplit split : splits) {
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        new TaskAttemptID());
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                while (reader.nextKeyValue()) {
+                    nodes.add(getSubject(reader.getCurrentValue().get()));
+                }
+            }
+            // Nodes must not have converged
+            Assert.assertEquals(2, nodes.size());
+
+        } finally {
+            a.delete();
+            b.delete();
+            deleteDirectory(intermediateOutputDir);
+        }
+    }
+
+    /**
+     * Test that starts with two blank nodes in two different files and checks
+     * that writing them to a single file does not conflate them
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void blank_node_identity_02() throws IOException, InterruptedException {
+        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
+        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
+        
+        // Temporary files
+        File a = File.createTempFile("bnode_identity", getInitialInputExtension());
+        File b = File.createTempFile("bnode_identity", getInitialInputExtension());
+        File intermediateOutputDir = Files.createTempDirectory("bnode_identity", new FileAttribute[0]).toFile();
+
+        try {
+            // Prepare the input data
+            // Same blank node but in different files so must be treated as
+            // different blank nodes and not converge
+            List<T> tuples = new ArrayList<>();
+            Node bnode = NodeFactory.createAnon();
+            Node pred = NodeFactory.createURI("http://example.org/predicate");
+
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
+            writeTuples(a, tuples);
+
+            tuples.clear();
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
+            writeTuples(b, tuples);
+
+            // Set up fake job which will process the two files
+            Configuration config = new Configuration(true);
+            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
+            Job job = Job.getInstance(config);
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()), new Path(b.getAbsolutePath()));
+            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            List<InputSplit> splits = inputFormat.getSplits(context);
+            Assert.assertEquals(2, splits.size());
+
+            // Prepare the output writing - putting all output to a single file
+            OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
+            TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(), createAttemptID(
+                    1, 2, 1));
+            RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
+
+            for (InputSplit split : splits) {
+                // Initialize the input reading
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        createAttemptID(1, 1, 1));
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                // Copy the input to the output - all triples go to a single
+                // output
+                while (reader.nextKeyValue()) {
+                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
+                }
+            }
+            writer.close(outputTaskContext);
+
+            // Promote outputs from temporary status
+            promoteInputs(intermediateOutputDir);
+
+            // Now we need to create a subsequent job that reads the
+            // intermediate outputs
+            // The Blank nodes should have been given separate identities so we
+            // should not be conflating them, this is the opposite problem to
+            // that described in JENA-820
+            System.out.println(intermediateOutputDir.getAbsolutePath());
+            job = Job.getInstance(config);
+            inputFormat = createIntermediateInputFormat();
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            splits = inputFormat.getSplits(context);
+            Assert.assertEquals(1, splits.size());
+
+            // Expect to end up with a single blank node
+            Set<Node> nodes = new HashSet<Node>();
+            for (InputSplit split : splits) {
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        new TaskAttemptID());
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                while (reader.nextKeyValue()) {
+                    nodes.add(getSubject(reader.getCurrentValue().get()));
+                }
+            }
+            // Nodes must not diverge
+            Assert.assertEquals(2, nodes.size());
+
+        } finally {
+            a.delete();
+            b.delete();
+            deleteDirectory(intermediateOutputDir);
+        }
+    }
+
+    private TaskAttemptID createAttemptID(int jobID, int taskID, int id) {
+        return new TaskAttemptID("outputTest", jobID, TaskType.MAP, taskID, 1);
+    }
+
+    private void promoteInputs(File baseDir) throws IOException {
+        for (File f : baseDir.listFiles()) {
+            if (f.isDirectory()) {
+                promoteInputs(baseDir, f);
+            }
+        }
+    }
+
+    private void promoteInputs(File targetDir, File dir) throws IOException {
+        java.nio.file.Path target = Paths.get(targetDir.toURI());
+        for (File f : dir.listFiles()) {
+            if (f.isDirectory()) {
+                promoteInputs(targetDir, f);
+            } else {
+                LOGGER.debug("Moving {} to {}", f.getAbsolutePath(), target.resolve(f.getName()));
+                Files.move(Paths.get(f.toURI()), target.resolve(f.getName()), StandardCopyOption.REPLACE_EXISTING);
+            }
+        }
+
+        // Remove defunct sub-directory
+        dir.delete();
+    }
+
+    private void deleteDirectory(File dir) throws IOException {
+        for (File f : dir.listFiles()) {
+            if (f.isFile())
+                f.delete();
+            if (f.isDirectory())
+                deleteDirectory(f);
+        }
+        dir.delete();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
new file mode 100644
index 0000000..bbd6742
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
@@ -0,0 +1,65 @@
+/*
+ * 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.bnodes;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.util.List;
+
+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.Node;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.graph.GraphFactory;
+
+/**
+ *
+ */
+public abstract class AbstractTripleBlankNodeTests extends AbstractBlankNodeTests<Triple, TripleWritable> {
+    
+    /**
+     * Gets the language to use
+     * 
+     * @return Language
+     */
+    protected abstract Lang getLanguage();
+
+    @Override
+    protected Triple createTuple(Node s, Node p, Node o) {
+        return new Triple(s, p, o);
+    }
+
+    @Override
+    protected void writeTuples(File f, List<Triple> tuples) throws FileNotFoundException {
+        Graph g = GraphFactory.createGraphMem();
+        for (Triple t : tuples) {
+            g.add(t);
+        }
+        RDFDataMgr.write(new FileOutputStream(f), g, getLanguage());
+    }
+
+    @Override
+    protected Node getSubject(Triple value) {
+        return value.getSubject();
+    }
+
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
new file mode 100644
index 0000000..56d031e
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
@@ -0,0 +1,344 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.util.BlockInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.ReaderRIOT;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+import org.apache.jena.riot.lang.PipedRDFStream;
+import org.apache.jena.riot.system.ParserProfile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract implementation for a record reader that reads records from blocks
+ * of files, this is a hybrid between {@link AbstractLineBasedNodeTupleReader}
+ * and {@link AbstractWholeFileNodeTupleReader} in that it can only be used by
+ * formats which can be split by lines but reduces the overhead by parsing the
+ * split as a whole rather than as individual lines.
+ * <p>
+ * The keys produced are the approximate position in the file at which a tuple
+ * was found and the values will be node tuples. Positions are approximate
+ * because they are recorded after the point at which the most recent tuple was
+ * parsed from the input thus they reflect the approximate position in the
+ * stream immediately after which the triple was found.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Value type
+ * @param <T>
+ *            Tuple type
+ */
+public abstract class AbstractBlockBasedNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractBlockBasedNodeTupleReader.class);
+    private CompressionCodec compressionCodecs;
+    private TrackableInputStream input;
+    private LongWritable key;
+    private long start, length;
+    private T tuple;
+    private TrackedPipedRDFStream<TValue> stream;
+    private PipedRDFIterator<TValue> iter;
+    private Thread parserThread;
+    private boolean finished = false;
+    private boolean ignoreBadTuples = true;
+    private boolean parserFinished = false;
+    private Throwable parserError = null;
+
+    @Override
+    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
+        LOG.debug("initialize({}, {})", genericSplit, context);
+
+        // Assuming file split
+        if (!(genericSplit instanceof FileSplit))
+            throw new IOException("This record reader only supports FileSplit inputs");
+        FileSplit split = (FileSplit) genericSplit;
+
+        // Configuration
+        Configuration config = context.getConfiguration();
+        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
+        if (this.ignoreBadTuples)
+            LOG.warn(
+                    "Configured to ignore bad tuples, parsing errors will be logged and further parsing aborted but no user visible errors will be thrown.  Consider setting {} to false to disable this behaviour",
+                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
+
+        // Figure out what portion of the file to read
+        start = split.getStart();
+        long end = start + split.getLength();
+        final Path file = split.getPath();
+        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
+        boolean readToEnd = end == totalLength;
+        CompressionCodecFactory factory = new CompressionCodecFactory(config);
+        this.compressionCodecs = factory.getCodec(file);
+
+        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { start, split.getLength(), totalLength }));
+
+        // Open the file and prepare the input stream
+        FileSystem fs = file.getFileSystem(config);
+        FSDataInputStream fileIn = fs.open(file);
+        this.length = split.getLength();
+        if (start > 0)
+            fileIn.seek(start);
+
+        if (this.compressionCodecs != null) {
+            // Compressed input
+            // For compressed input NLineInputFormat will have failed to find
+            // any line breaks and will give us a split from 0 -> (length - 1)
+            // Add 1 and re-verify readToEnd so we can abort correctly if ever
+            // given a partial split of a compressed file
+            end++;
+            readToEnd = end == totalLength;
+            if (start > 0 || !readToEnd)
+                throw new IOException("This record reader can only be used with compressed input where the split is a whole file");
+            input = new TrackedInputStream(this.compressionCodecs.createInputStream(fileIn));
+        } else {
+            // Uncompressed input
+
+            if (readToEnd) {
+                input = new TrackedInputStream(fileIn);
+            } else {
+                // Need to limit the portion of the file we are reading
+                input = new BlockInputStream(fileIn, split.getLength());
+            }
+        }
+
+        // Set up background thread for parser
+        iter = this.getPipedIterator();
+        this.stream = this.getPipedStream(iter, this.input);
+        ParserProfile profile = RdfIOUtils.createParserProfile(context, file);
+        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage(), profile);
+        this.parserThread = new Thread(parserRunnable);
+        this.parserThread.setDaemon(true);
+        this.parserThread.start();
+    }
+
+    /**
+     * Gets the RDF iterator to use
+     * 
+     * @return Iterator
+     */
+    protected abstract PipedRDFIterator<TValue> getPipedIterator();
+
+    /**
+     * Gets the RDF stream to parse to
+     * 
+     * @param iterator
+     *            Iterator
+     * @return RDF stream
+     */
+    protected abstract TrackedPipedRDFStream<TValue> getPipedStream(PipedRDFIterator<TValue> iterator, TrackableInputStream input);
+
+    /**
+     * Gets the RDF language to use for parsing
+     * 
+     * @return
+     */
+    protected abstract Lang getRdfLanguage();
+
+    /**
+     * Creates the runnable upon which the parsing will run
+     * 
+     * @param input
+     *            Input
+     * @param stream
+     *            Stream
+     * @param lang
+     *            Language to use for parsing
+     * @return Parser runnable
+     */
+    private Runnable createRunnable(@SuppressWarnings("rawtypes") final AbstractBlockBasedNodeTupleReader reader, final InputStream input,
+            final PipedRDFStream<TValue> stream, final Lang lang, final ParserProfile profile) {
+        return new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    ReaderRIOT riotReader = RDFDataMgr.createReader(lang);
+                    riotReader.setParserProfile(profile);
+                    riotReader.read(input, null, lang.getContentType(), stream, null);
+                    //RDFDataMgr.parse(stream, input, null, lang);
+                    reader.setParserFinished(null);
+                } catch (Throwable e) {
+                    reader.setParserFinished(e);
+                }
+            }
+        };
+    }
+
+    /**
+     * Sets the parser thread finished state
+     * 
+     * @param e
+     *            Error (if any)
+     */
+    private void setParserFinished(Throwable e) {
+        synchronized (this.parserThread) {
+            this.parserError = e;
+            this.parserFinished = true;
+        }
+    }
+
+    /**
+     * Waits for the parser thread to have reported as finished
+     * 
+     * @throws InterruptedException
+     */
+    private void waitForParserFinished() throws InterruptedException {
+        do {
+            synchronized (this.parserThread) {
+                if (this.parserFinished)
+                    return;
+            }
+            Thread.sleep(50);
+        } while (true);
+    }
+
+    /**
+     * Creates an instance of a writable tuple from the given tuple value
+     * 
+     * @param tuple
+     *            Tuple value
+     * @return Writable tuple
+     */
+    protected abstract T createInstance(TValue tuple);
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+        // Reuse key for efficiency
+        if (key == null) {
+            key = new LongWritable();
+        }
+
+        if (this.finished)
+            return false;
+
+        try {
+            if (this.iter.hasNext()) {
+                // Position will be relative to the start for the split we're
+                // processing
+                Long l = this.start + this.stream.getPosition();
+                if (l != null) {
+                    this.key.set(l);
+                    // For compressed input the actual length from which we
+                    // calculate progress is likely less than the actual
+                    // uncompressed length so we need to increment the
+                    // length as we go along
+                    // We always add 1 more than the current length because we
+                    // don't want to report 100% progress until we really have
+                    // finished
+                    if (this.compressionCodecs != null && l > this.length)
+                        this.length = l + 1;
+                }
+                this.tuple = this.createInstance(this.iter.next());
+                return true;
+            } else {
+                // Need to ensure that the parser thread has finished in order
+                // to determine whether we finished without error
+                this.waitForParserFinished();
+                if (this.parserError != null) {
+                    LOG.error("Error parsing block, aborting further parsing", this.parserError);
+                    if (!this.ignoreBadTuples)
+                        throw new IOException("Error parsing block at position " + (this.start + this.input.getBytesRead()) + ", aborting further parsing",
+                                this.parserError);
+                }
+
+                this.key = null;
+                this.tuple = null;
+                this.finished = true;
+                // This is necessary so that when compressed input is used we
+                // report 100% progress once we've reached the genuine end of
+                // the stream
+                if (this.compressionCodecs != null)
+                    this.length--;
+                return false;
+            }
+        } catch (IOException e) {
+            throw e;
+        } catch (Throwable e) {
+            // Failed to read the tuple on this line
+            LOG.error("Error parsing block, aborting further parsing", e);
+            if (!this.ignoreBadTuples) {
+                this.iter.close();
+                throw new IOException("Error parsing block at position " + (this.start + this.input.getBytesRead()) + ", aborting further parsing", e);
+            }
+            this.key = null;
+            this.tuple = null;
+            this.finished = true;
+            return false;
+        }
+    }
+
+    @Override
+    public LongWritable getCurrentKey() throws IOException, InterruptedException {
+        return this.key;
+    }
+
+    @Override
+    public T getCurrentValue() throws IOException, InterruptedException {
+        return this.tuple;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+        float progress = 0.0f;
+        if (this.key == null) {
+            // We've either not started or we've finished
+            progress = (this.finished ? 1.0f : 0.0f);
+        } else if (this.key.get() == Long.MIN_VALUE) {
+            // We don't have a position so we've either in-progress or finished
+            progress = (this.finished ? 1.0f : 0.5f);
+        } else {
+            // We're some way through the file
+            progress = (this.key.get() - this.start) / (float) this.length;
+        }
+        LOG.debug("getProgress() --> {}", progress);
+        return progress;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.iter.close();
+        this.input.close();
+        this.finished = true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
new file mode 100644
index 0000000..2279444
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.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.input.readers;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedQuadsStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract record reader for whole file triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractBlockBasedQuadReader extends AbstractBlockBasedNodeTupleReader<Quad, QuadWritable> {
+
+    @Override
+    protected PipedRDFIterator<Quad> getPipedIterator() {
+        return new PipedRDFIterator<Quad>();
+    }
+
+    @Override
+    protected TrackedPipedRDFStream<Quad> getPipedStream(PipedRDFIterator<Quad> iterator, TrackableInputStream input) {
+        return new TrackedPipedQuadsStream(iterator, input);
+    }
+
+    @Override
+    protected QuadWritable createInstance(Quad tuple) {
+        return new QuadWritable(tuple);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
new file mode 100644
index 0000000..2afd329
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.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.input.readers;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedTriplesStream;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An abstract record reader for whole file triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractBlockBasedTripleReader extends AbstractBlockBasedNodeTupleReader<Triple, TripleWritable> {
+
+    @Override
+    protected PipedRDFIterator<Triple> getPipedIterator() {
+        return new PipedRDFIterator<Triple>();
+    }
+
+    @Override
+    protected TrackedPipedRDFStream<Triple> getPipedStream(PipedRDFIterator<Triple> iterator, TrackableInputStream input) {
+        return new TrackedPipedTriplesStream(iterator, input);
+    }
+
+    @Override
+    protected TripleWritable createInstance(Triple tuple) {
+        return new TripleWritable(tuple);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
new file mode 100644
index 0000000..6c1abe9
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
@@ -0,0 +1,265 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.util.LineReader;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.system.ParserProfile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract implementation of a record reader that reads records from line
+ * based tuple formats. This only supports reading from file splits currently.
+ * <p>
+ * The keys produced are the position of the line in the file and the values
+ * will be node tuples
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TValue>
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractLineBasedNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractLineBasedNodeTupleReader.class);
+    private CompressionCodecFactory compressionCodecs = null;
+    private long start, pos, end, estLength;
+    private int maxLineLength;
+    private LineReader in;
+    private LongWritable key = null;
+    private Text value = null;
+    private T tuple = null;
+    private ParserProfile profile = null;
+    private boolean ignoreBadTuples = true;
+
+    @Override
+    public final void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
+        LOG.debug("initialize({}, {})", genericSplit, context);
+
+        // Assuming file split
+        if (!(genericSplit instanceof FileSplit))
+            throw new IOException("This record reader only supports FileSplit inputs");
+        FileSplit split = (FileSplit) genericSplit;
+
+        // Configuration
+        profile = RdfIOUtils.createParserProfile(context, split.getPath());
+        Configuration config = context.getConfiguration();
+        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
+        if (this.ignoreBadTuples)
+            LOG.warn(
+                    "Configured to ignore bad tuples, parsing errors will be logged and the bad line skipped but no errors will be thrownConsider setting {} to false to disable this behaviour",
+                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
+
+        // Figure out what portion of the file to read
+        this.maxLineLength = config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE);
+        start = split.getStart();
+        end = start + split.getLength();
+        final Path file = split.getPath();
+        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
+        compressionCodecs = new CompressionCodecFactory(config);
+        final CompressionCodec codec = compressionCodecs.getCodec(file);
+
+        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { start, split.getLength(), totalLength }));
+
+        // Open the file and seek to the start of the split
+        FileSystem fs = file.getFileSystem(config);
+        FSDataInputStream fileIn = fs.open(file);
+        boolean skipFirstLine = false;
+        if (codec != null) {
+            // Compressed input
+            // For compressed input NLineInputFormat will have failed to find
+            // any line breaks and will give us a split from 0 -> (length - 1)
+            // Add 1 and verify we got complete split
+            if (totalLength > split.getLength() + 1)
+                throw new IOException("This record reader can only be used with compressed input where the split covers the whole file");
+            in = new LineReader(codec.createInputStream(fileIn), config);
+            estLength = end;
+            end = Long.MAX_VALUE;
+        } else {
+            // Uncompressed input
+            if (start != 0) {
+                skipFirstLine = true;
+                --start;
+                fileIn.seek(start);
+            }
+            in = new LineReader(fileIn, config);
+        }
+        // Skip first line and re-establish "start".
+        // This is to do with how line reader reads lines and how
+        // NLineInputFormat will provide the split information to use
+        if (skipFirstLine) {
+            start += in.readLine(new Text(), 0, (int) Math.min((long) Integer.MAX_VALUE, end - start));
+        }
+        this.pos = start;
+    }
+
+    /**
+     * Gets an iterator over the data on the current line
+     * 
+     * @param line
+     *            Line
+     * @param profile
+     *            Parser profile
+     * @return Iterator
+     */
+    protected abstract Iterator<TValue> getIterator(String line, ParserProfile profile);
+
+    /**
+     * Creates an instance of a writable tuple from the given tuple value
+     * 
+     * @param tuple
+     *            Tuple value
+     * @return Writable tuple
+     */
+    protected abstract T createInstance(TValue tuple);
+
+    @Override
+    public final boolean nextKeyValue() throws IOException, InterruptedException {
+        // Reuse key for efficiency
+        if (key == null) {
+            key = new LongWritable();
+        }
+
+        // Reset value which we use for reading lines
+        if (value == null) {
+            value = new Text();
+        }
+        tuple = null;
+
+        // Try to read the next valid line
+        int newSize = 0;
+        while (pos < end) {
+            // Read next line
+            newSize = in.readLine(value, maxLineLength, Math.max((int) Math.min(Integer.MAX_VALUE, end - pos), maxLineLength));
+
+            // Once we get an empty line we've reached the end of our input
+            if (newSize == 0) {
+                break;
+            }
+
+            // Update position, remember that where inputs are compressed we may
+            // be at a larger position then we expected because the length of
+            // the split is likely less than the length of the data once
+            // decompressed
+            key.set(pos);
+            pos += newSize;
+            if (pos > estLength)
+                estLength = pos + 1;
+
+            // Skip lines that exceed the line length limit that has been set
+            if (newSize >= maxLineLength) {
+                LOG.warn("Skipped oversized line of size {} at position {}", newSize, (pos - newSize));
+                continue;
+            }
+
+            // Attempt to read the tuple from current line
+            try {
+                Iterator<TValue> iter = this.getIterator(value.toString(), profile);
+                if (iter.hasNext()) {
+                    tuple = this.createInstance(iter.next());
+
+                    // If we reach here we've found a valid tuple so we can
+                    // break out of the loop
+                    break;
+                } else {
+                    // Empty line/Comment line
+                    LOG.debug("Valid line with no triple at position {}", (pos - newSize));
+                    continue;
+                }
+            } catch (Throwable e) {
+                // Failed to read the tuple on this line
+                LOG.error("Bad tuple at position " + (pos - newSize), e);
+                if (this.ignoreBadTuples)
+                    continue;
+                throw new IOException(String.format("Bad tuple at position %d", (pos - newSize)), e);
+            }
+        }
+        boolean result = this.tuple != null;
+
+        // End of input
+        if (newSize == 0) {
+            key = null;
+            value = null;
+            tuple = null;
+            result = false;
+            estLength = pos;
+        }
+        LOG.debug("nextKeyValue() --> {}", result);
+        return result;
+    }
+
+    @Override
+    public LongWritable getCurrentKey() throws IOException, InterruptedException {
+        LOG.debug("getCurrentKey() --> {}", key);
+        return key;
+    }
+
+    @Override
+    public T getCurrentValue() throws IOException, InterruptedException {
+        LOG.debug("getCurrentValue() --> {}", tuple);
+        return tuple;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+        float progress = 0.0f;
+        if (start != end) {
+            if (end == Long.MAX_VALUE) {
+                if (estLength == 0)
+                    return 1.0f;
+                // Use estimated length
+                progress = Math.min(1.0f, (pos - start) / (float) (estLength - start));
+            } else {
+                // Use actual length
+                progress = Math.min(1.0f, (pos - start) / (float) (end - start));
+            }
+        }
+        LOG.debug("getProgress() --> {}", progress);
+        return progress;
+    }
+
+    @Override
+    public void close() throws IOException {
+        LOG.debug("close()");
+        if (in != null) {
+            in.close();
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
new file mode 100644
index 0000000..ac93865
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.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.input.readers;
+
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.tokens.Tokenizer;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract reader for line based quad formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractLineBasedQuadReader extends AbstractLineBasedNodeTupleReader<Quad, QuadWritable> {
+
+    @Override
+    protected Iterator<Quad> getIterator(String line, ParserProfile profile) {
+        Tokenizer tokenizer = getTokenizer(line);
+        return getQuadsIterator(tokenizer, profile);
+    }
+
+    @Override
+    protected QuadWritable createInstance(Quad q) {
+        return new QuadWritable(q);
+    }
+
+    protected abstract Tokenizer getTokenizer(String line);
+
+    protected abstract Iterator<Quad> getQuadsIterator(Tokenizer tokenizer, ParserProfile profile);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
new file mode 100644
index 0000000..a0232f5
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.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.input.readers;
+
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.tokens.Tokenizer;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An abstract record reader for line based triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractLineBasedTripleReader extends AbstractLineBasedNodeTupleReader<Triple, TripleWritable> {
+
+    @Override
+    protected Iterator<Triple> getIterator(String line, ParserProfile profile) {
+        Tokenizer tokenizer = getTokenizer(line);
+        return getTriplesIterator(tokenizer, profile);
+    }
+
+    @Override
+    protected TripleWritable createInstance(Triple t) {
+        return new TripleWritable(t);
+    }
+    
+    protected abstract Tokenizer getTokenizer(String line);
+
+    protected abstract Iterator<Triple> getTriplesIterator(Tokenizer tokenizer, ParserProfile profile);
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
new file mode 100644
index 0000000..d0ffed8
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
@@ -0,0 +1,108 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+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.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract record reader for arbitrary RDF which provides support for
+ * selecting the actual record reader to use based on detecting the RDF language
+ * from the file name
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractRdfReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        RecordReader<LongWritable, T> {
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractRdfReader.class);
+
+    private RecordReader<LongWritable, T> reader;
+
+    @Override
+    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException,
+            InterruptedException {
+        LOG.debug("initialize({}, {})", genericSplit, context);
+
+        // Assuming file split
+        if (!(genericSplit instanceof FileSplit))
+            throw new IOException("This record reader only supports FileSplit inputs");
+
+        // Find RDF language
+        FileSplit split = (FileSplit) genericSplit;
+        Path path = split.getPath();
+        Lang lang = RDFLanguages.filenameToLang(path.getName());
+        if (lang == null)
+            throw new IOException("There is no registered RDF language for the input file " + path.toString());
+
+        // Select the record reader and initialize
+        this.reader = this.selectRecordReader(lang);
+        this.reader.initialize(split, context);
+    }
+
+    /**
+     * Selects the appropriate record reader to use for the given RDF language
+     * 
+     * @param lang
+     *            RDF language
+     * @return Record reader
+     * @throws IOException
+     *             Should be thrown if no record reader can be selected
+     */
+    protected abstract RecordReader<LongWritable, T> selectRecordReader(Lang lang) throws IOException;
+
+    @Override
+    public final boolean nextKeyValue() throws IOException, InterruptedException {
+        return this.reader.nextKeyValue();
+    }
+
+    @Override
+    public final LongWritable getCurrentKey() throws IOException, InterruptedException {
+        return this.reader.getCurrentKey();
+    }
+
+    @Override
+    public final T getCurrentValue() throws IOException, InterruptedException {
+        return this.reader.getCurrentValue();
+    }
+
+    @Override
+    public final float getProgress() throws IOException, InterruptedException {
+        return this.reader.getProgress();
+    }
+
+    @Override
+    public final void close() throws IOException {
+        this.reader.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
new file mode 100644
index 0000000..c2da3f7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
@@ -0,0 +1,328 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.ReaderRIOT;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+import org.apache.jena.riot.lang.PipedRDFStream;
+import org.apache.jena.riot.system.ParserProfile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract implementation for a record reader that reads records from whole
+ * files i.e. the whole file must be kept together to allow tuples to be
+ * successfully read. This only supports reading from file splits currently.
+ * <p>
+ * The keys produced are the approximate position in the file at which a tuple
+ * was found and the values will be node tuples. Positions are approximate
+ * because they are recorded after the point at which the most recent tuple was
+ * parsed from the input thus they reflect the approximate position in the
+ * stream immediately after which the triple was found.
+ * </p>
+ * <p>
+ * You should also be aware that with whole file formats syntax compressions in
+ * the format may mean that there are multiple triples produced with the same
+ * position and thus key.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Value type
+ * @param <T>
+ *            Tuple type
+ */
+public abstract class AbstractWholeFileNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractLineBasedNodeTupleReader.class);
+    private CompressionCodec compressionCodecs;
+    private TrackedInputStream input;
+    private LongWritable key;
+    private long length;
+    private T tuple;
+    private TrackedPipedRDFStream<TValue> stream;
+    private PipedRDFIterator<TValue> iter;
+    private Thread parserThread;
+    private boolean finished = false;
+    private boolean ignoreBadTuples = true;
+    private boolean parserFinished = false;
+    private Throwable parserError = null;
+
+    @Override
+    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
+        LOG.debug("initialize({}, {})", genericSplit, context);
+
+        // Assuming file split
+        if (!(genericSplit instanceof FileSplit))
+            throw new IOException("This record reader only supports FileSplit inputs");
+        FileSplit split = (FileSplit) genericSplit;
+
+        // Configuration
+        Configuration config = context.getConfiguration();
+        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
+        if (this.ignoreBadTuples)
+            LOG.warn(
+                    "Configured to ignore bad tuples, parsing errors will be logged and further parsing aborted but no user visible errors will be thrown.  Consider setting {} to false to disable this behaviour",
+                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
+
+        // Figure out what portion of the file to read
+        if (split.getStart() > 0)
+            throw new IOException("This record reader requires a file split which covers the entire file");
+        final Path file = split.getPath();
+        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
+        CompressionCodecFactory factory = new CompressionCodecFactory(config);
+        this.compressionCodecs = factory.getCodec(file);
+
+        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { split.getStart(), split.getLength(),
+                totalLength }));
+
+        if (totalLength > split.getLength())
+            throw new IOException("This record reader requires a file split which covers the entire file");
+
+        // Open the file and prepare the input stream
+        FileSystem fs = file.getFileSystem(config);
+        FSDataInputStream fileIn = fs.open(file);
+        this.length = split.getLength();
+        if (this.compressionCodecs != null) {
+            // Compressed input
+            input = new TrackedInputStream(this.compressionCodecs.createInputStream(fileIn));
+        } else {
+            // Uncompressed input
+            input = new TrackedInputStream(fileIn);
+        }
+
+        // Set up background thread for parser
+        iter = this.getPipedIterator();
+        this.stream = this.getPipedStream(iter, this.input);
+        ParserProfile profile = RdfIOUtils.createParserProfile(context, file);
+        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage(), profile);
+        this.parserThread = new Thread(parserRunnable);
+        this.parserThread.setDaemon(true);
+        this.parserThread.start();
+    }
+
+    /**
+     * Gets the RDF iterator to use
+     * 
+     * @return Iterator
+     */
+    protected abstract PipedRDFIterator<TValue> getPipedIterator();
+
+    /**
+     * Gets the RDF stream to parse to
+     * 
+     * @param iterator
+     *            Iterator
+     * @return RDF stream
+     */
+    protected abstract TrackedPipedRDFStream<TValue> getPipedStream(PipedRDFIterator<TValue> iterator, TrackableInputStream input);
+
+    /**
+     * Gets the RDF language to use for parsing
+     * 
+     * @return
+     */
+    protected abstract Lang getRdfLanguage();
+
+    /**
+     * Creates the runnable upon which the parsing will run
+     * 
+     * @param input
+     *            Input
+     * @param stream
+     *            Stream
+     * @param lang
+     *            Language to use for parsing
+     * @return Parser runnable
+     */
+    private Runnable createRunnable(@SuppressWarnings("rawtypes") final AbstractWholeFileNodeTupleReader reader, final InputStream input,
+            final PipedRDFStream<TValue> stream, final Lang lang, final ParserProfile profile) {
+        return new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    ReaderRIOT riotReader = RDFDataMgr.createReader(lang);
+                    riotReader.setParserProfile(profile);
+                    riotReader.read(input, null, lang.getContentType(), stream, null);
+                    reader.setParserFinished(null);
+                } catch (Throwable e) {
+                    reader.setParserFinished(e);
+                }
+            }
+        };
+    }
+
+    /**
+     * Sets the parser thread finished state
+     * 
+     * @param e
+     *            Error (if any)
+     */
+    private void setParserFinished(Throwable e) {
+        synchronized (this.parserThread) {
+            this.parserError = e;
+            this.parserFinished = true;
+        }
+    }
+
+    /**
+     * Waits for the parser thread to have reported as finished
+     * 
+     * @throws InterruptedException
+     */
+    private void waitForParserFinished() throws InterruptedException {
+        do {
+            synchronized (this.parserThread) {
+                if (this.parserFinished)
+                    return;
+            }
+            Thread.sleep(50);
+        } while (true);
+    }
+
+    /**
+     * Creates an instance of a writable tuple from the given tuple value
+     * 
+     * @param tuple
+     *            Tuple value
+     * @return Writable tuple
+     */
+    protected abstract T createInstance(TValue tuple);
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+        // Reuse key for efficiency
+        if (key == null) {
+            key = new LongWritable();
+        }
+
+        if (this.finished)
+            return false;
+
+        try {
+            if (this.iter.hasNext()) {
+                Long l = this.stream.getPosition();
+                if (l != null) {
+                    this.key.set(l);
+                    // For compressed input the actual length from which we
+                    // calculate progress is likely less than the actual
+                    // uncompressed length so we may need to increment the
+                    // length as we go along
+                    // We always add 1 more than the current length because we
+                    // don't want to report 100% progress until we really have
+                    // finished
+                    if (this.compressionCodecs != null && l > this.length)
+                        this.length = l + 1;
+                }
+                this.tuple = this.createInstance(this.iter.next());
+                return true;
+            } else {
+                // Need to ensure that the parser thread has finished in order
+                // to determine whether we finished without error
+                this.waitForParserFinished();
+                if (this.parserError != null) {
+                    LOG.error("Error parsing whole file, aborting further parsing", this.parserError);
+                    if (!this.ignoreBadTuples)
+                        throw new IOException("Error parsing whole file at position " + this.input.getBytesRead() + ", aborting further parsing",
+                                this.parserError);
+
+                }
+
+                this.key = null;
+                this.tuple = null;
+                this.finished = true;
+                // This is necessary so that when compressed input is used we
+                // report 100% progress once we've reached the genuine end of
+                // the stream
+                if (this.compressionCodecs != null)
+                    this.length--;
+                return false;
+            }
+        } catch (Throwable e) {
+            // Failed to read the tuple on this line
+            LOG.error("Error parsing whole file, aborting further parsing", e);
+            if (!this.ignoreBadTuples) {
+                this.iter.close();
+                throw new IOException("Error parsing whole file at position " + this.input.getBytesRead() + ", aborting further parsing", e);
+            }
+            this.key = null;
+            this.tuple = null;
+            this.finished = true;
+            return false;
+        }
+    }
+
+    @Override
+    public LongWritable getCurrentKey() throws IOException, InterruptedException {
+        return this.key;
+    }
+
+    @Override
+    public T getCurrentValue() throws IOException, InterruptedException {
+        return this.tuple;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+        float progress = 0.0f;
+        if (this.key == null) {
+            // We've either not started or we've finished
+            progress = (this.finished ? 1.0f : 0.0f);
+        } else if (this.key.get() == Long.MIN_VALUE) {
+            // We don't have a position so we've either in-progress or finished
+            progress = (this.finished ? 1.0f : 0.5f);
+        } else {
+            // We're some way through the file
+            progress = this.key.get() / (float) this.length;
+        }
+        LOG.debug("getProgress() --> {}", progress);
+        return progress;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.iter.close();
+        this.input.close();
+        this.finished = true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
new file mode 100644
index 0000000..e525bea
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.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.input.readers;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedQuadsStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract record reader for whole file triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractWholeFileQuadReader extends AbstractWholeFileNodeTupleReader<Quad, QuadWritable> {
+
+    @Override
+    protected PipedRDFIterator<Quad> getPipedIterator() {
+        return new PipedRDFIterator<Quad>();
+    }
+
+    @Override
+    protected TrackedPipedRDFStream<Quad> getPipedStream(PipedRDFIterator<Quad> iterator, TrackableInputStream input) {
+        return new TrackedPipedQuadsStream(iterator, input);
+    }
+
+    @Override
+    protected QuadWritable createInstance(Quad tuple) {
+        return new QuadWritable(tuple);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
new file mode 100644
index 0000000..8710b99
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.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.input.readers;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedTriplesStream;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An abstract record reader for whole file triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractWholeFileTripleReader extends AbstractWholeFileNodeTupleReader<Triple, TripleWritable> {
+
+    @Override
+    protected PipedRDFIterator<Triple> getPipedIterator() {
+        return new PipedRDFIterator<Triple>();
+    }
+
+    @Override
+    protected TrackedPipedRDFStream<Triple> getPipedStream(PipedRDFIterator<Triple> iterator, TrackableInputStream input) {
+        return new TrackedPipedTriplesStream(iterator, input);
+    }
+
+    @Override
+    protected TripleWritable createInstance(Triple tuple) {
+        return new TripleWritable(tuple);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
new file mode 100644
index 0000000..26b0a8b
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+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;
+
+/**
+ * A record reader that reads triples from any RDF quads format
+ */
+public class QuadsReader extends AbstractRdfReader<Quad, QuadWritable> {
+
+    @Override
+    protected RecordReader<LongWritable, QuadWritable> selectRecordReader(Lang lang) throws IOException {
+        if (!RDFLanguages.isQuads(lang))
+            throw new IOException(
+                    lang.getLabel()
+                            + " is not a RDF quads format, perhaps you wanted TriplesInputFormat or TriplesOrQuadsInputFormat instead?");
+
+        // This will throw an appropriate error if the language does not support
+        // triples
+        return HadoopRdfIORegistry.createQuadReader(lang);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
new file mode 100644
index 0000000..b071f67
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
@@ -0,0 +1,72 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+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.graph.Node;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A record reader that reads RDF from any triples/quads format. Triples are
+ * converted into quads in the default graph. This behaviour can be changed by
+ * deriving from this class and overriding the {@link #getGraphNode()} method
+ * 
+ * 
+ * 
+ */
+@SuppressWarnings("javadoc")
+public class TriplesOrQuadsReader extends AbstractRdfReader<Quad, QuadWritable> {
+
+    @Override
+    protected RecordReader<LongWritable, QuadWritable> selectRecordReader(Lang lang) throws IOException {
+        if (!RDFLanguages.isQuads(lang) && !RDFLanguages.isTriples(lang))
+            throw new IOException(lang.getLabel() + " is not a RDF triples/quads format");
+
+        if (HadoopRdfIORegistry.hasQuadReader(lang)) {
+            // Supports quads directly
+            return HadoopRdfIORegistry.createQuadReader(lang);
+        } else {
+            // Try to create a triples reader and wrap upwards into quads
+            // This will throw an error if a triple reader is not available
+            return new TriplesToQuadsReader(HadoopRdfIORegistry.createTripleReader(lang));
+        }
+    }
+
+    /**
+     * Gets the graph node which represents the graph into which triples will be
+     * indicated to belong to when they are converting into quads.
+     * <p>
+     * Defaults to {@link Quad#defaultGraphNodeGenerated} which represents the
+     * default graph
+     * </p>
+     * 
+     * @return Graph node
+     */
+    protected Node getGraphNode() {
+        return Quad.defaultGraphNodeGenerated;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
new file mode 100644
index 0000000..96c4a65
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+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;
+
+/**
+ * A record reader that reads triples from any RDF triples format
+ */
+public class TriplesReader extends AbstractRdfReader<Triple, TripleWritable> {
+
+    @Override
+    protected RecordReader<LongWritable, TripleWritable> selectRecordReader(Lang lang) throws IOException {
+        if (!RDFLanguages.isTriples(lang))
+            throw new IOException(
+                    lang.getLabel()
+                            + " is not a RDF triples format, perhaps you wanted QuadsInputFormat or TriplesOrQuadsInputFormat instead?");
+
+        // This will throw an appropriate error if the language does not support
+        // triples
+        return HadoopRdfIORegistry.createTripleReader(lang);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
new file mode 100644
index 0000000..a388f0e
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
@@ -0,0 +1,102 @@
+/*
+ * 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.readers;
+
+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.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A record reader that converts triples into quads by wrapping a
+ * {@code RecordReader<LongWritable, TripleWritable>} implementation
+ * 
+ * 
+ * 
+ */
+public class TriplesToQuadsReader extends RecordReader<LongWritable, QuadWritable> {
+
+    private final RecordReader<LongWritable, TripleWritable> reader;
+    private Node graph;
+
+    /**
+     * Creates a new reader
+     * 
+     * @param reader
+     *            Triple reader
+     */
+    public TriplesToQuadsReader(RecordReader<LongWritable, TripleWritable> reader) {
+        this(reader, Quad.defaultGraphNodeGenerated);
+    }
+
+    /**
+     * Creates a new reader
+     * 
+     * @param reader
+     *            Triple reader
+     * @param graphNode
+     *            Graph node
+     */
+    public TriplesToQuadsReader(RecordReader<LongWritable, TripleWritable> reader, Node graphNode) {
+        if (reader == null)
+            throw new NullPointerException("reader cannot be null");
+        if (graphNode == null)
+            throw new NullPointerException("Graph node cannot be null");
+        this.reader = reader;
+        this.graph = graphNode;
+    }
+
+    @Override
+    public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
+        this.reader.initialize(split, context);
+    }
+
+    @Override
+    public final boolean nextKeyValue() throws IOException, InterruptedException {
+        return this.reader.nextKeyValue();
+    }
+
+    @Override
+    public final LongWritable getCurrentKey() throws IOException, InterruptedException {
+        return this.reader.getCurrentKey();
+    }
+
+    @Override
+    public final QuadWritable getCurrentValue() throws IOException, InterruptedException {
+        TripleWritable t = this.reader.getCurrentValue();
+        return new QuadWritable(new Quad(this.graph, t.get()));
+    }
+
+    @Override
+    public final float getProgress() throws IOException, InterruptedException {
+        return this.reader.getProgress();
+    }
+
+    @Override
+    public final void close() throws IOException {
+        this.reader.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
new file mode 100644
index 0000000..1b3f467
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
@@ -0,0 +1,32 @@
+/*
+ * 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.readers.jsonld;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
new file mode 100644
index 0000000..7cdea9e
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
@@ -0,0 +1,30 @@
+/*
+ * 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.readers.jsonld;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
+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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
new file mode 100644
index 0000000..cef8ef1
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.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.input.readers.nquads;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractBlockBasedQuadReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record record for NQuads
+ * <p>
+ * This is a hybrid of the {@link NQuadsReader} and the
+ * {@link WholeFileNQuadsReader} in that it does not process individual lines
+ * rather it processes the inputs in blocks of lines parsing the whole block
+ * rather than individual lines. This provides a compromise between the higher
+ * parser setup of creating more parsers and the benefit of being able to split
+ * input files over multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class BlockedNQuadsReader extends AbstractBlockBasedQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java
new file mode 100644
index 0000000..e00e318
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.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.readers.nquads;
+
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractLineBasedQuadReader;
+import org.apache.jena.riot.lang.LangNQuads;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.tokens.Tokenizer;
+import org.apache.jena.riot.tokens.TokenizerFactory;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A record reader for NQuads
+ * 
+ * 
+ * 
+ */
+public class NQuadsReader extends AbstractLineBasedQuadReader {
+
+    @Override
+    protected Tokenizer getTokenizer(String line) {
+        return TokenizerFactory.makeTokenizerString(line);
+    }
+
+    @Override
+    protected Iterator<Quad> getQuadsIterator(Tokenizer tokenizer, ParserProfile profile) {
+        return new LangNQuads(tokenizer, profile, null);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
new file mode 100644
index 0000000..96e6f80
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.readers.nquads;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record record for NQuads
+ * <p>
+ * Unlike the {@link NQuadsReader} this processes files as a whole rather than
+ * individual lines. This has the advantage of less parser setup overhead but
+ * the disadvantage that the input cannot be split between multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class WholeFileNQuadsReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java
new file mode 100644
index 0000000..7268d5a
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.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.input.readers.ntriples;
+
+import org.apache.jena.hadoop.rdf.io.input.readers.AbstractBlockBasedTripleReader;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record record for NTriples
+ * <p>
+ * This is a hybrid of the {@link NTriplesReader} and the
+ * {@link WholeFileNTriplesReader} in that it does not process individual lines
+ * rather it processes the inputs in blocks of lines parsing the whole block
+ * rather than individual lines. This provides a compromise between the higher
+ * parser setup of creating more parsers and the benefit of being able to split
+ * input files over multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class BlockedNTriplesReader extends AbstractBlockBasedTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java
new file mode 100644
index 0000000..30bd4fa
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.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.writers.rdfjson;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record writer for RDF/JSON
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * 
+ */
+public class RdfJsonWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
+
+    /**
+     * Creates a new record writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public RdfJsonWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFJSON;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java
new file mode 100644
index 0000000..f202914
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.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.writers.rdfxml;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record writer for RDF/XML
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * 
+ */
+public class RdfXmlWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
+
+    /**
+     * Creates a new record writer
+     * 
+     * @param writer
+     *            Writer
+     */
+    public RdfXmlWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFXML;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
new file mode 100644
index 0000000..599dba9
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
@@ -0,0 +1,38 @@
+/*
+ * 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.thrift;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileQuadWriter;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class ThriftQuadWriter<TKey> extends AbstractWholeFileQuadWriter<TKey> {
+
+    public ThriftQuadWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
new file mode 100644
index 0000000..0089459
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
@@ -0,0 +1,38 @@
+/*
+ * 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.thrift;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+public class ThriftTripleWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
+
+    public ThriftTripleWriter(Writer writer) {
+        super(writer);
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java
new file mode 100644
index 0000000..4c0c75f
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.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.writers.trig;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedQuadWriter;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record writer for TriG that uses the batched approach, note that this
+ * approach will produce invalid data when blank nodes span batches
+ *  
+ * @param <TKey>
+ *            Key type
+ */
+public class BatchedTriGWriter<TKey> extends AbstractBatchedQuadWriter<TKey> {
+
+	/**
+	 * Creates a new record writer
+	 * 
+	 * @param writer
+	 *            Writer
+	 * @param batchSize
+	 *            Batch size
+	 */
+	public BatchedTriGWriter(Writer writer, long batchSize) {
+		super(writer, batchSize);
+	}
+
+	@Override
+	protected Lang getRdfLanguage() {
+		return Lang.TRIG;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java
new file mode 100644
index 0000000..86ab2f9
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.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.writers.turtle;
+
+import java.io.Writer;
+
+import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedTripleWriter;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A record writer for Turtle that uses the batched approach, note that this
+ * approach will produce invalid data when blank nodes span batches
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class BatchedTurtleWriter<TKey> extends
+		AbstractBatchedTripleWriter<TKey> {
+
+	/**
+	 * Creates a new record writer
+	 * 
+	 * @param writer
+	 *            Writer
+	 * @param batchSize
+	 *            Batch size
+	 */
+	public BatchedTurtleWriter(Writer writer, long batchSize) {
+		super(writer, batchSize);
+	}
+
+	@Override
+	protected Lang getRdfLanguage() {
+		return Lang.TURTLE;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
new file mode 100644
index 0000000..7b04ef4
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
@@ -0,0 +1,310 @@
+/*
+ * 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.registry;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * A registry which is used by various classes to dynamically select record
+ * readers and writers based on a provided {@link Lang}
+ * <p>
+ * Readers and writers are dynamically discovered using the Java
+ * {@link ServiceLoader} mechanism. This will look for files under
+ * {@code META-INF/services} named
+ * {@code org.apache.jena.hadoop.rdf.io.registry.ReaderFactory} and
+ * {@code org.apache.jena.hadoop.rdf.io.registry.WriterFactory}. This follows
+ * the standard {@linkplain ServiceLoader} format of provided one class name per
+ * line which implements the relevant interface.
+ * </p>
+ * 
+ */
+public class HadoopRdfIORegistry {
+
+    private static Map<Lang, ReaderFactory> readerFactories = new HashMap<>();
+    private static Map<Lang, WriterFactory> writerFactories = new HashMap<>();
+    private static boolean init = false;
+
+    static {
+        init();
+    }
+
+    private static synchronized void init() {
+        if (init)
+            return;
+
+        // Dynamically load and register reader factories
+        ServiceLoader<ReaderFactory> readerFactoryLoader = ServiceLoader.load(ReaderFactory.class);
+        Iterator<ReaderFactory> readerFactoryIterator = readerFactoryLoader.iterator();
+        while (readerFactoryIterator.hasNext()) {
+            ReaderFactory f = readerFactoryIterator.next();
+            addReaderFactory(f);
+        }
+
+        // Dynamically load and register writer factories
+        ServiceLoader<WriterFactory> writerFactoryLoader = ServiceLoader.load(WriterFactory.class);
+        Iterator<WriterFactory> writerFactoryIterator = writerFactoryLoader.iterator();
+        while (writerFactoryIterator.hasNext()) {
+            WriterFactory f = writerFactoryIterator.next();
+            addWriterFactory(f);
+        }
+
+        init = true;
+    }
+
+    /**
+     * Resets the registry to the default configuration
+     */
+    public static synchronized void reset() {
+        if (!init)
+            return;
+
+        init = false;
+        init();
+    }
+
+    /**
+     * Registers the reader factory for all the languages it declares itself as
+     * supporting
+     * 
+     * @param f
+     *            Reader factory
+     */
+    public static void addReaderFactory(ReaderFactory f) {
+        if (f == null)
+            throw new NullPointerException("Factory cannot be null");
+
+        readerFactories.put(f.getPrimaryLanguage(), f);
+        for (Lang altLang : f.getAlternativeLanguages()) {
+            readerFactories.put(altLang, f);
+        }
+    }
+
+    /**
+     * Registers the writer factory for all the languages it declares itself as
+     * supporting
+     * 
+     * @param f
+     *            Writer factory
+     */
+    public static void addWriterFactory(WriterFactory f) {
+        if (f == null)
+            throw new NullPointerException("Factory cannot be null");
+
+        writerFactories.put(f.getPrimaryLanguage(), f);
+        for (Lang altLang : f.getAlternativeLanguages()) {
+            writerFactories.put(altLang, f);
+        }
+    }
+
+    /**
+     * Gets whether there is a quad reader available for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return True if available, false otherwise
+     */
+    public static boolean hasQuadReader(Lang lang) {
+        if (lang == null)
+            return false;
+
+        ReaderFactory f = readerFactories.get(lang);
+        if (f == null)
+            return false;
+        return f.canReadQuads();
+    }
+
+    /**
+     * Gets whether there is a triple reader available for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return True if available, false otherwise
+     */
+    public static boolean hasTriplesReader(Lang lang) {
+        if (lang == null)
+            return false;
+
+        ReaderFactory f = readerFactories.get(lang);
+        if (f == null)
+            return false;
+        return f.canReadTriples();
+    }
+
+    /**
+     * Tries to create a quad reader for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return Quad reader if one is available
+     * @throws IOException
+     *             Thrown if a quad reader is not available or the given
+     *             language does not support quads
+     */
+    public static RecordReader<LongWritable, QuadWritable> createQuadReader(Lang lang) throws IOException {
+        if (lang == null)
+            throw new IOException("Cannot create a quad reader for an undefined language");
+
+        ReaderFactory f = readerFactories.get(lang);
+        if (f == null)
+            throw new IOException("No factory registered for language " + lang.getName());
+        if (!f.canReadQuads())
+            throw new IOException(lang.getName() + " does not support reading quads");
+
+        RecordReader<LongWritable, QuadWritable> reader = f.createQuadReader();
+        if (reader == null)
+            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples reader");
+        return reader;
+    }
+
+    /**
+     * Tries to create a triple reader for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return Triple reader if one is available
+     * @throws IOException
+     *             Thrown if a triple reader is not available or the given
+     *             language does not support triple
+     */
+    public static RecordReader<LongWritable, TripleWritable> createTripleReader(Lang lang) throws IOException {
+        if (lang == null)
+            throw new IOException("Cannot create a triple reader for an undefined language");
+
+        ReaderFactory f = readerFactories.get(lang);
+        if (f == null)
+            throw new IOException("No factory registered for language " + lang.getName());
+        if (!f.canReadTriples())
+            throw new IOException(lang.getName() + " does not support reading triples");
+
+        RecordReader<LongWritable, TripleWritable> reader = f.createTripleReader();
+        if (reader == null)
+            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples reader");
+        return reader;
+    }
+
+    /**
+     * Gets whether there is a quad writer available for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return True if available, false otherwise
+     */
+    public static boolean hasQuadWriter(Lang lang) {
+        if (lang == null)
+            return false;
+
+        WriterFactory f = writerFactories.get(lang);
+        if (f == null)
+            return false;
+        return f.canWriteQuads();
+    }
+
+    /**
+     * Gets whether there is a triple writer available for the given language
+     * 
+     * @param lang
+     *            Language
+     * @return True if available, false otherwise
+     */
+    public static boolean hasTriplesWriter(Lang lang) {
+        if (lang == null)
+            return false;
+
+        WriterFactory f = writerFactories.get(lang);
+        if (f == null)
+            return false;
+        return f.canWriteTriples();
+    }
+
+    /**
+     * Tries to create a quad writer for the given language
+     * 
+     * @param lang
+     *            Language
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * 
+     * @return Quad writer if one is available
+     * @throws IOException
+     *             Thrown if a quad writer is not available or the given
+     *             language does not support quads
+     */
+    public static <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Lang lang, Writer writer,
+            Configuration config) throws IOException {
+        if (lang == null)
+            throw new IOException("Cannot create a quad writer for an undefined language");
+
+        WriterFactory f = writerFactories.get(lang);
+        if (f == null)
+            throw new IOException("No factory registered for language " + lang.getName());
+        if (!f.canWriteQuads())
+            throw new IOException(lang.getName() + " does not support writeing quads");
+
+        RecordWriter<TKey, QuadWritable> rwriter = f.<TKey> createQuadWriter(writer, config);
+        if (rwriter == null)
+            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples writer");
+        return rwriter;
+    }
+
+    /**
+     * Tries to create a triple writer for the given language
+     * 
+     * @param lang
+     *            Language
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * @return Triple writer if one is available
+     * @throws IOException
+     *             Thrown if a triple writer is not available or the given
+     *             language does not support triple
+     */
+    public static <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Lang lang, Writer writer,
+            Configuration config) throws IOException {
+        if (lang == null)
+            throw new IOException("Cannot create a triple writer for an undefined language");
+
+        WriterFactory f = writerFactories.get(lang);
+        if (f == null)
+            throw new IOException("No factory registered for language " + lang.getName());
+        if (!f.canWriteTriples())
+            throw new IOException(lang.getName() + " does not support writing triples");
+
+        RecordWriter<TKey, TripleWritable> rwriter = f.<TKey> createTripleWriter(writer, config);
+        if (rwriter == null)
+            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples writer");
+        return rwriter;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
new file mode 100644
index 0000000..e1c98c7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.registry;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Interface for reader factories
+ * 
+ */
+public interface ReaderFactory {
+
+    /**
+     * Gets the primary language this factory produces readers for
+     * 
+     * @return Primary language
+     */
+    public abstract Lang getPrimaryLanguage();
+
+    /**
+     * Gets the alternative languages this factory can produce readers for
+     * 
+     * @return Alternative languages
+     */
+    public abstract Collection<Lang> getAlternativeLanguages();
+
+    /**
+     * Gets whether this factory can produce readers that are capable of reading
+     * quads
+     * 
+     * @return True if quads can be read, false if not
+     */
+    public abstract boolean canReadQuads();
+
+    /**
+     * Gets whether this factory can produce readers that are capable of reading
+     * triples
+     * 
+     * @return True if triples can be read, false if not
+     */
+    public abstract boolean canReadTriples();
+
+    /**
+     * Creates a quad reader
+     * 
+     * @return Quad reader
+     * @throws IOException
+     *             May be thrown if a quad reader cannot be created
+     */
+    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
+
+    /**
+     * Creates a triples reader
+     * 
+     * @return Triples reader
+     * @throws IOException
+     *             May be thrown if a triple reader cannot be created
+     */
+    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java
new file mode 100644
index 0000000..db5635f
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.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.registry;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Interface for writer factories
+ * 
+ */
+public interface WriterFactory {
+
+    /**
+     * Gets the primary language this factory produces writers for
+     * 
+     * @return Primary language
+     */
+    public abstract Lang getPrimaryLanguage();
+
+    /**
+     * Gets the alternative languages this factory can produce writers for
+     * 
+     * @return Alternative languages
+     */
+    public abstract Collection<Lang> getAlternativeLanguages();
+
+    /**
+     * Gets whether this factory can produce writers that are capable of reading
+     * quads
+     * 
+     * @return True if quads can be read, false if not
+     */
+    public abstract boolean canWriteQuads();
+
+    /**
+     * Gets whether this factory can produce writers that are capable of reading
+     * triples
+     * 
+     * @return True if triples can be read, false if not
+     */
+    public abstract boolean canWriteTriples();
+
+    /**
+     * Creates a quad writer
+     * 
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * 
+     * @return Quad writer
+     * @throws IOException
+     *             May be thrown if a quad writer cannot be created
+     */
+    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException;
+
+    /**
+     * Creates a triples writer
+     * 
+     * @param writer
+     *            Writer
+     * @param config
+     *            Configuration
+     * 
+     * @return Triples writer
+     * @throws IOException
+     *             May be thrown if a triple writer cannot be created
+     */
+    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
new file mode 100644
index 0000000..7fe15a9
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.registry.readers;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract reader factory for languages that only support quads
+ */
+public abstract class AbstractQuadsOnlyReaderFactory implements ReaderFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
+
+    public AbstractQuadsOnlyReaderFactory(Lang lang) {
+        this(lang, (Collection<Lang>)null);
+    }
+    
+    public AbstractQuadsOnlyReaderFactory(Lang lang, Lang...altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractQuadsOnlyReaderFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+    
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canReadQuads() {
+        return true;
+    }
+
+    @Override
+    public final boolean canReadTriples() {
+        return false;
+    }
+
+    @Override
+    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
+
+    @Override
+    public final RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        throw new IOException(this.lang.getName() + " does not support reading triples");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java
new file mode 100644
index 0000000..60e45af
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.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.registry.readers;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract reader factory for languages that support triples and quads
+ */
+public abstract class AbstractReaderFactory implements ReaderFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
+
+    public AbstractReaderFactory(Lang lang) {
+        this(lang, (Collection<Lang>)null);
+    }
+    
+    public AbstractReaderFactory(Lang lang, Lang...altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractReaderFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+    
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canReadQuads() {
+        return true;
+    }
+
+    @Override
+    public final boolean canReadTriples() {
+        return true;
+    }
+
+    @Override
+    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
+
+    @Override
+    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
new file mode 100644
index 0000000..7fb8131
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.registry.readers;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract reader factory for languages that only support triples
+ */
+public abstract class AbstractTriplesOnlyReaderFactory implements ReaderFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
+
+    public AbstractTriplesOnlyReaderFactory(Lang lang) {
+        this(lang, (Collection<Lang>)null);
+    }
+    
+    public AbstractTriplesOnlyReaderFactory(Lang lang, Lang...altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractTriplesOnlyReaderFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+    
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+    
+    @Override
+    public final boolean canReadQuads() {
+        return false;
+    }
+
+    @Override
+    public final boolean canReadTriples() {
+        return true;
+    }
+
+    @Override
+    public final RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        throw new IOException(this.lang.getName() + " does not support reading quads");
+    }
+
+    @Override
+    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java
new file mode 100644
index 0000000..6b064a4
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.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.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.JsonLDQuadReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.JsonLDTripleReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class JsonLDReaderFactory extends AbstractReaderFactory {
+    
+    public JsonLDReaderFactory() {
+        super(Lang.JSONLD);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new JsonLDQuadReader();
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new JsonLDTripleReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
new file mode 100644
index 0000000..2296296
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.nquads.WholeFileNQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class NQuadsReaderFactory extends AbstractQuadsOnlyReaderFactory {
+    
+    public NQuadsReaderFactory() {
+        super(Lang.NQUADS, Lang.NQ);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new WholeFileNQuadsReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
new file mode 100644
index 0000000..a98a1ae
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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.registry.readers;
+
+import java.io.IOException;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.WholeFileNTriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+public class NTriplesReaderFactory extends AbstractTriplesOnlyReaderFactory {
+
+    public NTriplesReaderFactory() {
+        super(Lang.NTRIPLES, Lang.NT);
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new WholeFileNTriplesReader();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
new file mode 100644
index 0000000..ccf5feb
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.rdfjson.RdfJsonReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class RdfJsonReaderFactory extends AbstractTriplesOnlyReaderFactory {
+
+    public RdfJsonReaderFactory() {
+        super(Lang.RDFJSON);
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new RdfJsonReader();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
new file mode 100644
index 0000000..1aa88d7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.rdfxml.RdfXmlReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+public class RdfXmlReaderFactory extends AbstractTriplesOnlyReaderFactory {
+
+    public RdfXmlReaderFactory() {
+        super(Lang.RDFXML);
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new RdfXmlReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java
new file mode 100644
index 0000000..25e8234
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.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.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftQuadReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftTripleReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.RDFLanguages;
+
+/**
+ *
+ */
+public class ThriftReaderFactory extends AbstractReaderFactory {
+    
+    public ThriftReaderFactory() {
+        super(RDFLanguages.THRIFT);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new ThriftQuadReader();
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new ThriftTripleReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
new file mode 100644
index 0000000..83ea818
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.trig.TriGReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class TriGReaderFactory extends AbstractQuadsOnlyReaderFactory {
+
+    public TriGReaderFactory() {
+        super(Lang.TRIG);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new TriGReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
new file mode 100644
index 0000000..cb8795c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.trix.TriXReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class TriXReaderFactory extends AbstractQuadsOnlyReaderFactory {
+
+    public TriXReaderFactory() {
+        super(Lang.TRIX);
+    }
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
+        return new TriXReader();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
new file mode 100644
index 0000000..7800376
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.registry.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.jena.hadoop.rdf.io.input.readers.turtle.TurtleReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+public class TurtleReaderFactory extends AbstractTriplesOnlyReaderFactory {
+    
+    public TurtleReaderFactory() {
+        super(Lang.TURTLE, Lang.TTL, Lang.N3);
+    }
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
+        return new TurtleReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
new file mode 100644
index 0000000..0cf137e
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
@@ -0,0 +1,86 @@
+/*
+ * 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.registry.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract writer factory for languages that only support quads
+ */
+public abstract class AbstractQuadsOnlyWriterFactory implements WriterFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang> emptyList());
+
+    public AbstractQuadsOnlyWriterFactory(Lang lang) {
+        this(lang, (Collection<Lang>) null);
+    }
+
+    public AbstractQuadsOnlyWriterFactory(Lang lang, Lang... altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractQuadsOnlyWriterFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canWriteQuads() {
+        return true;
+    }
+
+    @Override
+    public final boolean canWriteTriples() {
+        return false;
+    }
+
+    @Override
+    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException;
+
+    @Override
+    public final <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        throw new IOException(this.lang.getName() + " does not support writing triples");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
new file mode 100644
index 0000000..e45c3da
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
@@ -0,0 +1,85 @@
+/*
+ * 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.registry.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract writer factory for languages that only support triples
+ */
+public abstract class AbstractTriplesOnlyWriterFactory implements WriterFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang> emptyList());
+
+    public AbstractTriplesOnlyWriterFactory(Lang lang) {
+        this(lang, (Collection<Lang>) null);
+    }
+
+    public AbstractTriplesOnlyWriterFactory(Lang lang, Lang... altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractTriplesOnlyWriterFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canWriteQuads() {
+        return false;
+    }
+
+    @Override
+    public final boolean canWriteTriples() {
+        return true;
+    }
+
+    @Override
+    public final <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        throw new IOException(this.lang.getName() + " does not support writing quads");
+    }
+
+    @Override
+    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
new file mode 100644
index 0000000..669b9c4
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
@@ -0,0 +1,82 @@
+/*
+ * 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.registry.writers;
+
+import java.io.IOException;
+import java.io.Writer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Abstract writer factory for languages that support triples and quads
+ */
+public abstract class AbstractWriterFactory implements WriterFactory {
+
+    private Lang lang;
+    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
+
+    public AbstractWriterFactory(Lang lang) {
+        this(lang, (Collection<Lang>)null);
+    }
+    
+    public AbstractWriterFactory(Lang lang, Lang...altLangs) {
+        this(lang, Arrays.asList(altLangs));
+    }
+
+    public AbstractWriterFactory(Lang lang, Collection<Lang> altLangs) {
+        this.lang = lang;
+        if (altLangs != null)
+            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
+    }
+
+    @Override
+    public final Lang getPrimaryLanguage() {
+        return this.lang;
+    }
+    
+    @Override
+    public final Collection<Lang> getAlternativeLanguages() {
+        return this.alternateLangs;
+    }
+
+    @Override
+    public final boolean canWriteQuads() {
+        return true;
+    }
+
+    @Override
+    public final boolean canWriteTriples() {
+        return true;
+    }
+
+    @Override
+    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config) throws IOException;
+
+    @Override
+    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java
new file mode 100644
index 0000000..89e93ed
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.jsonld.JsonLDQuadWriter;
+import org.apache.jena.hadoop.rdf.io.output.writers.jsonld.JsonLDTripleWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class JsonLDWriterFactory extends AbstractWriterFactory {
+    
+    public JsonLDWriterFactory() {
+        super(Lang.JSONLD);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new JsonLDQuadWriter<>(writer);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new JsonLDTripleWriter<>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java
new file mode 100644
index 0000000..abbbd0f
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.nquads.NQuadsWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class NQuadsWriterFactory extends AbstractQuadsOnlyWriterFactory {
+    
+    public NQuadsWriterFactory() {
+        super(Lang.NQUADS, Lang.NQ);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new NQuadsWriter<TKey>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java
new file mode 100644
index 0000000..88c9551
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.ntriples.NTriplesWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class NTriplesWriterFactory extends AbstractTriplesOnlyWriterFactory {
+    
+    public NTriplesWriterFactory() {
+        super(Lang.NTRIPLES, Lang.NT);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new NTriplesWriter<TKey>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
new file mode 100644
index 0000000..8252422
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jena.hadoop.rdf.io.registry.writers;
+
+import java.io.IOException;
+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.rdfjson.RdfJsonWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class RdfJsonWriterFactory extends AbstractTriplesOnlyWriterFactory {
+
+    public RdfJsonWriterFactory() {
+        super(Lang.RDFJSON);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new RdfJsonWriter<TKey>(writer);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java
new file mode 100644
index 0000000..b4ac8e3
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.rdfxml.RdfXmlWriter;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ *
+ */
+public class RdfXmlWriterFactory extends AbstractTriplesOnlyWriterFactory {
+    
+    public RdfXmlWriterFactory() {
+        super(Lang.RDFXML);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new RdfXmlWriter<TKey>(writer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java
new file mode 100644
index 0000000..757472c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.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.registry.writers;
+
+import java.io.IOException;
+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.writers.StreamRdfQuadWriter;
+import org.apache.jena.hadoop.rdf.io.output.writers.StreamRdfTripleWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.RDFLanguages;
+import org.apache.jena.riot.thrift.StreamRDF2Thrift;
+
+/**
+ *
+ */
+public class ThriftWriterFactory extends AbstractWriterFactory {
+
+    public ThriftWriterFactory() {
+        super(RDFLanguages.THRIFT);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfQuadWriter<TKey>(new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")),
+                false), writer);
+    }
+
+    @Override
+    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
+            throws IOException {
+        return new StreamRdfTripleWriter<TKey>(new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")),
+                false), writer);
+    }
+
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitWithNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitWithNodesTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitWithNodesTests.java
new file mode 100644
index 0000000..80517b2
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitWithNodesTests.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
+ * implementations that work on Quads
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractQuadSplitWithNodesTests extends AbstractNodeTupleSplitWithNodesTests<Quad, QuadWritable> {
+
+    @Override
+    protected QuadWritable createValue(int i) {
+        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
+                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
+    }
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitToNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitToNodesTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitToNodesTests.java
new file mode 100644
index 0000000..7e497ab
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitToNodesTests.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.mapreduce.split;
+
+import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
+ * implementations that work on Triples
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractTripleSplitToNodesTests extends AbstractNodeTupleSplitToNodesTests<Triple, TripleWritable> {
+
+    @Override
+    protected TripleWritable createValue(int i) {
+        return new TripleWritable(
+                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitWithNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitWithNodesTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitWithNodesTests.java
new file mode 100644
index 0000000..babcad1
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitWithNodesTests.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.mapreduce.split;
+
+import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
+ * implementations that work on Triples
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractTripleSplitWithNodesTests extends AbstractNodeTupleSplitWithNodesTests<Triple, TripleWritable> {
+
+    @Override
+    protected TripleWritable createValue(int i) {
+        return new TripleWritable(
+                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+
+}

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

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

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

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

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapperTest.java
new file mode 100644
index 0000000..51b29cb
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapperTest.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.mapreduce.transform;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
+import org.apache.jena.hadoop.rdf.mapreduce.transform.QuadsToTriplesMapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.junit.Test;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Tests for the {@link QuadsToTriplesMapper}
+ * 
+ * 
+ * 
+ */
+public class QuadsToTriplesMapperTest extends AbstractMapperTests<LongWritable, QuadWritable, LongWritable, TripleWritable> {
+
+    @Override
+    protected Mapper<LongWritable, QuadWritable, LongWritable, TripleWritable> getInstance() {
+        return new QuadsToTriplesMapper<LongWritable>();
+    }
+
+    protected void generateData(MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver, int num) {
+        for (int i = 0; i < num; i++) {
+            Triple t = new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
+            Quad q = new Quad(Quad.defaultGraphNodeGenerated, t);
+            driver.addInput(new LongWritable(i), new QuadWritable(q));
+            driver.addOutput(new LongWritable(i), new TripleWritable(t));
+        }
+    }
+
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void quads_to_triples_mapper_01() throws IOException {
+        MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver = this.getMapDriver();
+
+        Triple t = new Triple(NodeFactory.createURI("http://s"), NodeFactory.createURI("http://p"),
+                NodeFactory.createLiteral("test"));
+        Quad q = new Quad(Quad.defaultGraphNodeGenerated, t);
+        driver.withInput(new Pair<LongWritable, QuadWritable>(new LongWritable(1), new QuadWritable(q))).withOutput(
+                new Pair<LongWritable, TripleWritable>(new LongWritable(1), new TripleWritable(t)));
+        driver.runTest();
+    }
+    
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void quads_to_triples_mapper_02() throws IOException {
+        MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver = this.getMapDriver();
+        this.generateData(driver, 100);
+        driver.runTest();
+    }
+    
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void quads_to_triples_mapper_03() throws IOException {
+        MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver = this.getMapDriver();
+        this.generateData(driver, 1000);
+        driver.runTest();
+    }
+    
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void quads_to_triples_mapper_04() throws IOException {
+        MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver = this.getMapDriver();
+        this.generateData(driver, 10000);
+        driver.runTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapperTest.java
new file mode 100644
index 0000000..bdf39f5
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapperTest.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.mapreduce.transform;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
+import org.apache.jena.hadoop.rdf.mapreduce.transform.TriplesToQuadsBySubjectMapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.junit.Test;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Tests for the {@link TriplesToQuadsBySubjectMapper}
+ * 
+ * 
+ * 
+ */
+public class TriplesToQuadsBySubjectMapperTest extends AbstractMapperTests<LongWritable, TripleWritable, LongWritable, QuadWritable> {
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, LongWritable, QuadWritable> getInstance() {
+        return new TriplesToQuadsBySubjectMapper<LongWritable>();
+    }
+
+    protected void generateData(MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver, int num) {
+        for (int i = 0; i < num; i++) {
+            Triple t = new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
+            Quad q = new Quad(t.getSubject(), t);
+            driver.addInput(new LongWritable(i), new TripleWritable(t));
+            driver.addOutput(new LongWritable(i), new QuadWritable(q));
+        }
+    }
+
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void triples_to_quads_mapper_01() throws IOException {
+        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
+
+        Triple t = new Triple(NodeFactory.createURI("http://s"), NodeFactory.createURI("http://p"),
+                NodeFactory.createLiteral("test"));
+        Quad q = new Quad(t.getSubject(), t);
+        driver.withInput(new Pair<LongWritable, TripleWritable>(new LongWritable(1), new TripleWritable(t))).withOutput(
+                new Pair<LongWritable, QuadWritable>(new LongWritable(1), new QuadWritable(q)));
+        driver.runTest();
+    }
+    
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void triples_to_quads_mapper_02() throws IOException {
+        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
+        this.generateData(driver, 100);
+        driver.runTest();
+    }
+    
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void triples_to_quads_mapper_03() throws IOException {
+        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
+        this.generateData(driver, 1000);
+        driver.runTest();
+    }
+    
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void triples_to_quads_mapper_04() throws IOException {
+        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
+        this.generateData(driver, 10000);
+        driver.runTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapperTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapperTest.java
new file mode 100644
index 0000000..b82f74b
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapperTest.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.mapreduce.transform;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
+import org.apache.jena.hadoop.rdf.mapreduce.transform.TriplesToQuadsConstantGraphMapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.junit.Test;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Tests for the {@link TriplesToQuadsConstantGraphMapper}
+ * 
+ * 
+ * 
+ */
+public class TriplesToQuadsConstantGraphMapperTest extends AbstractMapperTests<LongWritable, TripleWritable, LongWritable, QuadWritable> {
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, LongWritable, QuadWritable> getInstance() {
+        return new TriplesToQuadsConstantGraphMapper<LongWritable>();
+    }
+
+    protected void generateData(MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver, int num) {
+        for (int i = 0; i < num; i++) {
+            Triple t = new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
+                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
+            Quad q = new Quad(Quad.defaultGraphNodeGenerated, t);
+            driver.addInput(new LongWritable(i), new TripleWritable(t));
+            driver.addOutput(new LongWritable(i), new QuadWritable(q));
+        }
+    }
+
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void triples_to_quads_mapper_01() throws IOException {
+        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
+
+        Triple t = new Triple(NodeFactory.createURI("http://s"), NodeFactory.createURI("http://p"),
+                NodeFactory.createLiteral("test"));
+        Quad q = new Quad(Quad.defaultGraphNodeGenerated, t);
+        driver.withInput(new Pair<LongWritable, TripleWritable>(new LongWritable(1), new TripleWritable(t))).withOutput(
+                new Pair<LongWritable, QuadWritable>(new LongWritable(1), new QuadWritable(q)));
+        driver.runTest();
+    }
+    
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void triples_to_quads_mapper_02() throws IOException {
+        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
+        this.generateData(driver, 100);
+        driver.runTest();
+    }
+    
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void triples_to_quads_mapper_03() throws IOException {
+        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
+        this.generateData(driver, 1000);
+        driver.runTest();
+    }
+    
+    /**
+     * Tests quads to triples conversion
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void triples_to_quads_mapper_04() throws IOException {
+        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
+        this.generateData(driver, 10000);
+        driver.runTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-stats/hadoop-job.xml
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-stats/hadoop-job.xml b/jena-elephas/jena-elephas-stats/hadoop-job.xml
new file mode 100644
index 0000000..de72645
--- /dev/null
+++ b/jena-elephas/jena-elephas-stats/hadoop-job.xml
@@ -0,0 +1,46 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<assembly>
+  <id>hadoop-job</id>
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <dependencySets>
+    <dependencySet>
+      <unpack>false</unpack>
+      <scope>runtime</scope>
+      <outputDirectory>lib</outputDirectory>
+      <excludes>
+        <exclude>${groupId}:${artifactId}</exclude>
+      </excludes>
+    </dependencySet>
+    <dependencySet>
+      <unpack>true</unpack>
+      <includes>
+        <include>${groupId}:${artifactId}</include>
+      </includes>
+    </dependencySet>
+  </dependencySets>
+  <fileSets>
+    <fileSet>
+      <directory>${basedir}/target/test-classes</directory>
+      <outputDirectory>/</outputDirectory>
+    </fileSet>
+ </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-stats/pom.xml
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-stats/pom.xml b/jena-elephas/jena-elephas-stats/pom.xml
new file mode 100644
index 0000000..526d060
--- /dev/null
+++ b/jena-elephas/jena-elephas-stats/pom.xml
@@ -0,0 +1,103 @@
+<!--
+   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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<groupId>org.apache.jena</groupId>
+		<artifactId>jena-elephas</artifactId>
+		<version>0.9.0-SNAPSHOT</version>
+	</parent>
+	<artifactId>jena-elephas-stats</artifactId>
+	<name>Apache Jena - Elephas - Statistics Demo App</name>
+	<description>A demo application that can be run on Hadoop to produce a statistical analysis on arbitrary RDF inputs</description>
+
+	<dependencies>
+		<!-- Internal Project Dependencies -->
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-elephas-io</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-elephas-mapreduce</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<!-- CLI related Dependencies -->
+		<dependency>
+			<groupId>io.airlift</groupId>
+			<artifactId>airline</artifactId>
+			<version>0.6</version>
+		</dependency>
+
+		<!-- Hadoop Dependencies -->
+		<!-- Note these will be provided on the Hadoop cluster hence the provided 
+			scope -->
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test Dependencies -->
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-hadoop-rdf-mapreduce</artifactId>
+			<version>${project.version}</version>
+			<classifier>tests</classifier>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.mrunit</groupId>
+			<artifactId>mrunit</artifactId>
+			<scope>test</scope>
+			<classifier>hadoop2</classifier>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<!-- Assembly plugin is used to produce the runnable Hadoop JAR with all 
+				dependencies contained therein -->
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<configuration>
+					<descriptors>
+						<descriptor>hadoop-job.xml</descriptor>
+					</descriptors>
+				</configuration>
+				<executions>
+					<execution>
+						<id>make-assembly</id>
+						<phase>package</phase>
+						<goals>
+							<goal>single</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java b/jena-elephas/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
new file mode 100644
index 0000000..5f870ee
--- /dev/null
+++ b/jena-elephas/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
@@ -0,0 +1,405 @@
+/*
+ * 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.stats;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Help;
+import io.airlift.command.HelpOption;
+import io.airlift.command.Option;
+import io.airlift.command.OptionType;
+import io.airlift.command.ParseArgumentsMissingException;
+import io.airlift.command.ParseArgumentsUnexpectedException;
+import io.airlift.command.ParseException;
+import io.airlift.command.ParseOptionMissingException;
+import io.airlift.command.ParseOptionMissingValueException;
+import io.airlift.command.SingleCommand;
+import io.airlift.command.model.CommandMetadata;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import javax.inject.Inject;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.jena.hadoop.rdf.stats.jobs.JobFactory;
+
+
+/**
+ * Entry point for the Hadoop job, handles launching all the relevant Hadoop
+ * jobs
+ */
+@Command(name = "bin/hadoop jar PATH_TO_JAR com.yarcdata.urika.hadoop.rdf.stats.RdfStats", description = "A command which computes statistics on RDF data using Hadoop")
+public class RdfStats implements Tool {
+
+    static final String ANSI_RED = "\u001B[31m";
+    static final String ANSI_RESET = "\u001B[0m";
+
+    private static final String DATA_TYPE_TRIPLES = "triples", DATA_TYPE_QUADS = "quads", DATA_TYPE_MIXED = "mixed";
+
+    /**
+     * Help option
+     */
+    @Inject
+    public HelpOption helpOption;
+
+    /**
+     * Gets/Sets whether all available statistics will be calculated
+     */
+    @Option(name = { "-a", "--all" }, description = "Requests that all available statistics be calculated", type = OptionType.COMMAND)
+    public boolean all = false;
+
+    /**
+     * Gets/Sets whether node usage counts will be calculated
+     */
+    @Option(name = { "-n", "--node-count" }, description = "Requests that node usage counts be calculated", type = OptionType.COMMAND)
+    public boolean nodeCount = false;
+
+    /**
+     * Gets/Sets whether characteristic sets will be calculated
+     */
+    @Option(name = { "-c", "--characteristic-sets" }, description = "Requests that characteristic sets be calculated", type = OptionType.COMMAND)
+    public boolean characteristicSets = false;
+
+    /**
+     * Gets/Sets whether type counts will be calculated
+     */
+    @Option(name = { "-t", "--type-counts" }, description = "Requests that rdf:type usage counts be calculated", type = OptionType.COMMAND)
+    public boolean typeCount = false;
+
+    /**
+     * Gets/Sets whether data type counts will be calculated
+     */
+    @Option(name = { "-d", "--data-types" }, description = "Requests that literal data type usage counts be calculated", type = OptionType.COMMAND)
+    public boolean dataTypeCount = false;
+
+    /**
+     * Gets/Sets whether namespace counts will be calculated
+     */
+    @Option(name = { "--namespaces" }, description = "Requests that namespace usage counts be calculated", type = OptionType.COMMAND)
+    public boolean namespaceCount = false;
+
+    /**
+     * Gets/Sets the input data type used
+     */
+    @Option(name = { "--input-type" }, allowedValues = { DATA_TYPE_MIXED, DATA_TYPE_QUADS, DATA_TYPE_TRIPLES }, description = "Specifies whether the input data is a mixture of quads and triples, just quads or just triples.  Using the most specific data type will yield the most accurrate statistics")
+    public String inputType = DATA_TYPE_MIXED;
+
+    /**
+     * Gets/Sets the output path
+     */
+    @Option(name = { "-o", "--output" }, title = "OutputPath", description = "Sets the output path", arity = 1, required = true)
+    public String outputPath = null;
+
+    /**
+     * Gets/Sets the input path(s)
+     */
+    @Arguments(description = "Sets the input path(s)", title = "InputPath", required = true)
+    public List<String> inputPaths = new ArrayList<String>();
+
+    private Configuration config;
+
+    /**
+     * Entry point method
+     * 
+     * @param args
+     *            Arguments
+     * @throws Exception
+     */
+    public static void main(String[] args) throws Exception {
+        try {
+            // Run and exit with result code if no errors bubble up
+            // Note that the exit code may still be a error code
+            int res = ToolRunner.run(new Configuration(true), new RdfStats(), args);
+            System.exit(res);
+        } catch (Exception e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            e.printStackTrace(System.err);
+        } finally {
+            System.err.print(ANSI_RESET);
+        }
+        // If any errors bubble up exit with non-zero code
+        System.exit(1);
+    }
+
+    private static void showUsage() {
+        CommandMetadata metadata = SingleCommand.singleCommand(RdfStats.class).getCommandMetadata();
+        StringBuilder builder = new StringBuilder();
+        Help.help(metadata, builder);
+        System.err.print(ANSI_RESET);
+        System.err.println(builder.toString());
+        System.exit(1);
+    }
+
+    @Override
+    public void setConf(Configuration conf) {
+        this.config = conf;
+    }
+
+    @Override
+    public Configuration getConf() {
+        return this.config;
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+        try {
+            // Parse custom arguments
+            RdfStats cmd = SingleCommand.singleCommand(RdfStats.class).parse(args);
+
+            // Copy Hadoop configuration across
+            cmd.setConf(this.getConf());
+
+            // Show help if requested and exit with success
+            if (cmd.helpOption.showHelpIfRequested()) {
+                return 0;
+            }
+
+            // Run the command and exit with success
+            cmd.run();
+            return 0;
+
+        } catch (ParseOptionMissingException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+        } catch (ParseOptionMissingValueException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+        } catch (ParseArgumentsMissingException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+        } catch (ParseArgumentsUnexpectedException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+            // TODO Re-enable as and when we upgrade Airline
+            // } catch (ParseOptionIllegalValueException e) {
+            // System.err.println(ANSI_RED + e.getMessage());
+            // System.err.println();
+            // showUsage();
+        } catch (ParseException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            System.err.println();
+            showUsage();
+        } catch (UnsupportedOperationException e) {
+            System.err.println(ANSI_RED + e.getMessage());
+        } catch (Throwable e) {
+            System.err.println(ANSI_RED + e.getMessage());
+            e.printStackTrace(System.err);
+        } finally {
+            System.err.print(ANSI_RESET);
+        }
+        return 1;
+    }
+
+    private void run() throws Throwable {
+        if (!this.outputPath.endsWith("/")) {
+            this.outputPath += "/";
+        }
+
+        // If all statistics requested turn on all statistics
+        if (this.all) {
+            this.nodeCount = true;
+            this.characteristicSets = true;
+            this.typeCount = true;
+            this.dataTypeCount = true;
+            this.namespaceCount = true;
+        }
+
+        // How many statistics were requested?
+        int statsRequested = 0;
+        if (this.nodeCount)
+            statsRequested++;
+        if (this.characteristicSets)
+            statsRequested++;
+        if (this.typeCount)
+            statsRequested++;
+        if (this.dataTypeCount)
+            statsRequested++;
+        if (this.namespaceCount)
+            statsRequested++;
+
+        // Error if no statistics requested
+        if (statsRequested == 0) {
+            System.err
+                    .println("You did not request any statistics to be calculated, please use one/more of the relevant options to select the statistics to be computed");
+            return;
+        }
+        int statsComputed = 1;
+
+        // Compute statistics
+        if (this.nodeCount) {
+            Job job = this.selectNodeCountJob();
+            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
+        }
+        if (this.typeCount) {
+            Job[] jobs = this.selectTypeCountJobs();
+            statsComputed = this.computeStatistic(jobs, false, false, statsComputed, statsRequested);
+        }
+        if (this.dataTypeCount) {
+            Job job = this.selectDataTypeCountJob();
+            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
+        }
+        if (this.namespaceCount) {
+            Job job = this.selectNamespaceCountJob();
+            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
+        }
+        if (this.characteristicSets) {
+            Job[] jobs = this.selectCharacteristicSetJobs();
+            statsComputed = this.computeStatistic(jobs, false, false, statsComputed, statsRequested);
+        }
+    }
+
+    private int computeStatistic(Job job, int statsComputed, int statsRequested) throws Throwable {
+        System.out.println(String.format("Computing Statistic %d of %d requested", statsComputed, statsRequested));
+        this.runJob(job);
+        System.out.println(String.format("Computed Statistic %d of %d requested", statsComputed, statsRequested));
+        System.out.println();
+        return ++statsComputed;
+    }
+
+    private int computeStatistic(Job[] jobs, boolean continueOnFailure, boolean continueOnError, int statsComputed,
+            int statsRequested) {
+        System.out.println(String.format("Computing Statistic %d of %d requested", statsComputed, statsRequested));
+        this.runJobSequence(jobs, continueOnFailure, continueOnError);
+        System.out.println(String.format("Computed Statistic %d of %d requested", statsComputed, statsRequested));
+        System.out.println();
+        return ++statsComputed;
+    }
+
+    private boolean runJob(Job job) throws Throwable {
+        System.out.println("Submitting Job " + job.getJobName());
+        long start = System.nanoTime();
+        try {
+            job.submit();
+            if (job.monitorAndPrintJob()) {
+                System.out.println("Job " + job.getJobName() + " succeeded");
+                return true;
+            } else {
+                System.out.println("Job " + job.getJobName() + " failed");
+                return false;
+            }
+        } catch (Throwable e) {
+            System.out.println("Unexpected failure in Job " + job.getJobName());
+            throw e;
+        } finally {
+            long end = System.nanoTime();
+            System.out.println("Job " + job.getJobName() + " finished after "
+                    + String.format("%,d milliseconds", TimeUnit.NANOSECONDS.toMillis(end - start)));
+            System.out.println();
+        }
+    }
+
+    private void runJobSequence(Job[] jobs, boolean continueOnFailure, boolean continueOnError) {
+        for (int i = 0; i < jobs.length; i++) {
+            Job job = jobs[i];
+            try {
+                boolean success = this.runJob(job);
+                if (!success && !continueOnFailure)
+                    throw new IllegalStateException("Unable to complete job sequence because Job " + job.getJobName() + " failed");
+            } catch (IllegalStateException e) {
+                throw e;
+            } catch (Throwable e) {
+                if (!continueOnError)
+                    throw new IllegalStateException("Unable to complete job sequence because job " + job.getJobName()
+                            + " errorred", e);
+            }
+        }
+    }
+
+    private Job selectNodeCountJob() throws IOException {
+        String realOutputPath = outputPath + "node-counts/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadNodeCountJob(this.config, inputs, realOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleNodeCountJob(this.config, inputs, realOutputPath);
+        } else {
+            return JobFactory.getNodeCountJob(this.config, inputs, realOutputPath);
+        }
+    }
+
+    private Job selectDataTypeCountJob() throws IOException {
+        String realOutputPath = outputPath + "data-type-counts/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadDataTypeCountJob(this.config, inputs, realOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleDataTypeCountJob(this.config, inputs, realOutputPath);
+        } else {
+            return JobFactory.getDataTypeCountJob(this.config, inputs, realOutputPath);
+        }
+    }
+
+    private Job selectNamespaceCountJob() throws IOException {
+        String realOutputPath = outputPath + "namespace-counts/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadNamespaceCountJob(this.config, inputs, realOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleNamespaceCountJob(this.config, inputs, realOutputPath);
+        } else {
+            return JobFactory.getNamespaceCountJob(this.config, inputs, realOutputPath);
+        }
+    }
+
+    private Job[] selectCharacteristicSetJobs() throws IOException {
+        String intermediateOutputPath = outputPath + "characteristics/intermediate/";
+        String finalOutputPath = outputPath + "characteristics/final/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        } else {
+            return JobFactory.getCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        }
+    }
+
+    private Job[] selectTypeCountJobs() throws IOException {
+        String intermediateOutputPath = outputPath + "type-declarations/";
+        String finalOutputPath = outputPath + "type-counts/";
+        String[] inputs = new String[this.inputPaths.size()];
+        this.inputPaths.toArray(inputs);
+
+        if (DATA_TYPE_QUADS.equals(this.inputType)) {
+            return JobFactory.getQuadTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
+            return JobFactory.getTripleTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        } else {
+            return JobFactory.getTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java b/jena-elephas/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
new file mode 100644
index 0000000..55bb8af
--- /dev/null
+++ b/jena-elephas/jena-elephas-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
@@ -0,0 +1,757 @@
+/*
+ * 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.stats.jobs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.BZip2Codec;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.jena.hadoop.rdf.io.input.QuadsInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.TriplesInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.TriplesOrQuadsInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.nquads.NQuadsInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.nquads.NQuadsOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesNodeOutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesOutputFormat;
+import org.apache.jena.hadoop.rdf.mapreduce.KeyMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.characteristics.CharacteristicSetReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.characteristics.QuadCharacteristicSetGeneratingReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.characteristics.TripleCharacteristicSetGeneratingReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.datatypes.QuadDataTypeCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.datatypes.TripleDataTypeCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.namespaces.QuadNamespaceCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.namespaces.TripleNamespaceCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.positional.QuadObjectCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.count.positional.TripleObjectCountMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.QuadFilterByPredicateMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupBySubjectMapper;
+import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupBySubjectMapper;
+import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
+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 com.hp.hpl.jena.vocabulary.RDF;
+
+/**
+ * Factory that can produce {@link Job} instances for computing various RDF
+ * statistics
+ * 
+ * 
+ * 
+ */
+public class JobFactory {
+
+    /**
+     * Private constructor prevents instantiation
+     */
+    private JobFactory() {
+    }
+
+    /**
+     * Gets a job for computing node counts on RDF triple inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getTripleNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Node Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleNodeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing node counts on RDF quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getQuadNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Node Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadNodeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing node counts on RDF triple and/or quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Node Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadNodeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a sequence of jobs that can be used to compute characteristic sets
+     * for RDF triples
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Intermediate output path
+     * @param outputPath
+     *            Final output path
+     * @return Sequence of jobs
+     * @throws IOException
+     */
+    public static Job[] getTripleCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Characteristic Set (Generation)");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleGroupBySubjectMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(TripleWritable.class);
+        job.setReducerClass(TripleCharacteristicSetGeneratingReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(NullWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+        SequenceFileOutputFormat.setCompressOutput(job, true);
+        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
+        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
+
+        jobs[0] = job;
+
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Characteristic Set (Reduction)");
+
+        // Map/Reduce classes
+        job.setMapperClass(KeyMapper.class);
+        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
+        job.setMapOutputValueClass(CharacteristicSetWritable.class);
+        job.setReducerClass(CharacteristicSetReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(CharacteristicSetWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(SequenceFileInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+        return jobs;
+    }
+
+    /**
+     * Gets a sequence of jobs that can be used to compute characteristic sets
+     * for RDF quads
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Intermediate output path
+     * @param outputPath
+     *            Final output path
+     * @return Sequence of jobs
+     * @throws IOException
+     */
+    public static Job[] getQuadCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Characteristic Set (Generation)");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadGroupBySubjectMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(QuadWritable.class);
+        job.setReducerClass(QuadCharacteristicSetGeneratingReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(NullWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+        SequenceFileOutputFormat.setCompressOutput(job, true);
+        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
+        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
+
+        jobs[0] = job;
+
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Characteristic Set (Reduction)");
+
+        // Map/Reduce classes
+        job.setMapperClass(KeyMapper.class);
+        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
+        job.setMapOutputValueClass(CharacteristicSetWritable.class);
+        job.setReducerClass(CharacteristicSetReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(CharacteristicSetWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(SequenceFileInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+        return jobs;
+    }
+
+    /**
+     * Gets a sequence of jobs that can be used to compute characteristic sets
+     * for RDF triple and/or quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Intermediate output path
+     * @param outputPath
+     *            Final output path
+     * @return Sequence of jobs
+     * @throws IOException
+     */
+    public static Job[] getCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Characteristic Set (Generation)");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadGroupBySubjectMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(QuadWritable.class);
+        job.setReducerClass(QuadCharacteristicSetGeneratingReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(NullWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+        SequenceFileOutputFormat.setCompressOutput(job, true);
+        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
+        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
+
+        jobs[0] = job;
+
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Characteristic Set (Reduction)");
+
+        // Map/Reduce classes
+        job.setMapperClass(KeyMapper.class);
+        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
+        job.setMapOutputValueClass(CharacteristicSetWritable.class);
+        job.setReducerClass(CharacteristicSetReducer.class);
+        job.setOutputKeyClass(CharacteristicSetWritable.class);
+        job.setOutputValueClass(CharacteristicSetWritable.class);
+
+        // Input and Output
+        job.setInputFormatClass(SequenceFileInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+        return jobs;
+    }
+
+    /**
+     * Gets a job for computing type counts on RDF triple inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Path for intermediate output which will be all the type
+     *            declaration triples present in the inputs
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job[] getTripleTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Type Triples Extraction");
+
+        // Map/Reduce classes
+        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
+        job.setMapperClass(TripleFilterByPredicateUriMapper.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(TripleWritable.class);
+
+        // Input and Output Format
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(NTriplesOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+
+        jobs[0] = job;
+
+        // Object Node Usage count job
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleObjectCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(NTriplesInputFormat.class);
+        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
+                                                          // better if this was
+                                                          // intelligently
+                                                          // configured
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+
+        return jobs;
+    }
+
+    /**
+     * Gets a job for computing type counts on RDF quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Path for intermediate output which will be all the type
+     *            declaration quads present in the inputs
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job[] getQuadTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Type Quads Extraction");
+
+        // Map/Reduce classes
+        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
+        job.setMapperClass(QuadFilterByPredicateMapper.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(QuadWritable.class);
+
+        // Input and Output Format
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(NQuadsOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+
+        jobs[0] = job;
+
+        // Object Node Usage count job
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadObjectCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(NQuadsInputFormat.class);
+        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
+                                                          // better if this was
+                                                          // intelligently
+                                                          // configured
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+
+        return jobs;
+    }
+
+    /**
+     * Gets a job for computing type counts on RDF triple and/or quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param intermediateOutputPath
+     *            Path for intermediate output which will be all the type
+     *            declaration quads present in the inputs
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job[] getTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
+            String outputPath) throws IOException {
+        Job[] jobs = new Job[2];
+
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Type Extraction");
+
+        // Map/Reduce classes
+        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
+        job.setMapperClass(QuadFilterByPredicateMapper.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(QuadWritable.class);
+
+        // Input and Output Format
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(NQuadsOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
+
+        jobs[0] = job;
+
+        // Object Node Usage count job
+        job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadObjectCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(NQuadsInputFormat.class);
+        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
+                                                          // better if this was
+                                                          // intelligently
+                                                          // configured
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, intermediateOutputPath);
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        jobs[1] = job;
+
+        return jobs;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF triple inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getTripleDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Literal Data Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleDataTypeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getQuadDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Literal Data Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadDataTypeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF triple and/or
+     * quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Literal Data Type Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadDataTypeCountMapper.class);
+        job.setMapOutputKeyClass(NodeWritable.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(NodeCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF triple inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getTripleNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Triples Namespace Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(TripleNamespaceCountMapper.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(TextCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getQuadNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Quads Namespace Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadNamespaceCountMapper.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(TextCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(QuadsInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+
+    /**
+     * Gets a job for computing literal data type counts on RDF triple and/or
+     * quad inputs
+     * 
+     * @param config
+     *            Configuration
+     * @param inputPaths
+     *            Input paths
+     * @param outputPath
+     *            Output path
+     * @return Job
+     * @throws IOException
+     */
+    public static Job getNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
+        Job job = Job.getInstance(config);
+        job.setJarByClass(JobFactory.class);
+        job.setJobName("RDF Namespace Usage Count");
+
+        // Map/Reduce classes
+        job.setMapperClass(QuadNamespaceCountMapper.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(LongWritable.class);
+        job.setReducerClass(TextCountReducer.class);
+
+        // Input and Output
+        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
+        job.setOutputFormatClass(TextOutputFormat.class);
+        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
+        FileOutputFormat.setOutputPath(job, new Path(outputPath));
+
+        return job;
+    }
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
new file mode 100644
index 0000000..d50b6eb
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+
+/**
+ * Tests for BZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class BZippedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedWholeFileNTriplesInputTest() {
+        super(".nt.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
new file mode 100644
index 0000000..9780707
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
@@ -0,0 +1,37 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for deflated blocked NTriples input
+ * 
+ * 
+ * 
+ */
+public class DeflatedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedBlockedNTriplesInput() {
+        super(".nt.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
new file mode 100644
index 0000000..b6dd1e9
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+
+/**
+ * Tests for deflated NTriples input
+ * 
+ * 
+ * 
+ */
+public class DeflatedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedNTriplesInputTest() {
+        super(".nt.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
new file mode 100644
index 0000000..e1e57f4
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+
+/**
+ * Tests for deflated NTriples input
+ * 
+ * 
+ * 
+ */
+public class DeflatedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedWholeFileNTriplesInputTest() {
+        super(".nt.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
new file mode 100644
index 0000000..012fd58
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
@@ -0,0 +1,37 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped blocked NTriples input
+ * 
+ * 
+ * 
+ */
+public class GZippedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedBlockedNTriplesInput() {
+        super(".nt.gz", new GzipCodec());
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
new file mode 100644
index 0000000..b7acc08
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class GZippedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedWholeFileNTriplesInputTest() {
+        super(".nt.gz", new GzipCodec());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
new file mode 100644
index 0000000..3b2546d
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.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.rdfjson;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed RDF/JSON input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedRdfJsonInputFormatTests extends AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedRdfJsonInputFormatTests(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.RDFJSON;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new RdfJsonInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
new file mode 100644
index 0000000..6e58d4b
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfjson;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped RDF/JSON input
+ * 
+ * 
+ * 
+ */
+public class BZippedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedRdfJsonInputTest() {
+        super(".rj.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
new file mode 100644
index 0000000..8b7b044
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfjson;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated RDF/JSON input
+ * 
+ * 
+ * 
+ */
+public class DeflatedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedRdfJsonInputTest() {
+        super(".rj.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
new file mode 100644
index 0000000..66996dd
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfjson;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped RDF/JSON input
+ * 
+ * 
+ * 
+ */
+public class GZippedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedRdfJsonInputTest() {
+        super(".rj.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
new file mode 100644
index 0000000..a6d1e24
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rdfxml;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.rdfxml.RdfXmlInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed RDF/XML input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedRdfXmlInputFormatTests extends
+ AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedRdfXmlInputFormatTests(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.RDFXML;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new RdfXmlInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
new file mode 100644
index 0000000..30a6c39
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfxml;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped RDF/XML input
+ * 
+ * 
+ * 
+ */
+public class BZippedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedRdfXmlInputTest() {
+        super(".rdf.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
new file mode 100644
index 0000000..a3d747c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfxml;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated RDF/XML input
+ * 
+ * 
+ * 
+ */
+public class DeflatedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedRdfXmlInputTest() {
+        super(".rdf.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
new file mode 100644
index 0000000..748785c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.rdfxml;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped RDF/XML input
+ * 
+ * 
+ * 
+ */
+public class GZippedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedRdfXmlInputTest() {
+        super(".rdf.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
new file mode 100644
index 0000000..78affb2
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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.thrift;
+
+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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftQuadInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+/**
+ * Abstract compressed Thrift quad input tests
+ */
+public abstract class AbstractCompressedThriftQuadInputFormatTests extends
+        AbstractCompressedWholeFileQuadInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedThriftQuadInputFormatTests(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 RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new ThriftQuadInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
new file mode 100644
index 0000000..f837b89
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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.thrift;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftTripleInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+/**
+ * Abstract compressed Thrift triple input tests
+ */
+public abstract class AbstractCompressedThriftTripleInputFormatTests extends
+        AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedThriftTripleInputFormatTests(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 RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new ThriftTripleInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
new file mode 100644
index 0000000..320d278
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped Thrift input
+ */
+public class BZippedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedThriftQuadInputTest() {
+        super(".trdf.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
new file mode 100644
index 0000000..bb2d65b
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped Thrift input
+ */
+public class BZippedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedThriftTripleInputTest() {
+        super(".trdf.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
new file mode 100644
index 0000000..6872583
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated Thrift input
+ */
+public class DeflatedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedThriftQuadInputTest() {
+        super(".trdf.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
new file mode 100644
index 0000000..e76d2d7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated Thrift input
+ */
+public class DeflatedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedThriftTripleInputTest() {
+        super(".trdf.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
new file mode 100644
index 0000000..6590f22
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped Thrift input
+ */
+public class GZippedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedThriftQuadInputTest() {
+        super(".trdf.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
new file mode 100644
index 0000000..1ce74f4
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.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.thrift;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped Thrift input
+ */
+public class GZippedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedThriftTripleInputTest() {
+        super(".trdf.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
new file mode 100644
index 0000000..2975f29
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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.trig;
+
+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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.trig.TriGInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed TriG input tests
+ */
+public abstract class AbstractCompressedTriGInputFormatTests extends
+        AbstractCompressedWholeFileQuadInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedTriGInputFormatTests(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.TRIG;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new TriGInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
new file mode 100644
index 0000000..b2b3c33
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.trig;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped TriG input
+ * 
+ * 
+ * 
+ */
+public class BZippedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedTriGInputTest() {
+        super(".trig.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
new file mode 100644
index 0000000..c9579a9
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.trig;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated TriG input
+ * 
+ * 
+ * 
+ */
+public class DeflatedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedTriGInputTest() {
+        super(".trig.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
new file mode 100644
index 0000000..c3e4106
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.trig;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped TriG input
+ * 
+ * 
+ * 
+ */
+public class GZippedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedTriGInputTest() {
+        super(".trig.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
new file mode 100644
index 0000000..ad98e35
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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.trix;
+
+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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.trix.TriXInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed TriX input tests
+ */
+public abstract class AbstractCompressedTriXInputFormatTests extends
+        AbstractCompressedWholeFileQuadInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedTriXInputFormatTests(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.TRIX;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new TriXInputFormat();
+    }
+
+}
\ No newline at end of file

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

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

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

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
new file mode 100644
index 0000000..68d776a
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
@@ -0,0 +1,75 @@
+/*
+ * 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.turtle;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.turtle.TurtleInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed Turtle input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedTurtleInputFormatTests extends
+        AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedTurtleInputFormatTests(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.TURTLE;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new TurtleInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
new file mode 100644
index 0000000..724b847
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.turtle;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class BZippedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedTurtleInputTest() {
+        super(".nt.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
new file mode 100644
index 0000000..eb5ee03
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.turtle;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated NTriples input
+ * 
+ * 
+ * 
+ */
+public class DeflatedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedTurtleInputTest() {
+        super(".nt.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
new file mode 100644
index 0000000..817805c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.turtle;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class GZippedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedTurtleInputTest() {
+        super(".nt.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
new file mode 100644
index 0000000..92aac53
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.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.input.jsonld;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
new file mode 100644
index 0000000..63b6738
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.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.input.jsonld;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
new file mode 100644
index 0000000..6d1d02a
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.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.input.nquads;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractBlockedQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for blocked NTriples input
+ * 
+ * 
+ * 
+ */
+public class BlockedNQuadsInputTest extends AbstractBlockedQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nq";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new BlockedNQuadsInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
new file mode 100644
index 0000000..3823728
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.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.input.nquads;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractQuadsInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Tests for the NQuads input format
+ * 
+ *
+ */
+public class NQuadsInputTest extends AbstractQuadsInputFormatTests {
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new NQuadsInputFormat();
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nq";
+    }
+
+}


[13/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
deleted file mode 100644
index f8edcc4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.rdfxml;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for RDF/XML input
- * 
- * 
- * 
- */
-public class RdfXmlInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFXML;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".rdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new RdfXmlInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
deleted file mode 100644
index 8d79295..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-
-/**
- * Tests for JSON-LD input
- * 
- *
- */
-public class ThriftQuadInputTest extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new ThriftQuadInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
deleted file mode 100644
index 6b5e0b7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-
-/**
- * Tests for JSON-LD input
- * 
- *
- */
-public class ThriftTripleInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new ThriftTripleInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
deleted file mode 100644
index 1fad0dc..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.trig;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for TriG input
- * 
- *
- */
-public class TriGInputTest extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIG;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trig";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new TriGInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
deleted file mode 100644
index 4a3a66a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.trix;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for TriX input
- * 
- *
- */
-public class TriXInputTest extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIX;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trix";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new TriXInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
deleted file mode 100644
index e6211ba..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.turtle;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for turtle input format
- * 
- * 
- * 
- */
-public class TurtleInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected final String getFileExtension() {
-        return ".ttl";
-    }
-
-    @Override
-    protected final Lang getRdfLanguage() {
-        return Lang.TURTLE;
-    }
-    
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new TurtleInputFormat();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
deleted file mode 100644
index 9532d56..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
+++ /dev/null
@@ -1,701 +0,0 @@
-/*
- * 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.util;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Abstract tests for {@link TrackableInputStream} implementations
- * 
- * 
- * 
- */
-public abstract class AbstractTrackableInputStreamTests {
-
-    protected static final int KILO = 1024;
-    protected static final int BYTES_PER_KB = KILO;
-    protected static final int BYTES_PER_MB = BYTES_PER_KB * KILO;
-
-    /**
-     * Gets the instance to test using the given input as the stream to track
-     * 
-     * @param input
-     *            Input Stream
-     * @return Trackable Input Stream
-     */
-    protected abstract TrackableInputStream getInstance(InputStream input);
-
-    /**
-     * Generates an input stream containing the given number of bytes
-     * 
-     * @param length
-     *            Number of bytes
-     * @return Input stream
-     */
-    protected final InputStream generateData(int length) {
-        ByteArrayOutputStream output = new ByteArrayOutputStream(length);
-        byte b = (byte) 'b';
-        for (int i = 0; i < length; i++) {
-            output.write(b);
-        }
-        return new ByteArrayInputStream(output.toByteArray());
-    }
-
-    protected final void testSingleByteRead(int length) throws IOException {
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input);
-        long count = 0;
-        while (trackable.read() >= 0) {
-            count++;
-        }
-        Assert.assertEquals(length, count);
-        Assert.assertEquals(length, trackable.getBytesRead());
-        trackable.close();
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_single_01() throws IOException {
-        this.testSingleByteRead(0);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_single_02() throws IOException {
-        this.testSingleByteRead(100);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_single_03() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_single_04() throws IOException {
-        // 1 MB
-        this.testSingleByteRead(BYTES_PER_MB);
-    }
-
-    protected final void testMultiByteRead(int length, int bufferSize) throws IOException {
-        if (bufferSize < 1)
-            throw new IllegalArgumentException("bufferSize must be >= 1");
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input);
-        long count = 0;
-        byte[] buffer = new byte[bufferSize];
-        long read;
-        do {
-            read = trackable.read(buffer);
-            if (read > 0)
-                count += read;
-        } while (read >= 0);
-        Assert.assertEquals(length, count);
-        Assert.assertEquals(length, trackable.getBytesRead());
-        trackable.close();
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_01() throws IOException {
-        this.testMultiByteRead(0, 1);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_02() throws IOException {
-        this.testMultiByteRead(0, 16);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_03() throws IOException {
-        this.testMultiByteRead(0, BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_04() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_05() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_06() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_07() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_08() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_09() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB);
-    }
-
-    protected final void testMultiByteRead(int length, int bufferSize, int readSize) throws IOException {
-        if (bufferSize < 1)
-            throw new IllegalArgumentException("bufferSize must be >= 1");
-        if (readSize < 1 || readSize > bufferSize)
-            throw new IllegalArgumentException("readSize must be >= 1 and <= bufferSize");
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input);
-        long count = 0;
-        byte[] buffer = new byte[bufferSize];
-        long read;
-        do {
-            read = trackable.read(buffer, 0, readSize);
-            if (read > 0)
-                count += read;
-        } while (read >= 0);
-        Assert.assertEquals(length, count);
-        Assert.assertEquals(length, trackable.getBytesRead());
-        trackable.close();
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_01() throws IOException {
-        this.testMultiByteRead(0, 1, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_02() throws IOException {
-        this.testMultiByteRead(0, 16, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_03() throws IOException {
-        this.testMultiByteRead(0, 16, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_04() throws IOException {
-        this.testMultiByteRead(0, BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_05() throws IOException {
-        this.testMultiByteRead(0, BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_06() throws IOException {
-        this.testMultiByteRead(0, BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_07() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 1, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_08() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 16, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_09() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 16, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_10() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_11() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_12() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_13() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 1, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_14() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 16, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_15() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 16, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_16() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_17() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_18() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    protected final void testSkip(int length, long skipSize) throws IOException {
-        if (skipSize < 1)
-            throw new IllegalArgumentException("skipSize must be >= 1");
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input);
-        long count = 0;
-        long skipped;
-        do {
-            skipped = trackable.skip(skipSize);
-            if (skipped > 0)
-                count += skipped;
-        } while (skipped > 0);
-        Assert.assertEquals(length, count);
-        Assert.assertEquals(length, trackable.getBytesRead());
-        trackable.close();
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_01() throws IOException {
-        this.testSkip(0, 1);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_02() throws IOException {
-        this.testSkip(100, 1);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_03() throws IOException {
-        this.testSkip(100, 16);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_04() throws IOException {
-        this.testSkip(100, BYTES_PER_KB);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_05() throws IOException {
-        // 1KB
-        this.testSkip(BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_06() throws IOException {
-        // 1KB
-        this.testSkip(BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_07() throws IOException {
-        // 1KB
-        this.testSkip(BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_08() throws IOException {
-        // 1KB
-        this.testSkip(BYTES_PER_KB, BYTES_PER_MB);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_09() throws IOException {
-        // 1 MB
-        this.testSkip(BYTES_PER_MB, 1);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_10() throws IOException {
-        // 1 MB
-        this.testSkip(BYTES_PER_MB, 16);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_11() throws IOException {
-        // 1 MB
-        this.testSkip(BYTES_PER_MB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_12() throws IOException {
-        // 1 MB
-        this.testSkip(BYTES_PER_MB, BYTES_PER_MB);
-    }
-
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_01() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(-1, trackable.read());
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_02() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(0, trackable.read(new byte[0]));
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_03() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(-1, trackable.read(new byte[1]));
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_04() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(0, trackable.read(new byte[16], 0, 0));
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_05() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(-1, trackable.read(new byte[16], 0, 8));
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_06() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(0, trackable.skip(0));
-    }
-    
-    /**
-     * Tests exceptions are thrown trying to perform actions after closing the
-     * input
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_07() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(0, trackable.skip(1));
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
deleted file mode 100644
index f8819bc..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/*
- * 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.util;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.jena.hadoop.rdf.io.input.util.BlockInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for the {@link BlockInputStream}
- * 
- * 
- * 
- */
-public class BlockInputStreamTest extends AbstractTrackableInputStreamTests {
-
-    @Override
-    protected TrackableInputStream getInstance(InputStream input) {
-        return new BlockInputStream(input, Long.MAX_VALUE);
-    }
-
-    /**
-     * Gets an instance of a block input stream
-     * 
-     * @param input
-     *            Underlying input stream
-     * @param limit
-     *            Limit on bytes to read
-     * @return Block input stream
-     */
-    protected BlockInputStream getInstance(InputStream input, long limit) {
-        return new BlockInputStream(input, limit);
-    }
-    
-    protected final void testSingleByteRead(int length, long limit) throws IOException {
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input, limit);
-        long count = 0;
-        while (trackable.read() >= 0) {
-            count++;
-        }
-        int expected = (int) Math.min(length, limit);
-        Assert.assertEquals(expected, count);
-        Assert.assertEquals(expected, trackable.getBytesRead());
-        trackable.close();
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_01() throws IOException {
-        this.testSingleByteRead(0, 0);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_02() throws IOException {
-        this.testSingleByteRead(100, 0);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_03() throws IOException {
-        this.testSingleByteRead(100, 50);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_04() throws IOException {
-        this.testSingleByteRead(100, 100);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_05() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, 1);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_06() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, 100);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_07() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_KB / 2);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_08() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_KB);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_09() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_MB);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_10() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_MB * 10);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_11() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, 1);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_12() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, 100);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_13() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_KB);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_14() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB / 2);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_15() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_16() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB * 10);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
deleted file mode 100644
index f453aa5..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.util;
-
-import java.io.InputStream;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
-
-/**
- * Tests for the {@link TrackedInputStream}
- * 
- * 
- * 
- */
-public class TrackedInputStreamTest extends AbstractTrackableInputStreamTests {
-
-    @Override
-    protected TrackableInputStream getInstance(InputStream input) {
-        return new TrackedInputStream(input);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/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
deleted file mode 100644
index b5ea2d8..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * 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.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFDataMgr;
-import org.apache.jena.riot.lang.StreamRDFCounting;
-import org.apache.jena.riot.system.StreamRDFLib;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Abstract node tuple output format tests
- * 
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- * 
- */
-public abstract class AbstractNodeTupleOutputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> {
-
-    @SuppressWarnings("unused")
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleOutputFormatTests.class);
-
-    protected static final int EMPTY_SIZE = 0, SMALL_SIZE = 100, LARGE_SIZE = 10000, VERY_LARGE_SIZE = 100000;
-
-    /**
-     * Temporary folder for the tests
-     */
-    @Rule
-    public TemporaryFolder folder = new TemporaryFolder();
-
-    /**
-     * Prepares a fresh configuration
-     * 
-     * @return Configuration
-     */
-    protected Configuration prepareConfiguration() {
-        Configuration config = new Configuration(true);
-        // Nothing else to do
-        return config;
-    }
-
-    /**
-     * Gets the extra file extension to add to the filenames
-     * 
-     * @return File extension
-     */
-    protected abstract String getFileExtension();
-
-    /**
-     * Generates tuples to be output for testing
-     * 
-     * @param num
-     *            Number of tuples to generate
-     * @return Iterator of tuples
-     */
-    protected abstract Iterator<T> generateTuples(int num);
-
-    /**
-     * Counts tuples in the output file
-     * 
-     * @param f
-     *            Output file
-     * @return Tuple count
-     */
-    protected final long countTuples(File f) {
-        StreamRDFCounting counter = StreamRDFLib.count();
-        RDFDataMgr.parse(counter, f.getAbsolutePath(), this.getRdfLanguage(), null);
-        return counter.count();
-    }
-
-    /**
-     * Checks that tuples are as expected
-     * 
-     * @param f
-     *            File
-     * @param expected
-     *            Expected number of tuples
-     */
-    protected void checkTuples(File f, long expected) {
-        Assert.assertEquals(expected, this.countTuples(f));
-    }
-
-    /**
-     * Gets the RDF language of the produced output which is used to parse back
-     * in the output to validate the correct amount of output was produced
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-    /**
-     * Gets the output format to test
-     * 
-     * @return Output format
-     */
-    protected abstract OutputFormat<NullWritable, T> getOutputFormat();
-
-    /**
-     * Adds an output path to the job configuration
-     * 
-     * @param f
-     *            File
-     * @param config
-     *            Configuration
-     * @param job
-     *            Job
-     * @throws IOException
-     */
-    protected void addOutputPath(File f, Configuration config, Job job) throws IOException {
-        FileSystem fs = FileSystem.getLocal(config);
-        Path outputPath = fs.makeQualified(new Path(f.getAbsolutePath()));
-        FileOutputFormat.setOutputPath(job, outputPath);
-    }
-
-    protected File findOutputFile(File dir, JobContext context) throws FileNotFoundException, IOException {
-        Path outputPath = FileOutputFormat.getOutputPath(context);
-        RemoteIterator<LocatedFileStatus> files = outputPath.getFileSystem(context.getConfiguration()).listFiles(
-                outputPath, true);
-        while (files.hasNext()) {
-            LocatedFileStatus status = files.next();
-            if (status.isFile() && !status.getPath().getName().startsWith("_")) {
-                return new File(status.getPath().toUri());
-            }
-        }
-        return null;
-    }
-
-    /**
-     * Tests output
-     * 
-     * @param f
-     *            File to output to
-     * @param num
-     *            Number of tuples to output
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected final void testOutput(File f, int num) throws IOException, InterruptedException {
-        // Prepare configuration
-        Configuration config = this.prepareConfiguration();
-
-        // Set up fake job
-        OutputFormat<NullWritable, T> outputFormat = this.getOutputFormat();
-        Job job = Job.getInstance(config);
-        job.setOutputFormatClass(outputFormat.getClass());
-        this.addOutputPath(f, job.getConfiguration(), job);
-        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-        Assert.assertNotNull(FileOutputFormat.getOutputPath(context));
-
-        // Output the data
-        TaskAttemptID id = new TaskAttemptID("outputTest", 1, TaskType.MAP, 1, 1);
-        TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), id);
-        RecordWriter<NullWritable, T> writer = outputFormat.getRecordWriter(taskContext);
-        Iterator<T> tuples = this.generateTuples(num);
-        while (tuples.hasNext()) {
-            writer.write(NullWritable.get(), tuples.next());
-        }
-        writer.close(taskContext);
-
-        // Check output
-        File outputFile = this.findOutputFile(this.folder.getRoot(), context);
-        Assert.assertNotNull(outputFile);
-        this.checkTuples(outputFile, num);
-    }
-
-    /**
-     * Basic output tests
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void output_01() throws IOException, InterruptedException {
-        this.testOutput(this.folder.getRoot(), EMPTY_SIZE);
-    }
-
-    /**
-     * Basic output tests
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void output_02() throws IOException, InterruptedException {
-        this.testOutput(this.folder.getRoot(), SMALL_SIZE);
-    }
-
-    /**
-     * Basic output tests
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void output_03() throws IOException, InterruptedException {
-        this.testOutput(this.folder.getRoot(), LARGE_SIZE);
-    }
-
-    /**
-     * Basic output tests
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
deleted file mode 100644
index f1822f6..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for quad output formats
- * 
- * 
- * 
- */
-public abstract class AbstractQuadOutputFormatTests extends AbstractNodeTupleOutputFormatTests<Quad, QuadWritable> {
-
-    @Override
-    protected Iterator<QuadWritable> generateTuples(int num) {
-        List<QuadWritable> qs = new ArrayList<QuadWritable>();
-        for (int i = 0; i < num; i++) {
-            Quad q = new Quad(NodeFactory.createURI("http://example.org/graphs/" + i),
-                    NodeFactory.createURI("http://example.org/subjects/" + i),
-                    NodeFactory.createURI("http://example.org/predicate"), NodeFactory.createLiteral(Integer.toString(i),
-                            XSDDatatype.XSDinteger));
-            qs.add(new QuadWritable(q));
-        }
-        return qs.iterator();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
deleted file mode 100644
index 90eb531..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for triple output formats
- * 
- *
- */
-public abstract class AbstractTripleOutputFormatTests extends AbstractNodeTupleOutputFormatTests<Triple, TripleWritable> {
-
-    @Override
-    protected Iterator<TripleWritable> generateTuples(int num) {
-        List<TripleWritable> ts = new ArrayList<TripleWritable>();
-        for (int i = 0; i < num; i++) {
-            Triple t = new Triple(NodeFactory.createURI("http://example.org/subjects/" + i), NodeFactory.createURI("http://example.org/predicate"), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
-            ts.add(new TripleWritable(t));
-        }
-        return ts.iterator();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
deleted file mode 100644
index c6784a5..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * 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>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
deleted file mode 100644
index d157409..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * 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>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
deleted file mode 100644
index 1a7ffa4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for NQuads output format
- * 
- * 
- * 
- */
-public class NQuadsOutputTest extends AbstractQuadOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".nq";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new NQuadsOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
deleted file mode 100644
index ad9be56..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for NTriples output format
- * 
- * 
- * 
- */
-public class NTriplesOutputTest extends AbstractTripleOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".nt";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new NTriplesOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
deleted file mode 100644
index 833f89b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for RDF/JSON output
- * 
- * 
- * 
- */
-public class RdfJsonOutputTest extends AbstractTripleOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".rj";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFJSON;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new RdfJsonOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
deleted file mode 100644
index 40bc937..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for RDF/XML output
- * 
- * 
- * 
- */
-public class RdfXmlOutputTest extends AbstractTripleOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".rdf";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFXML;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new RdfXmlOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputTest.java
deleted file mode 100644
index 91509da..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputTest.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Tests for Thrift output format
- */
-public class ThriftQuadOutputTest extends AbstractQuadOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new ThriftQuadOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputTest.java
deleted file mode 100644
index cf50330..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputTest.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Tests for Thrift output format
- */
-public class ThriftTripleOutputTest extends AbstractTripleOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new ThriftTripleOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
deleted file mode 100644
index fd886a3..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.util.Arrays;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-
-/**
- * Tests for TriG output
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class BatchedTriGOutputTest extends AbstractQuadOutputFormatTests {
-
-    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-    static long $bs2 = 1000;
-    static long $bs3 = 100;
-    static long $bs4 = 1;
-
-    /**
-     * @return Test parameters
-     */
-    @Parameters
-    public static Collection<Object[]> data() {
-        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
-    }
-
-    private final long batchSize;
-
-    /**
-     * Creates new tests
-     * 
-     * @param batchSize
-     *            Batch size
-     */
-    public BatchedTriGOutputTest(long batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trig";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIG;
-    }
-
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
-        return config;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new BatchedTriGOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
deleted file mode 100644
index 9b2b669..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.util.Arrays;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-
-/**
- * Tests for Turtle output
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class StreamedTriGOutputTest extends AbstractQuadOutputFormatTests {
-
-    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-    static long $bs2 = 1000;
-    static long $bs3 = 100;
-    static long $bs4 = 1;
-
-    /**
-     * @return Test parameters
-     */
-    @Parameters
-    public static Collection<Object[]> data() {
-        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
-    }
-
-    private final long batchSize;
-
-    /**
-     * Creates new tests
-     * 
-     * @param batchSize
-     *            Batch size
-     */
-    public StreamedTriGOutputTest(long batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trig";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIG;
-    }
-    
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
-        return config;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new TriGOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
deleted file mode 100644
index c9b3a26..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * 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.File;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.RDFDataMgr;
-import org.junit.Assert;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ResIterator;
-import com.hp.hpl.jena.rdf.model.Resource;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Tests for TriG output with blank nodes
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class TriGBlankNodeOutputTests extends StreamedTriGOutputTest {
-
-	static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-	static long $bs2 = 1000;
-	static long $bs3 = 100;
-	static long $bs4 = 1;
-
-	/**
-	 * @return Test parameters
-	 */
-	@Parameters
-	public static Collection<Object[]> data() {
-		return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 },
-				{ $bs4 } });
-	}
-
-	/**
-	 * Creates new tests
-	 * 
-	 * @param batchSize
-	 *            Batch size
-	 */
-	public TriGBlankNodeOutputTests(long batchSize) {
-		super(batchSize);
-	}
-
-	@Override
-	protected Iterator<QuadWritable> generateTuples(int num) {
-		List<QuadWritable> qs = new ArrayList<QuadWritable>();
-		Node subject = NodeFactory.createAnon();
-		for (int i = 0; i < num; i++) {
-			Quad t = new Quad(
-					NodeFactory.createURI("http://example.org/graphs/" + i),
-					subject,
-					NodeFactory.createURI("http://example.org/predicate"),
-					NodeFactory.createLiteral(Integer.toString(i),
-							XSDDatatype.XSDinteger));
-			qs.add(new QuadWritable(t));
-		}
-		return qs.iterator();
-	}
-
-	@Override
-	protected void checkTuples(File f, long expected) {
-		super.checkTuples(f, expected);
-
-		Model m = RDFDataMgr.loadModel("file://" + f.getAbsolutePath(),
-				this.getRdfLanguage());
-		ResIterator iter = m.listSubjects();
-		Set<Node> subjects = new HashSet<Node>();
-		while (iter.hasNext()) {
-			Resource res = iter.next();
-			Assert.assertTrue(res.isAnon());
-			subjects.add(res.asNode());
-		}
-		// Should only be one subject unless the data was empty in which case
-		// there will be zero subjects
-		Assert.assertEquals(expected == 0 ? 0 : 1, subjects.size());
-	}
-
-	@Override
-	protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-		return new TriGOutputFormat<NullWritable>();
-	}
-
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
new file mode 100644
index 0000000..3d9dd00
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
@@ -0,0 +1,136 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
+import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
+import org.apache.jena.riot.thrift.ThriftConvert;
+import org.apache.jena.riot.thrift.wire.RDF_Quad;
+import org.apache.thrift.TException;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A writable quad
+ */
+public class QuadWritable extends AbstractNodeTupleWritable<Quad> {
+
+    static {
+        WritableComparator.define(QuadWritable.class, new SimpleBinaryComparator());
+    }
+
+    private RDF_Quad quad = new RDF_Quad();
+
+    /**
+     * Creates a new empty instance
+     */
+    public QuadWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance with the given value
+     * 
+     * @param q
+     *            Quad
+     */
+    public QuadWritable(Quad q) {
+        super(q);
+    }
+
+    /**
+     * Creates a new instance from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static QuadWritable read(DataInput input) throws IOException {
+        QuadWritable q = new QuadWritable();
+        q.readFields(input);
+        return q;
+    }
+
+    @Override
+    public void set(Quad tuple) {
+        super.set(tuple);
+        this.quad.clear();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.quad.clear();
+        int tripleLength = input.readInt();
+        byte[] buffer = new byte[tripleLength];
+        input.readFully(buffer);
+        try {
+            ThriftConverter.fromBytes(buffer, this.quad);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        this.setInternal(new Quad(ThriftConvert.convert(this.quad.getG()), ThriftConvert.convert(this.quad.getS()),
+                ThriftConvert.convert(this.quad.getP()), ThriftConvert.convert(this.quad.getO())));
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        if (this.get() == null)
+            throw new IOException(
+                    "Null quads cannot be written using this class, consider using NodeTupleWritable instead");
+
+        // May not have yet prepared the Thrift triple
+        if (!this.quad.isSetS()) {
+            Quad tuple = this.get();
+            this.quad.setG(ThriftConvert.convert(tuple.getGraph(), false));
+            this.quad.setS(ThriftConvert.convert(tuple.getSubject(), false));
+            this.quad.setP(ThriftConvert.convert(tuple.getPredicate(), false));
+            this.quad.setO(ThriftConvert.convert(tuple.getObject(), false));
+        }
+
+        byte[] buffer;
+        try {
+            buffer = ThriftConverter.toBytes(this.quad);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        output.writeInt(buffer.length);
+        output.write(buffer);
+    }
+
+    @Override
+    protected Quad createTuple(Node[] ns) {
+        if (ns.length != 4)
+            throw new IllegalArgumentException(String.format(
+                    "Incorrect number of nodes to form a quad - got %d but expected 4", ns.length));
+        return new Quad(ns[0], ns[1], ns[2], ns[3]);
+    }
+
+    @Override
+    protected Node[] createNodes(Quad tuple) {
+        return new Node[] { tuple.getGraph(), tuple.getSubject(), tuple.getPredicate(), tuple.getObject() };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
new file mode 100644
index 0000000..a17052b
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
@@ -0,0 +1,138 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
+import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
+import org.apache.jena.riot.thrift.ThriftConvert;
+import org.apache.jena.riot.thrift.wire.RDF_Triple;
+import org.apache.thrift.TException;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A writable triple
+ * 
+ * 
+ * 
+ */
+public class TripleWritable extends AbstractNodeTupleWritable<Triple> {
+    
+    static {
+        WritableComparator.define(TripleWritable.class, new SimpleBinaryComparator());
+    }
+
+    private RDF_Triple triple = new RDF_Triple();
+
+    /**
+     * Creates a new instance using the default NTriples node formatter
+     */
+    public TripleWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance with a given value that uses a specific node
+     * formatter
+     * 
+     * @param t
+     *            Triple
+     */
+    public TripleWritable(Triple t) {
+        super(t);
+    }
+
+    /**
+     * Creates a new instance from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static TripleWritable read(DataInput input) throws IOException {
+        TripleWritable t = new TripleWritable();
+        t.readFields(input);
+        return t;
+    }
+
+    @Override
+    public void set(Triple tuple) {
+        super.set(tuple);
+        this.triple.clear();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.triple.clear();
+        int tripleLength = input.readInt();
+        byte[] buffer = new byte[tripleLength];
+        input.readFully(buffer);
+        try {
+            ThriftConverter.fromBytes(buffer, this.triple);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        this.setInternal(new Triple(ThriftConvert.convert(this.triple.getS()),
+                ThriftConvert.convert(this.triple.getP()), ThriftConvert.convert(this.triple.getO())));
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        if (this.get() == null)
+            throw new IOException(
+                    "Null triples cannot be written using this class, consider using NodeTupleWritable instead");
+        
+        // May not have yet prepared the Thrift triple
+        if (!this.triple.isSetS()) {
+            Triple tuple = this.get();
+            this.triple.setS(ThriftConvert.convert(tuple.getSubject(), false));
+            this.triple.setP(ThriftConvert.convert(tuple.getPredicate(), false));
+            this.triple.setO(ThriftConvert.convert(tuple.getObject(), false));
+        }
+
+        byte[] buffer;
+        try {
+            buffer = ThriftConverter.toBytes(this.triple);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        output.writeInt(buffer.length);
+        output.write(buffer);
+    }
+
+    @Override
+    protected Triple createTuple(Node[] ns) {
+        if (ns.length != 3)
+            throw new IllegalArgumentException(String.format(
+                    "Incorrect number of nodes to form a triple - got %d but expected 3", ns.length));
+        return new Triple(ns[0], ns[1], ns[2]);
+    }
+
+    @Override
+    protected Node[] createNodes(Triple tuple) {
+        return new Node[] { tuple.getSubject(), tuple.getPredicate(), tuple.getObject() };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
new file mode 100644
index 0000000..6c46714
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.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.types.comparators;
+
+import org.apache.hadoop.io.WritableComparator;
+
+/**
+ * A general purpose comparator that may be used with any types which can be
+ * compared directly on their binary encodings
+ */
+public class SimpleBinaryComparator extends WritableComparator {
+
+    @Override
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+        return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
new file mode 100644
index 0000000..0675afc
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
@@ -0,0 +1,147 @@
+/*
+ * 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.types.converters;
+
+import java.io.ByteArrayOutputStream;
+
+import org.apache.jena.riot.thrift.wire.RDF_Quad;
+import org.apache.jena.riot.thrift.wire.RDF_Term;
+import org.apache.jena.riot.thrift.wire.RDF_Triple;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TIOStreamTransport;
+import org.apache.thrift.transport.TMemoryInputTransport;
+import org.apache.thrift.transport.TTransport;
+
+/**
+ * Helper for converting between the binary representation of Nodes, Triples and
+ * Quads and their Jena API equivalents
+ * 
+ */
+public class ThriftConverter {
+
+    private static ThreadLocal<TMemoryInputTransport> inputTransports = new ThreadLocal<>();
+    private static ThreadLocal<TProtocol> inputProtocols = new ThreadLocal<>();
+
+    private static ThreadLocal<ByteArrayOutputStream> outputStreams = new ThreadLocal<>();
+    private static ThreadLocal<TTransport> outputTransports = new ThreadLocal<>();
+    private static ThreadLocal<TProtocol> outputProtocols = new ThreadLocal<>();
+
+    private static TMemoryInputTransport getInputTransport() {
+        TMemoryInputTransport transport = inputTransports.get();
+        if (transport != null)
+            return transport;
+
+        transport = new TMemoryInputTransport();
+        inputTransports.set(transport);
+        return transport;
+    }
+
+    private static TProtocol getInputProtocol() {
+        TProtocol protocol = inputProtocols.get();
+        if (protocol != null)
+            return protocol;
+
+        protocol = new TCompactProtocol(getInputTransport());
+        inputProtocols.set(protocol);
+        return protocol;
+    }
+
+    private static ByteArrayOutputStream getOutputStream() {
+        ByteArrayOutputStream output = outputStreams.get();
+        if (output != null)
+            return output;
+
+        output = new ByteArrayOutputStream();
+        outputStreams.set(output);
+        return output;
+    }
+
+    private static TTransport getOutputTransport() {
+        TTransport transport = outputTransports.get();
+        if (transport != null)
+            return transport;
+
+        transport = new TIOStreamTransport(getOutputStream());
+        outputTransports.set(transport);
+        return transport;
+    }
+
+    private static TProtocol getOutputProtocol() {
+        TProtocol protocol = outputProtocols.get();
+        if (protocol != null)
+            return protocol;
+
+        protocol = new TCompactProtocol(getOutputTransport());
+        outputProtocols.set(protocol);
+        return protocol;
+    }
+
+    public static byte[] toBytes(RDF_Term term) throws TException {
+        ByteArrayOutputStream output = getOutputStream();
+        output.reset();
+
+        TProtocol protocol = getOutputProtocol();
+        term.write(protocol);
+
+        return output.toByteArray();
+    }
+
+    public static void fromBytes(byte[] bs, RDF_Term term) throws TException {
+        TMemoryInputTransport transport = getInputTransport();
+        transport.reset(bs);
+        TProtocol protocol = getInputProtocol();
+        term.read(protocol);
+    }
+
+    public static void fromBytes(byte[] buffer, RDF_Triple triple) throws TException {
+        TMemoryInputTransport transport = getInputTransport();
+        transport.reset(buffer);
+        TProtocol protocol = getInputProtocol();
+        triple.read(protocol);
+    }
+
+    public static byte[] toBytes(RDF_Triple triple) throws TException {
+        ByteArrayOutputStream output = getOutputStream();
+        output.reset();
+
+        TProtocol protocol = getOutputProtocol();
+        triple.write(protocol);
+
+        return output.toByteArray();
+    }
+
+    public static void fromBytes(byte[] buffer, RDF_Quad quad) throws TException {
+        TMemoryInputTransport transport = getInputTransport();
+        transport.reset(buffer);
+        TProtocol protocol = getInputProtocol();
+        quad.read(protocol);
+    }
+
+    public static byte[] toBytes(RDF_Quad quad) throws TException {
+        ByteArrayOutputStream output = getOutputStream();
+        output.reset();
+
+        TProtocol protocol = getOutputProtocol();
+        quad.write(protocol);
+
+        return output.toByteArray();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java b/jena-elephas/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
new file mode 100644
index 0000000..7214b14
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
@@ -0,0 +1,210 @@
+/*
+ * 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.types;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
+import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * Tests for {@link CharacteristicWritable} and
+ * {@link CharacteristicSetWritable}
+ * 
+ * 
+ * 
+ */
+public class CharacteristicTests {
+
+    /**
+     * Checks whether a writable round trips successfully
+     * 
+     * @param cw
+     *            Characteristic writable
+     * @throws IOException
+     */
+    private void checkRoundTrip(CharacteristicWritable cw) throws IOException {
+        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        DataOutputStream output = new DataOutputStream(outputStream);
+        cw.write(output);
+
+        ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray());
+        DataInputStream input = new DataInputStream(inputStream);
+        CharacteristicWritable actual = CharacteristicWritable.read(input);
+        Assert.assertEquals(cw, actual);
+    }
+
+    /**
+     * Tests characteristic round tripping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_writable_01() throws IOException {
+        Node n = NodeFactory.createURI("http://example.org");
+        CharacteristicWritable expected = new CharacteristicWritable(n);
+        Assert.assertEquals(1, expected.getCount().get());
+
+        this.checkRoundTrip(expected);
+    }
+
+    /**
+     * Tests characteristic properties
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_writable_02() throws IOException {
+        Node n = NodeFactory.createURI("http://example.org");
+        CharacteristicWritable cw1 = new CharacteristicWritable(n);
+        CharacteristicWritable cw2 = new CharacteristicWritable(n, 100);
+        this.checkRoundTrip(cw1);
+        this.checkRoundTrip(cw2);
+
+        // Should still be equal since equality is only on the node not the
+        // count
+        Assert.assertEquals(cw1, cw2);
+    }
+
+    /**
+     * Tests characteristic properties
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_writable_03() throws IOException {
+        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
+        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
+        this.checkRoundTrip(cw1);
+        this.checkRoundTrip(cw2);
+
+        // Should not be equal as different nodes
+        Assert.assertNotEquals(cw1, cw2);
+    }
+
+    /**
+     * Checks that a writable round trips
+     * 
+     * @param set
+     *            Characteristic set
+     * @throws IOException
+     */
+    private void checkRoundTrip(CharacteristicSetWritable set) throws IOException {
+        // Test round trip
+        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        DataOutputStream output = new DataOutputStream(outputStream);
+        set.write(output);
+
+        ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray());
+        DataInputStream input = new DataInputStream(inputStream);
+        CharacteristicSetWritable actual = CharacteristicSetWritable.read(input);
+        Assert.assertEquals(set, actual);
+    }
+
+    /**
+     * Checks a characteristic set
+     * 
+     * @param set
+     *            Set
+     * @param expectedItems
+     *            Expected number of characteristics
+     * @param expectedCounts
+     *            Expected counts for characteristics
+     */
+    protected final void checkCharacteristicSet(CharacteristicSetWritable set, int expectedItems, long[] expectedCounts) {
+        Assert.assertEquals(expectedItems, set.size());
+        Assert.assertEquals(expectedItems, expectedCounts.length);
+        Iterator<CharacteristicWritable> iter = set.getCharacteristics();
+        int i = 0;
+        while (iter.hasNext()) {
+            CharacteristicWritable cw = iter.next();
+            Assert.assertEquals(expectedCounts[i], cw.getCount().get());
+            i++;
+        }
+    }
+
+    /**
+     * Tests characteristic sets
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_writable_01() throws IOException {
+        CharacteristicSetWritable set = new CharacteristicSetWritable();
+
+        // Add some characteristics
+        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
+        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
+        set.add(cw1);
+        set.add(cw2);
+        this.checkCharacteristicSet(set, 2, new long[] { 1, 1 });
+        this.checkRoundTrip(set);
+    }
+
+    /**
+     * Tests characteristic sets
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_writable_02() throws IOException {
+        CharacteristicSetWritable set = new CharacteristicSetWritable();
+
+        // Add some characteristics
+        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
+        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"), 2);
+        set.add(cw1);
+        set.add(cw2);
+        this.checkCharacteristicSet(set, 1, new long[] { 3 });
+        this.checkRoundTrip(set);
+    }
+    
+    /**
+     * Tests characteristic sets
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_writable_03() throws IOException {
+        CharacteristicSetWritable set1 = new CharacteristicSetWritable();
+        CharacteristicSetWritable set2 = new CharacteristicSetWritable();
+
+        // Add some characteristics
+        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
+        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
+        set1.add(cw1);
+        set2.add(cw2);
+        this.checkCharacteristicSet(set1, 1, new long[] { 1 });
+        this.checkCharacteristicSet(set2, 1, new long[] { 1 });
+        this.checkRoundTrip(set1);
+        this.checkRoundTrip(set2);
+        
+        Assert.assertNotEquals(set1, set2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java b/jena-elephas/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
new file mode 100644
index 0000000..a70dfb0
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
@@ -0,0 +1,406 @@
+/*
+ * 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.types;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableComparable;
+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.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Tests for the various RDF types defined by the
+ * {@link org.apache.jena.hadoop.rdf.types} package
+ * 
+ * 
+ * 
+ */
+public class RdfTypesTest {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RdfTypesTest.class);
+
+    private ByteArrayOutputStream outputStream;
+    private ByteArrayInputStream inputStream;
+
+    /**
+     * Prepare for output
+     * 
+     * @return Data output
+     */
+    private DataOutput prepareOutput() {
+        this.outputStream = new ByteArrayOutputStream();
+        return new DataOutputStream(this.outputStream);
+    }
+
+    /**
+     * Prepare for input from the previously written output
+     * 
+     * @return Data Input
+     */
+    private DataInput prepareInput() {
+        this.inputStream = new ByteArrayInputStream(this.outputStream.toByteArray());
+        return new DataInputStream(this.inputStream);
+    }
+
+    /**
+     * Prepare for input from the given data
+     * 
+     * @param data
+     *            Data
+     * @return Data Input
+     */
+    @SuppressWarnings("unused")
+    private DataInput prepareInput(byte[] data) {
+        this.inputStream = new ByteArrayInputStream(data);
+        return new DataInputStream(this.inputStream);
+    }
+
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    private <T extends WritableComparable> void testWriteRead(T writable, T expected) throws IOException, InstantiationException, IllegalAccessException,
+            ClassNotFoundException {
+        // Write out data
+        DataOutput output = this.prepareOutput();
+        writable.write(output);
+
+        // Read back in data
+        DataInput input = this.prepareInput();
+        T actual = (T) Class.forName(writable.getClass().getName()).newInstance();
+        actual.readFields(input);
+
+        LOG.info("Original = " + writable.toString());
+        LOG.info("Round Tripped = " + actual.toString());
+
+        // Check equivalent
+        Assert.assertEquals(0, expected.compareTo(actual));
+    }
+    
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_null() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = null;
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+    
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    @Ignore
+    public void node_writable_variable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createVariable("x");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+    
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    @Ignore
+    public void node_writable_variable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createVariable("really-log-variable-name-asddsfr4545egfdgdfgfdgdtgvdg-dfgfdgdfgdfgdfg4-dfvdfgdfgdfgfdgfdgdfgdfgfdg");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_uri_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createURI("http://example.org");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_uri_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createURI("http://user:password@example.org/some/path?key=value#id");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("simple");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("language", "en", null);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_03() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("string", XSDDatatype.XSDstring);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_04() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("1234", XSDDatatype.XSDinteger);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_05() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("123.4", XSDDatatype.XSDdecimal);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_06() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("12.3e4", XSDDatatype.XSDdouble);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_07() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("true", XSDDatatype.XSDboolean);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_bnode_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createAnon();
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_bnode_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createAnon();
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+        NodeWritable nw2 = new NodeWritable(n);
+        testWriteRead(nw2, nw2);
+
+        Assert.assertEquals(0, nw.compareTo(nw2));
+    }
+
+    /**
+     * Basic triple writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void triple_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Triple t = new Triple(NodeFactory.createURI("http://example"), NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral("value"));
+        TripleWritable tw = new TripleWritable(t);
+        testWriteRead(tw, tw);
+    }
+    
+    /**
+     * Basic triple writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void triple_writable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Triple t = new Triple(NodeFactory.createAnon(), NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral("value"));
+        TripleWritable tw = new TripleWritable(t);
+        testWriteRead(tw, tw);
+    }
+
+    /**
+     * Basic quad writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void quad_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Quad q = new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://example"), NodeFactory.createURI("http://predicate"),
+                NodeFactory.createLiteral("value"));
+        QuadWritable qw = new QuadWritable(q);
+        testWriteRead(qw, qw);
+    }
+    
+    /**
+     * Basic quad writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void quad_writable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Quad q = new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createAnon(), NodeFactory.createURI("http://predicate"),
+                NodeFactory.createLiteral("value"));
+        QuadWritable qw = new QuadWritable(q);
+        testWriteRead(qw, qw);
+    }
+
+    /**
+     * Basic tuple writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void tuple_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Tuple<Node> t = Tuple.createTuple(NodeFactory.createURI("http://one"), NodeFactory.createURI("http://two"), NodeFactory.createLiteral("value"),
+                NodeFactory.createLiteral("foo"), NodeFactory.createURI("http://three"));
+        NodeTupleWritable tw = new NodeTupleWritable(t);
+        testWriteRead(tw, tw);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/pom.xml
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/pom.xml b/jena-elephas/jena-elephas-io/pom.xml
new file mode 100644
index 0000000..e5134d0
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/pom.xml
@@ -0,0 +1,67 @@
+<!--
+   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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.jena</groupId>
+    <artifactId>jena-elephas</artifactId>
+    <version>0.9.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>jena-elephas-io</artifactId>
+  <name>Apache Jena - Elephas - I/O</name>
+  <description>RDF Input/Output formats library for Hadoop</description>
+
+	<!-- Note that versions are managed by parent POMs -->
+  <dependencies>
+		<!-- Internal Project Dependencies -->
+    <dependency>
+      <groupId>org.apache.jena</groupId>
+      <artifactId>jena-elephas-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+	<!-- Hadoop Dependencies -->
+	<!-- Note these will be provided on the Hadoop cluster hence the provided 
+		scope -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+	<!-- Jena dependencies -->
+    <dependency>
+      <groupId>org.apache.jena</groupId>
+      <artifactId>jena-arq</artifactId>
+    </dependency>
+
+	<!-- Test Dependencies -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java
new file mode 100644
index 0000000..5c1b41c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.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;
+
+/**
+ * Hadoop IO related constants
+ * 
+ * 
+ * 
+ */
+public class HadoopIOConstants {
+
+    /**
+     * Private constructor prevents instantiation
+     */
+    private HadoopIOConstants() {
+    }
+
+    /**
+     * Map Reduce configuration setting for max line length
+     */
+    public static final String MAX_LINE_LENGTH = "mapreduce.input.linerecordreader.line.maxlength";
+
+    /**
+     * Run ID
+     */
+    public static final String RUN_ID = "runId";
+    
+    /**
+     * Compression codecs to use
+     */
+    public static final String IO_COMPRESSION_CODECS = "io.compression.codecs";
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
new file mode 100644
index 0000000..27c2bb2
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+/**
+ * RDF IO related constants
+ * 
+ * 
+ * 
+ */
+public class RdfIOConstants {
+
+    /**
+     * Private constructor prevents instantiation
+     */
+    private RdfIOConstants() {
+    }
+
+    /**
+     * Configuration key used to set whether bad tuples are ignored. This is the
+     * default behaviour, when explicitly set to {@code false} bad tuples will
+     * result in {@link IOException} being thrown by the relevant record
+     * readers.
+     */
+    public static final String INPUT_IGNORE_BAD_TUPLES = "rdf.io.input.ignore-bad-tuples";
+
+    /**
+     * Configuration key used to set the batch size used for RDF output formats
+     * that take a batched writing approach. Default value is given by the
+     * constant {@link #DEFAULT_OUTPUT_BATCH_SIZE}.
+     */
+    public static final String OUTPUT_BATCH_SIZE = "rdf.io.output.batch-size";
+
+    /**
+     * Default batch size for batched output formats
+     */
+    public static final long DEFAULT_OUTPUT_BATCH_SIZE = 10000;
+
+    /**
+     * Configuration key used to control behaviour with regards to how blank
+     * nodes are handled.
+     * <p>
+     * The default behaviour is that blank nodes are file scoped which is what
+     * the RDF specifications require.
+     * </p>
+     * <p>
+     * However in the case of a multi-stage pipeline this behaviour can cause
+     * blank nodes to diverge over several jobs and introduce spurious blank
+     * nodes over time. This is described in <a
+     * href="https://issues.apache.org/jira/browse/JENA-820">JENA-820</a> and
+     * enabling this flag for jobs in your pipeline allow you to work around
+     * this problem.
+     * </p>
+     * <h3>Warning</h3> You should only enable this flag for jobs that take in
+     * RDF output originating from previous jobs since our normal blank node
+     * allocation policy ensures that blank nodes will be file scoped and unique
+     * over all files (barring unfortunate hasing collisions). If you enable
+     * this for jobs that take in RDF originating from other sources you may
+     * incorrectly conflate blank nodes that are supposed to distinct and
+     * separate nodes.
+     */
+    public static final String GLOBAL_BNODE_IDENTITY = "rdf.io.input.bnodes.global-identity";
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
new file mode 100644
index 0000000..1fcb030
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
@@ -0,0 +1,70 @@
+/*
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract line based input format that reuses the machinery from
+ * {@link NLineInputFormat} to calculate the splits
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Value type
+ */
+public abstract class AbstractNLineFileInputFormat<TKey, TValue> extends FileInputFormat<TKey, TValue> {
+    
+    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractNLineFileInputFormat.class);
+
+    /**
+     * Logically splits the set of input files for the job, splits N lines of
+     * the input as one split.
+     * 
+     * @see FileInputFormat#getSplits(JobContext)
+     */
+    public final List<InputSplit> getSplits(JobContext job) throws IOException {
+        boolean debug = LOGGER.isDebugEnabled();
+        if (debug && FileInputFormat.getInputDirRecursive(job)) {
+            LOGGER.debug("Recursive searching for input data is enabled");
+        }
+        
+        List<InputSplit> splits = new ArrayList<InputSplit>();
+        int numLinesPerSplit = NLineInputFormat.getNumLinesPerSplit(job);
+        for (FileStatus status : listStatus(job)) {
+            if (debug) {
+                LOGGER.debug("Determining how to split input file/directory {}", status.getPath());
+            }
+            splits.addAll(NLineInputFormat.getSplitsForFile(status, job.getConfiguration(), numLinesPerSplit));
+        }
+        return splits;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
new file mode 100644
index 0000000..e561cdb
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+/**
+ * Abstract implementation of a while file input format where each file is a
+ * single split
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Value type
+ */
+public abstract class AbstractWholeFileInputFormat<TKey, TValue> extends FileInputFormat<TKey, TValue> {
+
+    @Override
+    protected final boolean isSplitable(JobContext context, Path filename) {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
new file mode 100644
index 0000000..b8fdbd5
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.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.QuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * RDF input format that can handle any RDF quads format that ARQ supports
+ * selecting the format to use for each file based upon the file extension
+ * 
+ * 
+ * 
+ */
+public class QuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new QuadsReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
new file mode 100644
index 0000000..03f394a
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.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.TriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+/**
+ * RDF input format that can handle any RDF triples format that ARQ supports
+ * selecting the format to use for each file based upon the file extension
+ */
+public class TriplesInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TriplesReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
new file mode 100644
index 0000000..bfd643e
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.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.TriplesOrQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * RDF input format that can handle any RDF triple/quads format that ARQ
+ * supports selecting the format to use for each file based upon the file
+ * extension. Triples are converted into quads in the default graph.
+ * 
+ * 
+ * 
+ */
+public class TriplesOrQuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TriplesOrQuadsReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
new file mode 100644
index 0000000..2464946
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.jsonld;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
new file mode 100644
index 0000000..0e08a4b
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.jsonld;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.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/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
new file mode 100644
index 0000000..6829c4d
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
@@ -0,0 +1,53 @@
+/*
+ * 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.nquads;
+
+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.AbstractNLineFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.nquads.BlockedNQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * NTriples input format where files are processed as blocks of lines rather
+ * than in a line based manner as with the {@link NQuadsInputFormat} or as
+ * whole files with the {@link WholeFileNQuadsInputFormat}
+ * <p>
+ * This provides a compromise between the higher parser setup of creating more
+ * parsers and the benefit of being able to split input files over multiple
+ * mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class BlockedNQuadsInputFormat extends AbstractNLineFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new BlockedNQuadsReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
new file mode 100644
index 0000000..802fbea
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.nquads;
+
+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.AbstractNLineFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.nquads.NQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * NQuads input format
+ * 
+ * 
+ * 
+ */
+public class NQuadsInputFormat extends AbstractNLineFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit arg0, TaskAttemptContext arg1)
+            throws IOException, InterruptedException {
+        return new NQuadsReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java
new file mode 100644
index 0000000..128d079
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.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.input.nquads;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.nquads.WholeFileNQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * NQuads input format where files are processed as complete files rather than
+ * in a line based manner as with the {@link NQuadsInputFormat}
+ * <p>
+ * This has the advantage of less parser setup overhead but the disadvantage
+ * that the input cannot be split over multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class WholeFileNQuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new WholeFileNQuadsReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
new file mode 100644
index 0000000..292167b
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
@@ -0,0 +1,53 @@
+/*
+ * 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.ntriples;
+
+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.AbstractNLineFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.BlockedNTriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * NTriples input format where files are processed as blocks of lines rather
+ * than in a line based manner as with the {@link NTriplesInputFormat} or as
+ * whole files with the {@link WholeFileNTriplesInputFormat}
+ * <p>
+ * This provides a compromise between the higher parser setup of creating more
+ * parsers and the benefit of being able to split input files over multiple
+ * mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class BlockedNTriplesInputFormat extends AbstractNLineFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new BlockedNTriplesReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
new file mode 100644
index 0000000..1694c87
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.ntriples;
+
+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.AbstractNLineFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.NTriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * NTriples input format
+ * 
+ * 
+ * 
+ */
+public class NTriplesInputFormat extends AbstractNLineFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit inputSplit, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new NTriplesReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java
new file mode 100644
index 0000000..31c1252
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.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.input.ntriples;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.WholeFileNTriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * NTriples input format where files are processed as complete files rather than
+ * in a line based manner as with the {@link NTriplesInputFormat}
+ * <p>
+ * This has the advantage of less parser setup overhead but the disadvantage
+ * that the input cannot be split over multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class WholeFileNTriplesInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new WholeFileNTriplesReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
new file mode 100644
index 0000000..e5a7940
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.rdfjson;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.rdfjson.RdfJsonReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * RDF/JSON input format
+ * 
+ * 
+ * 
+ */
+public class RdfJsonInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new RdfJsonReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
new file mode 100644
index 0000000..4deb925
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.rdfxml;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.rdfxml.RdfXmlReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * RDF/XML input format
+ * 
+ * 
+ * 
+ */
+public class RdfXmlInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new RdfXmlReader();
+    }
+
+}


[12/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputTest.java
deleted file mode 100644
index 9b6e307..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests for TriX output format
- */
-public class TriXOutputTest extends AbstractQuadOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".trix";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIX;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new TriXOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
deleted file mode 100644
index a6c4d70..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.util.Arrays;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-
-/**
- * Tests for Turtle output
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class BatchedTurtleOutputTest extends AbstractTripleOutputFormatTests {
-
-    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-    static long $bs2 = 1000;
-    static long $bs3 = 100;
-    static long $bs4 = 1;
-
-    /**
-     * @return Test parameters
-     */
-    @Parameters
-    public static Collection<Object[]> data() {
-        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
-    }
-
-    private final long batchSize;
-
-    /**
-     * Creates new tests
-     * 
-     * @param batchSize
-     *            Batch size
-     */
-    public BatchedTurtleOutputTest(long batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".ttl";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TURTLE;
-    }
-    
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
-        return config;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new BatchedTurtleOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
deleted file mode 100644
index d8843d3..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.util.Arrays;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-
-/**
- * Tests for Turtle output
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class StreamedTurtleOutputTest extends AbstractTripleOutputFormatTests {
-
-    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-    static long $bs2 = 1000;
-    static long $bs3 = 100;
-    static long $bs4 = 1;
-
-    /**
-     * @return Test parameters
-     */
-    @Parameters
-    public static Collection<Object[]> data() {
-        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
-    }
-
-    private final long batchSize;
-
-    /**
-     * Creates new tests
-     * 
-     * @param batchSize
-     *            Batch size
-     */
-    public StreamedTurtleOutputTest(long batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".ttl";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TURTLE;
-    }
-    
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
-        return config;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new TurtleOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
deleted file mode 100644
index 8dcae4e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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.File;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.RDFDataMgr;
-import org.junit.Assert;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ResIterator;
-import com.hp.hpl.jena.rdf.model.Resource;
-
-/**
- * Tests for Turtle output with blank nodes
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class TurtleBlankNodeOutputTests extends StreamedTurtleOutputTest {
-
-	static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-	static long $bs2 = 1000;
-	static long $bs3 = 100;
-	static long $bs4 = 1;
-
-	/**
-	 * @return Test parameters
-	 */
-	@Parameters
-	public static Collection<Object[]> data() {
-		return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 },
-				{ $bs4 } });
-	}
-
-	/**
-	 * Creates new tests
-	 * 
-	 * @param batchSize
-	 *            Batch size
-	 */
-	public TurtleBlankNodeOutputTests(long batchSize) {
-		super(batchSize);
-	}
-
-	@Override
-	protected Iterator<TripleWritable> generateTuples(int num) {
-		List<TripleWritable> ts = new ArrayList<TripleWritable>();
-		Node subject = NodeFactory.createAnon();
-		for (int i = 0; i < num; i++) {
-			Triple t = new Triple(subject,
-					NodeFactory.createURI("http://example.org/predicate"),
-					NodeFactory.createLiteral(Integer.toString(i),
-							XSDDatatype.XSDinteger));
-			ts.add(new TripleWritable(t));
-		}
-		return ts.iterator();
-	}
-
-	@Override
-	protected void checkTuples(File f, long expected) {
-		super.checkTuples(f, expected);
-
-		Model m = RDFDataMgr.loadModel("file://" + f.getAbsolutePath(),
-				this.getRdfLanguage());
-		ResIterator iter = m.listSubjects();
-		Set<Node> subjects = new HashSet<Node>();
-		while (iter.hasNext()) {
-			Resource res = iter.next();
-			Assert.assertTrue(res.isAnon());
-			subjects.add(res.asNode());
-		}
-		// Should only be one subject unless the data was empty in which case
-		// there will be zero subjects
-		Assert.assertEquals(expected == 0 ? 0 : 1, subjects.size());
-	}
-
-	@Override
-	protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-		return new TurtleOutputFormat<NullWritable>();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
deleted file mode 100644
index 2eae232..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * 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.registry;
-
-import java.io.IOException;
-import java.io.StringWriter;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for the {@link HadoopRdfIORegistry}
- */
-public class TestHadoopRdfIORegistry {
-
-    private void testLang(Lang lang, boolean triples, boolean quads, boolean writesSupported) {
-        Assert.assertEquals(triples, HadoopRdfIORegistry.hasTriplesReader(lang));
-        Assert.assertEquals(quads, HadoopRdfIORegistry.hasQuadReader(lang));
-
-        // Some formats may be asymmetric
-        if (writesSupported) {
-            Assert.assertEquals(triples, HadoopRdfIORegistry.hasTriplesWriter(lang));
-            Assert.assertEquals(quads, HadoopRdfIORegistry.hasQuadWriter(lang));
-        } else {
-            Assert.assertFalse(HadoopRdfIORegistry.hasTriplesWriter(lang));
-            Assert.assertFalse(HadoopRdfIORegistry.hasQuadWriter(lang));
-        }
-
-        if (triples) {
-            // Check that triples are supported
-            RecordReader<LongWritable, TripleWritable> tripleReader;
-            try {
-                tripleReader = HadoopRdfIORegistry.createTripleReader(lang);
-                Assert.assertNotNull(tripleReader);
-            } catch (IOException e) {
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " can read triples but fails to produce a triple reader when asked: " + e.getMessage());
-            }
-
-            if (writesSupported) {
-                RecordWriter<NullWritable, TripleWritable> tripleWriter;
-                try {
-                    tripleWriter = HadoopRdfIORegistry.createTripleWriter(lang, new StringWriter(), new Configuration(
-                            false));
-                    Assert.assertNotNull(tripleWriter);
-                } catch (IOException e) {
-                    Assert.fail("Registry indicates that " + lang.getName()
-                            + " can write triples but fails to produce a triple writer when asked: " + e.getMessage());
-                }
-            }
-        } else {
-            // Check that triples are not supported
-            try {
-                HadoopRdfIORegistry.createTripleReader(lang);
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " cannot read triples but produced a triple reader when asked (error was expected)");
-            } catch (IOException e) {
-                // This is expected
-            }
-            try {
-                HadoopRdfIORegistry.createTripleWriter(lang, new StringWriter(), new Configuration(false));
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " cannot write triples but produced a triple write when asked (error was expected)");
-            } catch (IOException e) {
-                // This is expected
-            }
-        }
-
-        if (quads) {
-            // Check that quads are supported
-            RecordReader<LongWritable, QuadWritable> quadReader;
-            try {
-                quadReader = HadoopRdfIORegistry.createQuadReader(lang);
-                Assert.assertNotNull(quadReader);
-            } catch (IOException e) {
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " can read quads but fails to produce a quad reader when asked: " + e.getMessage());
-            }
-
-            if (writesSupported) {
-                RecordWriter<NullWritable, QuadWritable> quadWriter;
-                try {
-                    quadWriter = HadoopRdfIORegistry.createQuadWriter(lang, new StringWriter(),
-                            new Configuration(false));
-                    Assert.assertNotNull(quadWriter);
-                } catch (IOException e) {
-                    Assert.fail("Registry indicates that " + lang.getName()
-                            + " can write quads but fails to produce a triple writer when asked: " + e.getMessage());
-                }
-            }
-        } else {
-            try {
-                HadoopRdfIORegistry.createQuadReader(lang);
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " cannot read quads but produced a quad reader when asked (error was expected)");
-            } catch (IOException e) {
-                // This is expected
-            }
-            try {
-                HadoopRdfIORegistry.createQuadWriter(lang, new StringWriter(), new Configuration(false));
-                Assert.fail("Registry indicates that " + lang.getName()
-                        + " cannot write quads but produced a quad writer when asked (error was expected)");
-            } catch (IOException e) {
-                // This is expected
-            }
-        }
-    }
-
-    @Test
-    public void json_ld_registered() {
-        testLang(Lang.JSONLD, true, true, true);
-    }
-
-    @Test
-    public void nquads_registered() {
-        testLang(Lang.NQUADS, false, true, true);
-        testLang(Lang.NQ, false, true, true);
-    }
-
-    @Test
-    public void ntriples_registered() {
-        testLang(Lang.NTRIPLES, true, false, true);
-        testLang(Lang.NT, true, false, true);
-    }
-
-    @Test
-    public void rdf_json_registered() {
-        testLang(Lang.RDFJSON, true, false, true);
-    }
-
-    @Test
-    public void rdf_xml_registered() {
-        testLang(Lang.RDFXML, true, false, true);
-    }
-
-    @Test
-    public void rdf_thrift_registered() {
-        testLang(RDFLanguages.THRIFT, true, true, true);
-    }
-
-    @Test
-    public void trig_registered() {
-        testLang(Lang.TRIG, false, true, true);
-    }
-
-    @Test
-    public void trix_registered() {
-        testLang(Lang.TRIX, false, true, true);
-    }
-
-    @Test
-    public void turtle_registered() {
-        testLang(Lang.TURTLE, true, false, true);
-        testLang(Lang.TTL, true, false, true);
-        testLang(Lang.N3, true, false, true);
-    }
-
-    @Test
-    public void unregistered() {
-        testLang(Lang.RDFNULL, false, false, true);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-stats/hadoop-job.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-stats/hadoop-job.xml b/jena-hadoop-rdf/hadoop-rdf-stats/hadoop-job.xml
deleted file mode 100644
index de72645..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-stats/hadoop-job.xml
+++ /dev/null
@@ -1,46 +0,0 @@
-<!--
-   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.
--->
-
-<assembly>
-  <id>hadoop-job</id>
-  <formats>
-    <format>jar</format>
-  </formats>
-  <includeBaseDirectory>false</includeBaseDirectory>
-  <dependencySets>
-    <dependencySet>
-      <unpack>false</unpack>
-      <scope>runtime</scope>
-      <outputDirectory>lib</outputDirectory>
-      <excludes>
-        <exclude>${groupId}:${artifactId}</exclude>
-      </excludes>
-    </dependencySet>
-    <dependencySet>
-      <unpack>true</unpack>
-      <includes>
-        <include>${groupId}:${artifactId}</include>
-      </includes>
-    </dependencySet>
-  </dependencySets>
-  <fileSets>
-    <fileSet>
-      <directory>${basedir}/target/test-classes</directory>
-      <outputDirectory>/</outputDirectory>
-    </fileSet>
- </fileSets>
-</assembly>

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-stats/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-stats/pom.xml b/jena-hadoop-rdf/hadoop-rdf-stats/pom.xml
deleted file mode 100644
index bf69fa6..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-stats/pom.xml
+++ /dev/null
@@ -1,103 +0,0 @@
-<!--
-   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.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<groupId>org.apache.jena</groupId>
-		<artifactId>jena-hadoop-rdf</artifactId>
-		<version>0.9.0-SNAPSHOT</version>
-	</parent>
-	<artifactId>jena-hadoop-rdf-stats</artifactId>
-	<name>Apache Jena - RDF Tools for Hadoop - Statistics Demo App</name>
-	<description>A demo application that can be run on Hadoop to produce a statistical analysis on arbitrary RDF inputs</description>
-
-	<dependencies>
-		<!-- Internal Project Dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-hadoop-rdf-io</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-hadoop-rdf-mapreduce</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<!-- CLI related Dependencies -->
-		<dependency>
-			<groupId>io.airlift</groupId>
-			<artifactId>airline</artifactId>
-			<version>0.6</version>
-		</dependency>
-
-		<!-- Hadoop Dependencies -->
-		<!-- Note these will be provided on the Hadoop cluster hence the provided 
-			scope -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-mapreduce-client-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Test Dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-hadoop-rdf-mapreduce</artifactId>
-			<version>${project.version}</version>
-			<classifier>tests</classifier>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.mrunit</groupId>
-			<artifactId>mrunit</artifactId>
-			<scope>test</scope>
-			<classifier>hadoop2</classifier>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<plugins>
-			<!-- Assembly plugin is used to produce the runnable Hadoop JAR with all 
-				dependencies contained therein -->
-			<plugin>
-				<artifactId>maven-assembly-plugin</artifactId>
-				<configuration>
-					<descriptors>
-						<descriptor>hadoop-job.xml</descriptor>
-					</descriptors>
-				</configuration>
-				<executions>
-					<execution>
-						<id>make-assembly</id>
-						<phase>package</phase>
-						<goals>
-							<goal>single</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java b/jena-hadoop-rdf/hadoop-rdf-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
deleted file mode 100644
index 5f870ee..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/RdfStats.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/*
- * 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.stats;
-
-import io.airlift.command.Arguments;
-import io.airlift.command.Command;
-import io.airlift.command.Help;
-import io.airlift.command.HelpOption;
-import io.airlift.command.Option;
-import io.airlift.command.OptionType;
-import io.airlift.command.ParseArgumentsMissingException;
-import io.airlift.command.ParseArgumentsUnexpectedException;
-import io.airlift.command.ParseException;
-import io.airlift.command.ParseOptionMissingException;
-import io.airlift.command.ParseOptionMissingValueException;
-import io.airlift.command.SingleCommand;
-import io.airlift.command.model.CommandMetadata;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import javax.inject.Inject;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.jena.hadoop.rdf.stats.jobs.JobFactory;
-
-
-/**
- * Entry point for the Hadoop job, handles launching all the relevant Hadoop
- * jobs
- */
-@Command(name = "bin/hadoop jar PATH_TO_JAR com.yarcdata.urika.hadoop.rdf.stats.RdfStats", description = "A command which computes statistics on RDF data using Hadoop")
-public class RdfStats implements Tool {
-
-    static final String ANSI_RED = "\u001B[31m";
-    static final String ANSI_RESET = "\u001B[0m";
-
-    private static final String DATA_TYPE_TRIPLES = "triples", DATA_TYPE_QUADS = "quads", DATA_TYPE_MIXED = "mixed";
-
-    /**
-     * Help option
-     */
-    @Inject
-    public HelpOption helpOption;
-
-    /**
-     * Gets/Sets whether all available statistics will be calculated
-     */
-    @Option(name = { "-a", "--all" }, description = "Requests that all available statistics be calculated", type = OptionType.COMMAND)
-    public boolean all = false;
-
-    /**
-     * Gets/Sets whether node usage counts will be calculated
-     */
-    @Option(name = { "-n", "--node-count" }, description = "Requests that node usage counts be calculated", type = OptionType.COMMAND)
-    public boolean nodeCount = false;
-
-    /**
-     * Gets/Sets whether characteristic sets will be calculated
-     */
-    @Option(name = { "-c", "--characteristic-sets" }, description = "Requests that characteristic sets be calculated", type = OptionType.COMMAND)
-    public boolean characteristicSets = false;
-
-    /**
-     * Gets/Sets whether type counts will be calculated
-     */
-    @Option(name = { "-t", "--type-counts" }, description = "Requests that rdf:type usage counts be calculated", type = OptionType.COMMAND)
-    public boolean typeCount = false;
-
-    /**
-     * Gets/Sets whether data type counts will be calculated
-     */
-    @Option(name = { "-d", "--data-types" }, description = "Requests that literal data type usage counts be calculated", type = OptionType.COMMAND)
-    public boolean dataTypeCount = false;
-
-    /**
-     * Gets/Sets whether namespace counts will be calculated
-     */
-    @Option(name = { "--namespaces" }, description = "Requests that namespace usage counts be calculated", type = OptionType.COMMAND)
-    public boolean namespaceCount = false;
-
-    /**
-     * Gets/Sets the input data type used
-     */
-    @Option(name = { "--input-type" }, allowedValues = { DATA_TYPE_MIXED, DATA_TYPE_QUADS, DATA_TYPE_TRIPLES }, description = "Specifies whether the input data is a mixture of quads and triples, just quads or just triples.  Using the most specific data type will yield the most accurrate statistics")
-    public String inputType = DATA_TYPE_MIXED;
-
-    /**
-     * Gets/Sets the output path
-     */
-    @Option(name = { "-o", "--output" }, title = "OutputPath", description = "Sets the output path", arity = 1, required = true)
-    public String outputPath = null;
-
-    /**
-     * Gets/Sets the input path(s)
-     */
-    @Arguments(description = "Sets the input path(s)", title = "InputPath", required = true)
-    public List<String> inputPaths = new ArrayList<String>();
-
-    private Configuration config;
-
-    /**
-     * Entry point method
-     * 
-     * @param args
-     *            Arguments
-     * @throws Exception
-     */
-    public static void main(String[] args) throws Exception {
-        try {
-            // Run and exit with result code if no errors bubble up
-            // Note that the exit code may still be a error code
-            int res = ToolRunner.run(new Configuration(true), new RdfStats(), args);
-            System.exit(res);
-        } catch (Exception e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            e.printStackTrace(System.err);
-        } finally {
-            System.err.print(ANSI_RESET);
-        }
-        // If any errors bubble up exit with non-zero code
-        System.exit(1);
-    }
-
-    private static void showUsage() {
-        CommandMetadata metadata = SingleCommand.singleCommand(RdfStats.class).getCommandMetadata();
-        StringBuilder builder = new StringBuilder();
-        Help.help(metadata, builder);
-        System.err.print(ANSI_RESET);
-        System.err.println(builder.toString());
-        System.exit(1);
-    }
-
-    @Override
-    public void setConf(Configuration conf) {
-        this.config = conf;
-    }
-
-    @Override
-    public Configuration getConf() {
-        return this.config;
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-        try {
-            // Parse custom arguments
-            RdfStats cmd = SingleCommand.singleCommand(RdfStats.class).parse(args);
-
-            // Copy Hadoop configuration across
-            cmd.setConf(this.getConf());
-
-            // Show help if requested and exit with success
-            if (cmd.helpOption.showHelpIfRequested()) {
-                return 0;
-            }
-
-            // Run the command and exit with success
-            cmd.run();
-            return 0;
-
-        } catch (ParseOptionMissingException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-        } catch (ParseOptionMissingValueException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-        } catch (ParseArgumentsMissingException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-        } catch (ParseArgumentsUnexpectedException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-            // TODO Re-enable as and when we upgrade Airline
-            // } catch (ParseOptionIllegalValueException e) {
-            // System.err.println(ANSI_RED + e.getMessage());
-            // System.err.println();
-            // showUsage();
-        } catch (ParseException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            System.err.println();
-            showUsage();
-        } catch (UnsupportedOperationException e) {
-            System.err.println(ANSI_RED + e.getMessage());
-        } catch (Throwable e) {
-            System.err.println(ANSI_RED + e.getMessage());
-            e.printStackTrace(System.err);
-        } finally {
-            System.err.print(ANSI_RESET);
-        }
-        return 1;
-    }
-
-    private void run() throws Throwable {
-        if (!this.outputPath.endsWith("/")) {
-            this.outputPath += "/";
-        }
-
-        // If all statistics requested turn on all statistics
-        if (this.all) {
-            this.nodeCount = true;
-            this.characteristicSets = true;
-            this.typeCount = true;
-            this.dataTypeCount = true;
-            this.namespaceCount = true;
-        }
-
-        // How many statistics were requested?
-        int statsRequested = 0;
-        if (this.nodeCount)
-            statsRequested++;
-        if (this.characteristicSets)
-            statsRequested++;
-        if (this.typeCount)
-            statsRequested++;
-        if (this.dataTypeCount)
-            statsRequested++;
-        if (this.namespaceCount)
-            statsRequested++;
-
-        // Error if no statistics requested
-        if (statsRequested == 0) {
-            System.err
-                    .println("You did not request any statistics to be calculated, please use one/more of the relevant options to select the statistics to be computed");
-            return;
-        }
-        int statsComputed = 1;
-
-        // Compute statistics
-        if (this.nodeCount) {
-            Job job = this.selectNodeCountJob();
-            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
-        }
-        if (this.typeCount) {
-            Job[] jobs = this.selectTypeCountJobs();
-            statsComputed = this.computeStatistic(jobs, false, false, statsComputed, statsRequested);
-        }
-        if (this.dataTypeCount) {
-            Job job = this.selectDataTypeCountJob();
-            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
-        }
-        if (this.namespaceCount) {
-            Job job = this.selectNamespaceCountJob();
-            statsComputed = this.computeStatistic(job, statsComputed, statsRequested);
-        }
-        if (this.characteristicSets) {
-            Job[] jobs = this.selectCharacteristicSetJobs();
-            statsComputed = this.computeStatistic(jobs, false, false, statsComputed, statsRequested);
-        }
-    }
-
-    private int computeStatistic(Job job, int statsComputed, int statsRequested) throws Throwable {
-        System.out.println(String.format("Computing Statistic %d of %d requested", statsComputed, statsRequested));
-        this.runJob(job);
-        System.out.println(String.format("Computed Statistic %d of %d requested", statsComputed, statsRequested));
-        System.out.println();
-        return ++statsComputed;
-    }
-
-    private int computeStatistic(Job[] jobs, boolean continueOnFailure, boolean continueOnError, int statsComputed,
-            int statsRequested) {
-        System.out.println(String.format("Computing Statistic %d of %d requested", statsComputed, statsRequested));
-        this.runJobSequence(jobs, continueOnFailure, continueOnError);
-        System.out.println(String.format("Computed Statistic %d of %d requested", statsComputed, statsRequested));
-        System.out.println();
-        return ++statsComputed;
-    }
-
-    private boolean runJob(Job job) throws Throwable {
-        System.out.println("Submitting Job " + job.getJobName());
-        long start = System.nanoTime();
-        try {
-            job.submit();
-            if (job.monitorAndPrintJob()) {
-                System.out.println("Job " + job.getJobName() + " succeeded");
-                return true;
-            } else {
-                System.out.println("Job " + job.getJobName() + " failed");
-                return false;
-            }
-        } catch (Throwable e) {
-            System.out.println("Unexpected failure in Job " + job.getJobName());
-            throw e;
-        } finally {
-            long end = System.nanoTime();
-            System.out.println("Job " + job.getJobName() + " finished after "
-                    + String.format("%,d milliseconds", TimeUnit.NANOSECONDS.toMillis(end - start)));
-            System.out.println();
-        }
-    }
-
-    private void runJobSequence(Job[] jobs, boolean continueOnFailure, boolean continueOnError) {
-        for (int i = 0; i < jobs.length; i++) {
-            Job job = jobs[i];
-            try {
-                boolean success = this.runJob(job);
-                if (!success && !continueOnFailure)
-                    throw new IllegalStateException("Unable to complete job sequence because Job " + job.getJobName() + " failed");
-            } catch (IllegalStateException e) {
-                throw e;
-            } catch (Throwable e) {
-                if (!continueOnError)
-                    throw new IllegalStateException("Unable to complete job sequence because job " + job.getJobName()
-                            + " errorred", e);
-            }
-        }
-    }
-
-    private Job selectNodeCountJob() throws IOException {
-        String realOutputPath = outputPath + "node-counts/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadNodeCountJob(this.config, inputs, realOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleNodeCountJob(this.config, inputs, realOutputPath);
-        } else {
-            return JobFactory.getNodeCountJob(this.config, inputs, realOutputPath);
-        }
-    }
-
-    private Job selectDataTypeCountJob() throws IOException {
-        String realOutputPath = outputPath + "data-type-counts/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadDataTypeCountJob(this.config, inputs, realOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleDataTypeCountJob(this.config, inputs, realOutputPath);
-        } else {
-            return JobFactory.getDataTypeCountJob(this.config, inputs, realOutputPath);
-        }
-    }
-
-    private Job selectNamespaceCountJob() throws IOException {
-        String realOutputPath = outputPath + "namespace-counts/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadNamespaceCountJob(this.config, inputs, realOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleNamespaceCountJob(this.config, inputs, realOutputPath);
-        } else {
-            return JobFactory.getNamespaceCountJob(this.config, inputs, realOutputPath);
-        }
-    }
-
-    private Job[] selectCharacteristicSetJobs() throws IOException {
-        String intermediateOutputPath = outputPath + "characteristics/intermediate/";
-        String finalOutputPath = outputPath + "characteristics/final/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        } else {
-            return JobFactory.getCharacteristicSetJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        }
-    }
-
-    private Job[] selectTypeCountJobs() throws IOException {
-        String intermediateOutputPath = outputPath + "type-declarations/";
-        String finalOutputPath = outputPath + "type-counts/";
-        String[] inputs = new String[this.inputPaths.size()];
-        this.inputPaths.toArray(inputs);
-
-        if (DATA_TYPE_QUADS.equals(this.inputType)) {
-            return JobFactory.getQuadTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        } else if (DATA_TYPE_TRIPLES.equals(this.inputType)) {
-            return JobFactory.getTripleTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        } else {
-            return JobFactory.getTypeCountJobs(this.config, inputs, intermediateOutputPath, finalOutputPath);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java b/jena-hadoop-rdf/hadoop-rdf-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
deleted file mode 100644
index 55bb8af..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-stats/src/main/java/org/apache/jena/hadoop/rdf/stats/jobs/JobFactory.java
+++ /dev/null
@@ -1,757 +0,0 @@
-/*
- * 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.stats.jobs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.BZip2Codec;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
-import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.jena.hadoop.rdf.io.input.QuadsInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.TriplesInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.TriplesOrQuadsInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.nquads.NQuadsInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.nquads.NQuadsOutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesNodeOutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesOutputFormat;
-import org.apache.jena.hadoop.rdf.mapreduce.KeyMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.characteristics.CharacteristicSetReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.characteristics.QuadCharacteristicSetGeneratingReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.characteristics.TripleCharacteristicSetGeneratingReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.datatypes.QuadDataTypeCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.datatypes.TripleDataTypeCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.namespaces.QuadNamespaceCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.namespaces.TripleNamespaceCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.positional.QuadObjectCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.positional.TripleObjectCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.QuadFilterByPredicateMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupBySubjectMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupBySubjectMapper;
-import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
-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 com.hp.hpl.jena.vocabulary.RDF;
-
-/**
- * Factory that can produce {@link Job} instances for computing various RDF
- * statistics
- * 
- * 
- * 
- */
-public class JobFactory {
-
-    /**
-     * Private constructor prevents instantiation
-     */
-    private JobFactory() {
-    }
-
-    /**
-     * Gets a job for computing node counts on RDF triple inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getTripleNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Node Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleNodeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing node counts on RDF quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getQuadNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Node Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadNodeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing node counts on RDF triple and/or quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getNodeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Node Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadNodeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a sequence of jobs that can be used to compute characteristic sets
-     * for RDF triples
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Intermediate output path
-     * @param outputPath
-     *            Final output path
-     * @return Sequence of jobs
-     * @throws IOException
-     */
-    public static Job[] getTripleCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Characteristic Set (Generation)");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleGroupBySubjectMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(TripleWritable.class);
-        job.setReducerClass(TripleCharacteristicSetGeneratingReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(NullWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(SequenceFileOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-        SequenceFileOutputFormat.setCompressOutput(job, true);
-        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
-        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
-
-        jobs[0] = job;
-
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Characteristic Set (Reduction)");
-
-        // Map/Reduce classes
-        job.setMapperClass(KeyMapper.class);
-        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
-        job.setMapOutputValueClass(CharacteristicSetWritable.class);
-        job.setReducerClass(CharacteristicSetReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(CharacteristicSetWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(SequenceFileInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-        return jobs;
-    }
-
-    /**
-     * Gets a sequence of jobs that can be used to compute characteristic sets
-     * for RDF quads
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Intermediate output path
-     * @param outputPath
-     *            Final output path
-     * @return Sequence of jobs
-     * @throws IOException
-     */
-    public static Job[] getQuadCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Characteristic Set (Generation)");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadGroupBySubjectMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(QuadWritable.class);
-        job.setReducerClass(QuadCharacteristicSetGeneratingReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(NullWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(SequenceFileOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-        SequenceFileOutputFormat.setCompressOutput(job, true);
-        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
-        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
-
-        jobs[0] = job;
-
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Characteristic Set (Reduction)");
-
-        // Map/Reduce classes
-        job.setMapperClass(KeyMapper.class);
-        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
-        job.setMapOutputValueClass(CharacteristicSetWritable.class);
-        job.setReducerClass(CharacteristicSetReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(CharacteristicSetWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(SequenceFileInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-        return jobs;
-    }
-
-    /**
-     * Gets a sequence of jobs that can be used to compute characteristic sets
-     * for RDF triple and/or quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Intermediate output path
-     * @param outputPath
-     *            Final output path
-     * @return Sequence of jobs
-     * @throws IOException
-     */
-    public static Job[] getCharacteristicSetJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Characteristic Set (Generation)");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadGroupBySubjectMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(QuadWritable.class);
-        job.setReducerClass(QuadCharacteristicSetGeneratingReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(NullWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(SequenceFileOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-        SequenceFileOutputFormat.setCompressOutput(job, true);
-        FileOutputFormat.setOutputCompressorClass(job, BZip2Codec.class);
-        SequenceFileOutputFormat.setOutputCompressionType(job, CompressionType.BLOCK);
-
-        jobs[0] = job;
-
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Characteristic Set (Reduction)");
-
-        // Map/Reduce classes
-        job.setMapperClass(KeyMapper.class);
-        job.setMapOutputKeyClass(CharacteristicSetWritable.class);
-        job.setMapOutputValueClass(CharacteristicSetWritable.class);
-        job.setReducerClass(CharacteristicSetReducer.class);
-        job.setOutputKeyClass(CharacteristicSetWritable.class);
-        job.setOutputValueClass(CharacteristicSetWritable.class);
-
-        // Input and Output
-        job.setInputFormatClass(SequenceFileInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-        return jobs;
-    }
-
-    /**
-     * Gets a job for computing type counts on RDF triple inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Path for intermediate output which will be all the type
-     *            declaration triples present in the inputs
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job[] getTripleTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Type Triples Extraction");
-
-        // Map/Reduce classes
-        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
-        job.setMapperClass(TripleFilterByPredicateUriMapper.class);
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setMapOutputValueClass(TripleWritable.class);
-
-        // Input and Output Format
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(NTriplesOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-
-        jobs[0] = job;
-
-        // Object Node Usage count job
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleObjectCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(NTriplesInputFormat.class);
-        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
-                                                          // better if this was
-                                                          // intelligently
-                                                          // configured
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-
-        return jobs;
-    }
-
-    /**
-     * Gets a job for computing type counts on RDF quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Path for intermediate output which will be all the type
-     *            declaration quads present in the inputs
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job[] getQuadTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Type Quads Extraction");
-
-        // Map/Reduce classes
-        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
-        job.setMapperClass(QuadFilterByPredicateMapper.class);
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setMapOutputValueClass(QuadWritable.class);
-
-        // Input and Output Format
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(NQuadsOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-
-        jobs[0] = job;
-
-        // Object Node Usage count job
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadObjectCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(NQuadsInputFormat.class);
-        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
-                                                          // better if this was
-                                                          // intelligently
-                                                          // configured
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-
-        return jobs;
-    }
-
-    /**
-     * Gets a job for computing type counts on RDF triple and/or quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param intermediateOutputPath
-     *            Path for intermediate output which will be all the type
-     *            declaration quads present in the inputs
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job[] getTypeCountJobs(Configuration config, String[] inputPaths, String intermediateOutputPath,
-            String outputPath) throws IOException {
-        Job[] jobs = new Job[2];
-
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Type Extraction");
-
-        // Map/Reduce classes
-        job.getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, RDF.type.getURI());
-        job.setMapperClass(QuadFilterByPredicateMapper.class);
-        job.setMapOutputKeyClass(LongWritable.class);
-        job.setMapOutputValueClass(QuadWritable.class);
-
-        // Input and Output Format
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(NQuadsOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(intermediateOutputPath));
-
-        jobs[0] = job;
-
-        // Object Node Usage count job
-        job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadObjectCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(NQuadsInputFormat.class);
-        NLineInputFormat.setNumLinesPerSplit(job, 10000); // TODO Would be
-                                                          // better if this was
-                                                          // intelligently
-                                                          // configured
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, intermediateOutputPath);
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        jobs[1] = job;
-
-        return jobs;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF triple inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getTripleDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Literal Data Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleDataTypeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getQuadDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Literal Data Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadDataTypeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF triple and/or
-     * quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getDataTypeCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Literal Data Type Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadDataTypeCountMapper.class);
-        job.setMapOutputKeyClass(NodeWritable.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(NodeCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(NTriplesNodeOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF triple inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getTripleNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Triples Namespace Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(TripleNamespaceCountMapper.class);
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(TextCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getQuadNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Quads Namespace Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadNamespaceCountMapper.class);
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(TextCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(QuadsInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-
-    /**
-     * Gets a job for computing literal data type counts on RDF triple and/or
-     * quad inputs
-     * 
-     * @param config
-     *            Configuration
-     * @param inputPaths
-     *            Input paths
-     * @param outputPath
-     *            Output path
-     * @return Job
-     * @throws IOException
-     */
-    public static Job getNamespaceCountJob(Configuration config, String[] inputPaths, String outputPath) throws IOException {
-        Job job = Job.getInstance(config);
-        job.setJarByClass(JobFactory.class);
-        job.setJobName("RDF Namespace Usage Count");
-
-        // Map/Reduce classes
-        job.setMapperClass(QuadNamespaceCountMapper.class);
-        job.setMapOutputKeyClass(Text.class);
-        job.setMapOutputValueClass(LongWritable.class);
-        job.setReducerClass(TextCountReducer.class);
-
-        // Input and Output
-        job.setInputFormatClass(TriplesOrQuadsInputFormat.class);
-        job.setOutputFormatClass(TextOutputFormat.class);
-        FileInputFormat.setInputPaths(job, StringUtils.arrayToString(inputPaths));
-        FileOutputFormat.setOutputPath(job, new Path(outputPath));
-
-        return job;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/pom.xml b/jena-hadoop-rdf/jena-elephas-common/pom.xml
new file mode 100644
index 0000000..7dd68a0
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/pom.xml
@@ -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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<groupId>org.apache.jena</groupId>
+		<artifactId>jena-elephas</artifactId>
+		<version>0.9.0-SNAPSHOT</version>
+	</parent>
+	<artifactId>jena-elephas-common</artifactId>
+	<name>Apache Jena - Elephas - Common API</name>
+	<description>Common code for RDF on Hadoop such as writable types for RDF primitives</description>
+
+	<!-- Note that versions are managed by parent POMs -->
+	<dependencies>
+		<!-- Hadoop Dependencies -->
+		<!-- Note these will be provided on the Hadoop cluster hence the provided 
+			scope -->
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Jena dependencies -->
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-arq</artifactId>
+		</dependency>
+
+		<!-- Test Dependencies -->
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
new file mode 100644
index 0000000..f0acc09
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
@@ -0,0 +1,193 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.util.NodeUtils;
+
+/**
+ * A abstract general purpose writable where the actual class represented is
+ * composed of a number of {@link Node} instances
+ * <p>
+ * The binary encoding of this base implementation is just a variable integer
+ * indicating the number of nodes present followed by the binary encodings of
+ * the {@link NodeWritable} instances. Derived implementations may wish to
+ * override the {@link #readFields(DataInput)} and {@link #write(DataOutput)}
+ * methods in order to use more specialised encodings.
+ * </p>
+ * 
+ * @param <T>
+ *            Tuple type
+ */
+public abstract class AbstractNodeTupleWritable<T> implements WritableComparable<AbstractNodeTupleWritable<T>> {
+
+    private T tuple;
+
+    /**
+     * Creates a new empty instance
+     */
+    protected AbstractNodeTupleWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance with the given value
+     * 
+     * @param tuple
+     *            Tuple value
+     */
+    protected AbstractNodeTupleWritable(T tuple) {
+        this.tuple = tuple;
+    }
+
+    /**
+     * Gets the tuple
+     * 
+     * @return Tuple
+     */
+    public T get() {
+        return this.tuple;
+    }
+
+    /**
+     * Sets the tuple
+     * 
+     * @param tuple
+     *            Tuple
+     */
+    public void set(T tuple) {
+        this.tuple = tuple;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        // Determine how many nodes
+        int size = WritableUtils.readVInt(input);
+        Node[] ns = new Node[size];
+
+        NodeWritable nw = new NodeWritable();
+        for (int i = 0; i < ns.length; i++) {
+            nw.readFields(input);
+            ns[i] = nw.get();
+        }
+
+        // Load the tuple
+        this.tuple = this.createTuple(ns);
+    }
+
+    /**
+     * Creates the actual tuple type from an array of nodes
+     * 
+     * @param ns
+     *            Nodes
+     * @return Tuple
+     */
+    protected abstract T createTuple(Node[] ns);
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        // Determine how many nodes
+        Node[] ns = this.createNodes(this.tuple);
+        WritableUtils.writeVInt(output, ns.length);
+
+        // Write out nodes
+        NodeWritable nw = new NodeWritable();
+        for (int i = 0; i < ns.length; i++) {
+            nw.set(ns[i]);
+            nw.write(output);
+        }
+    }
+
+    /**
+     * Sets the tuple value
+     * <p>
+     * Intended only for internal use i.e. when a derived implementation
+     * overrides {@link #readFields(DataInput)} and needs to set the tuple value
+     * directly i.e. when a derived implementation is using a custom encoding
+     * scheme
+     * </p>
+     * 
+     * @param tuple
+     *            Tuple
+     */
+    protected final void setInternal(T tuple) {
+        this.tuple = tuple;
+    }
+
+    /**
+     * Converts the actual tuple type into an array of nodes
+     * 
+     * @param tuple
+     *            Tuples
+     * @return Nodes
+     */
+    protected abstract Node[] createNodes(T tuple);
+
+    /**
+     * Compares instances node by node
+     * <p>
+     * Derived implementations may wish to override this and substitute native
+     * tuple based comparisons
+     * </p>
+     * 
+     * @param other
+     *            Instance to compare with
+     */
+    @Override
+    public int compareTo(AbstractNodeTupleWritable<T> other) {
+        Node[] ns = this.createNodes(this.tuple);
+        Node[] otherNs = this.createNodes(other.tuple);
+
+        if (ns.length < otherNs.length) {
+            return -1;
+        } else if (ns.length > otherNs.length) {
+            return 1;
+        }
+        // Compare node by node
+        for (int i = 0; i < ns.length; i++) {
+            int c = NodeUtils.compareRDFTerms(ns[i], otherNs[i]);
+            if (c != 0)
+                return c;
+        }
+        return 0;
+    }
+
+    @Override
+    public String toString() {
+        return this.get().toString();
+    }
+
+    @Override
+    public int hashCode() {
+        return this.get().hashCode();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof AbstractNodeTupleWritable))
+            return false;
+        return this.compareTo((AbstractNodeTupleWritable<T>) other) == 0;
+    }
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
deleted file mode 100644
index 4731832..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.rdfjson;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for RDF/JSON input
- * 
- * 
- * 
- */
-public class RdfJsonInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFJSON;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".rj";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new RdfJsonInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
deleted file mode 100644
index f8edcc4..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.rdfxml;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for RDF/XML input
- * 
- * 
- * 
- */
-public class RdfXmlInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFXML;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".rdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new RdfXmlInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
deleted file mode 100644
index 8d79295..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-
-/**
- * Tests for JSON-LD input
- * 
- *
- */
-public class ThriftQuadInputTest extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new ThriftQuadInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
deleted file mode 100644
index 6b5e0b7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-
-/**
- * Tests for JSON-LD input
- * 
- *
- */
-public class ThriftTripleInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new ThriftTripleInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
deleted file mode 100644
index 1fad0dc..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.trig;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for TriG input
- * 
- *
- */
-public class TriGInputTest extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIG;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trig";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new TriGInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
deleted file mode 100644
index 4a3a66a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.trix;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for TriX input
- * 
- *
- */
-public class TriXInputTest extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIX;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trix";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new TriXInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
deleted file mode 100644
index e6211ba..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.turtle;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for turtle input format
- * 
- * 
- * 
- */
-public class TurtleInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected final String getFileExtension() {
-        return ".ttl";
-    }
-
-    @Override
-    protected final Lang getRdfLanguage() {
-        return Lang.TURTLE;
-    }
-    
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new TurtleInputFormat();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
deleted file mode 100644
index 9532d56..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
+++ /dev/null
@@ -1,701 +0,0 @@
-/*
- * 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.util;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Abstract tests for {@link TrackableInputStream} implementations
- * 
- * 
- * 
- */
-public abstract class AbstractTrackableInputStreamTests {
-
-    protected static final int KILO = 1024;
-    protected static final int BYTES_PER_KB = KILO;
-    protected static final int BYTES_PER_MB = BYTES_PER_KB * KILO;
-
-    /**
-     * Gets the instance to test using the given input as the stream to track
-     * 
-     * @param input
-     *            Input Stream
-     * @return Trackable Input Stream
-     */
-    protected abstract TrackableInputStream getInstance(InputStream input);
-
-    /**
-     * Generates an input stream containing the given number of bytes
-     * 
-     * @param length
-     *            Number of bytes
-     * @return Input stream
-     */
-    protected final InputStream generateData(int length) {
-        ByteArrayOutputStream output = new ByteArrayOutputStream(length);
-        byte b = (byte) 'b';
-        for (int i = 0; i < length; i++) {
-            output.write(b);
-        }
-        return new ByteArrayInputStream(output.toByteArray());
-    }
-
-    protected final void testSingleByteRead(int length) throws IOException {
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input);
-        long count = 0;
-        while (trackable.read() >= 0) {
-            count++;
-        }
-        Assert.assertEquals(length, count);
-        Assert.assertEquals(length, trackable.getBytesRead());
-        trackable.close();
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_single_01() throws IOException {
-        this.testSingleByteRead(0);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_single_02() throws IOException {
-        this.testSingleByteRead(100);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_single_03() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_single_04() throws IOException {
-        // 1 MB
-        this.testSingleByteRead(BYTES_PER_MB);
-    }
-
-    protected final void testMultiByteRead(int length, int bufferSize) throws IOException {
-        if (bufferSize < 1)
-            throw new IllegalArgumentException("bufferSize must be >= 1");
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input);
-        long count = 0;
-        byte[] buffer = new byte[bufferSize];
-        long read;
-        do {
-            read = trackable.read(buffer);
-            if (read > 0)
-                count += read;
-        } while (read >= 0);
-        Assert.assertEquals(length, count);
-        Assert.assertEquals(length, trackable.getBytesRead());
-        trackable.close();
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_01() throws IOException {
-        this.testMultiByteRead(0, 1);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_02() throws IOException {
-        this.testMultiByteRead(0, 16);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_03() throws IOException {
-        this.testMultiByteRead(0, BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_04() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_05() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_06() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_07() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_08() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_09() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB);
-    }
-
-    protected final void testMultiByteRead(int length, int bufferSize, int readSize) throws IOException {
-        if (bufferSize < 1)
-            throw new IllegalArgumentException("bufferSize must be >= 1");
-        if (readSize < 1 || readSize > bufferSize)
-            throw new IllegalArgumentException("readSize must be >= 1 and <= bufferSize");
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input);
-        long count = 0;
-        byte[] buffer = new byte[bufferSize];
-        long read;
-        do {
-            read = trackable.read(buffer, 0, readSize);
-            if (read > 0)
-                count += read;
-        } while (read >= 0);
-        Assert.assertEquals(length, count);
-        Assert.assertEquals(length, trackable.getBytesRead());
-        trackable.close();
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_01() throws IOException {
-        this.testMultiByteRead(0, 1, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_02() throws IOException {
-        this.testMultiByteRead(0, 16, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_03() throws IOException {
-        this.testMultiByteRead(0, 16, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_04() throws IOException {
-        this.testMultiByteRead(0, BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_05() throws IOException {
-        this.testMultiByteRead(0, BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_06() throws IOException {
-        this.testMultiByteRead(0, BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_07() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 1, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_08() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 16, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_09() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, 16, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_10() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_11() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_12() throws IOException {
-        // 1KB
-        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_13() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 1, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_14() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 16, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_15() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, 16, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_16() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_17() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test reading multiple bytes while reading less than the buffer size bytes
-     * i.e. calling {@link InputStream#read(byte[], int, int)}
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_read_multiple_partial_18() throws IOException {
-        // 1MB
-        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    protected final void testSkip(int length, long skipSize) throws IOException {
-        if (skipSize < 1)
-            throw new IllegalArgumentException("skipSize must be >= 1");
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input);
-        long count = 0;
-        long skipped;
-        do {
-            skipped = trackable.skip(skipSize);
-            if (skipped > 0)
-                count += skipped;
-        } while (skipped > 0);
-        Assert.assertEquals(length, count);
-        Assert.assertEquals(length, trackable.getBytesRead());
-        trackable.close();
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_01() throws IOException {
-        this.testSkip(0, 1);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_02() throws IOException {
-        this.testSkip(100, 1);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_03() throws IOException {
-        this.testSkip(100, 16);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_04() throws IOException {
-        this.testSkip(100, BYTES_PER_KB);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_05() throws IOException {
-        // 1KB
-        this.testSkip(BYTES_PER_KB, 1);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_06() throws IOException {
-        // 1KB
-        this.testSkip(BYTES_PER_KB, 16);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_07() throws IOException {
-        // 1KB
-        this.testSkip(BYTES_PER_KB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_08() throws IOException {
-        // 1KB
-        this.testSkip(BYTES_PER_KB, BYTES_PER_MB);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_09() throws IOException {
-        // 1 MB
-        this.testSkip(BYTES_PER_MB, 1);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_10() throws IOException {
-        // 1 MB
-        this.testSkip(BYTES_PER_MB, 16);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_11() throws IOException {
-        // 1 MB
-        this.testSkip(BYTES_PER_MB, BYTES_PER_KB);
-    }
-
-    /**
-     * Test skipping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_skip_single_12() throws IOException {
-        // 1 MB
-        this.testSkip(BYTES_PER_MB, BYTES_PER_MB);
-    }
-
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_01() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(-1, trackable.read());
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_02() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(0, trackable.read(new byte[0]));
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_03() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(-1, trackable.read(new byte[1]));
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_04() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(0, trackable.read(new byte[16], 0, 0));
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_05() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(-1, trackable.read(new byte[16], 0, 8));
-    }
-    
-    /**
-     * Tests behaviour after closing
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_06() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(0, trackable.skip(0));
-    }
-    
-    /**
-     * Tests exceptions are thrown trying to perform actions after closing the
-     * input
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void trackable_input_post_close_07() throws IOException {
-        InputStream input = this.generateData(0);
-        TrackableInputStream trackable = this.getInstance(input);
-        trackable.close();
-        Assert.assertEquals(0, trackable.skip(1));
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
deleted file mode 100644
index f8819bc..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/*
- * 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.util;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.jena.hadoop.rdf.io.input.util.BlockInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for the {@link BlockInputStream}
- * 
- * 
- * 
- */
-public class BlockInputStreamTest extends AbstractTrackableInputStreamTests {
-
-    @Override
-    protected TrackableInputStream getInstance(InputStream input) {
-        return new BlockInputStream(input, Long.MAX_VALUE);
-    }
-
-    /**
-     * Gets an instance of a block input stream
-     * 
-     * @param input
-     *            Underlying input stream
-     * @param limit
-     *            Limit on bytes to read
-     * @return Block input stream
-     */
-    protected BlockInputStream getInstance(InputStream input, long limit) {
-        return new BlockInputStream(input, limit);
-    }
-    
-    protected final void testSingleByteRead(int length, long limit) throws IOException {
-        InputStream input = this.generateData(length);
-        TrackableInputStream trackable = this.getInstance(input, limit);
-        long count = 0;
-        while (trackable.read() >= 0) {
-            count++;
-        }
-        int expected = (int) Math.min(length, limit);
-        Assert.assertEquals(expected, count);
-        Assert.assertEquals(expected, trackable.getBytesRead());
-        trackable.close();
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_01() throws IOException {
-        this.testSingleByteRead(0, 0);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_02() throws IOException {
-        this.testSingleByteRead(100, 0);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_03() throws IOException {
-        this.testSingleByteRead(100, 50);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_04() throws IOException {
-        this.testSingleByteRead(100, 100);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_05() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, 1);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_06() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, 100);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_07() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_KB / 2);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_08() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_KB);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_09() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_MB);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_10() throws IOException {
-        // 1KB
-        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_MB * 10);
-    }
-
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_11() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, 1);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_12() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, 100);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_13() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_KB);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_14() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB / 2);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_15() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB);
-    }
-    
-    /**
-     * Test reading byte by byte
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void block_input_read_single_16() throws IOException {
-        // 1MB
-        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB * 10);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
deleted file mode 100644
index f453aa5..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.util;
-
-import java.io.InputStream;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
-
-/**
- * Tests for the {@link TrackedInputStream}
- * 
- * 
- * 
- */
-public class TrackedInputStreamTest extends AbstractTrackableInputStreamTests {
-
-    @Override
-    protected TrackableInputStream getInstance(InputStream input) {
-        return new TrackedInputStream(input);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
deleted file mode 100644
index b5ea2d8..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * 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.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFDataMgr;
-import org.apache.jena.riot.lang.StreamRDFCounting;
-import org.apache.jena.riot.system.StreamRDFLib;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Abstract node tuple output format tests
- * 
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- * 
- */
-public abstract class AbstractNodeTupleOutputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> {
-
-    @SuppressWarnings("unused")
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleOutputFormatTests.class);
-
-    protected static final int EMPTY_SIZE = 0, SMALL_SIZE = 100, LARGE_SIZE = 10000, VERY_LARGE_SIZE = 100000;
-
-    /**
-     * Temporary folder for the tests
-     */
-    @Rule
-    public TemporaryFolder folder = new TemporaryFolder();
-
-    /**
-     * Prepares a fresh configuration
-     * 
-     * @return Configuration
-     */
-    protected Configuration prepareConfiguration() {
-        Configuration config = new Configuration(true);
-        // Nothing else to do
-        return config;
-    }
-
-    /**
-     * Gets the extra file extension to add to the filenames
-     * 
-     * @return File extension
-     */
-    protected abstract String getFileExtension();
-
-    /**
-     * Generates tuples to be output for testing
-     * 
-     * @param num
-     *            Number of tuples to generate
-     * @return Iterator of tuples
-     */
-    protected abstract Iterator<T> generateTuples(int num);
-
-    /**
-     * Counts tuples in the output file
-     * 
-     * @param f
-     *            Output file
-     * @return Tuple count
-     */
-    protected final long countTuples(File f) {
-        StreamRDFCounting counter = StreamRDFLib.count();
-        RDFDataMgr.parse(counter, f.getAbsolutePath(), this.getRdfLanguage(), null);
-        return counter.count();
-    }
-
-    /**
-     * Checks that tuples are as expected
-     * 
-     * @param f
-     *            File
-     * @param expected
-     *            Expected number of tuples
-     */
-    protected void checkTuples(File f, long expected) {
-        Assert.assertEquals(expected, this.countTuples(f));
-    }
-
-    /**
-     * Gets the RDF language of the produced output which is used to parse back
-     * in the output to validate the correct amount of output was produced
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-    /**
-     * Gets the output format to test
-     * 
-     * @return Output format
-     */
-    protected abstract OutputFormat<NullWritable, T> getOutputFormat();
-
-    /**
-     * Adds an output path to the job configuration
-     * 
-     * @param f
-     *            File
-     * @param config
-     *            Configuration
-     * @param job
-     *            Job
-     * @throws IOException
-     */
-    protected void addOutputPath(File f, Configuration config, Job job) throws IOException {
-        FileSystem fs = FileSystem.getLocal(config);
-        Path outputPath = fs.makeQualified(new Path(f.getAbsolutePath()));
-        FileOutputFormat.setOutputPath(job, outputPath);
-    }
-
-    protected File findOutputFile(File dir, JobContext context) throws FileNotFoundException, IOException {
-        Path outputPath = FileOutputFormat.getOutputPath(context);
-        RemoteIterator<LocatedFileStatus> files = outputPath.getFileSystem(context.getConfiguration()).listFiles(
-                outputPath, true);
-        while (files.hasNext()) {
-            LocatedFileStatus status = files.next();
-            if (status.isFile() && !status.getPath().getName().startsWith("_")) {
-                return new File(status.getPath().toUri());
-            }
-        }
-        return null;
-    }
-
-    /**
-     * Tests output
-     * 
-     * @param f
-     *            File to output to
-     * @param num
-     *            Number of tuples to output
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected final void testOutput(File f, int num) throws IOException, InterruptedException {
-        // Prepare configuration
-        Configuration config = this.prepareConfiguration();
-
-        // Set up fake job
-        OutputFormat<NullWritable, T> outputFormat = this.getOutputFormat();
-        Job job = Job.getInstance(config);
-        job.setOutputFormatClass(outputFormat.getClass());
-        this.addOutputPath(f, job.getConfiguration(), job);
-        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-        Assert.assertNotNull(FileOutputFormat.getOutputPath(context));
-
-        // Output the data
-        TaskAttemptID id = new TaskAttemptID("outputTest", 1, TaskType.MAP, 1, 1);
-        TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), id);
-        RecordWriter<NullWritable, T> writer = outputFormat.getRecordWriter(taskContext);
-        Iterator<T> tuples = this.generateTuples(num);
-        while (tuples.hasNext()) {
-            writer.write(NullWritable.get(), tuples.next());
-        }
-        writer.close(taskContext);
-
-        // Check output
-        File outputFile = this.findOutputFile(this.folder.getRoot(), context);
-        Assert.assertNotNull(outputFile);
-        this.checkTuples(outputFile, num);
-    }
-
-    /**
-     * Basic output tests
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void output_01() throws IOException, InterruptedException {
-        this.testOutput(this.folder.getRoot(), EMPTY_SIZE);
-    }
-
-    /**
-     * Basic output tests
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void output_02() throws IOException, InterruptedException {
-        this.testOutput(this.folder.getRoot(), SMALL_SIZE);
-    }
-
-    /**
-     * Basic output tests
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void output_03() throws IOException, InterruptedException {
-        this.testOutput(this.folder.getRoot(), LARGE_SIZE);
-    }
-
-    /**
-     * Basic output tests
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
deleted file mode 100644
index f1822f6..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for quad output formats
- * 
- * 
- * 
- */
-public abstract class AbstractQuadOutputFormatTests extends AbstractNodeTupleOutputFormatTests<Quad, QuadWritable> {
-
-    @Override
-    protected Iterator<QuadWritable> generateTuples(int num) {
-        List<QuadWritable> qs = new ArrayList<QuadWritable>();
-        for (int i = 0; i < num; i++) {
-            Quad q = new Quad(NodeFactory.createURI("http://example.org/graphs/" + i),
-                    NodeFactory.createURI("http://example.org/subjects/" + i),
-                    NodeFactory.createURI("http://example.org/predicate"), NodeFactory.createLiteral(Integer.toString(i),
-                            XSDDatatype.XSDinteger));
-            qs.add(new QuadWritable(q));
-        }
-        return qs.iterator();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
deleted file mode 100644
index 90eb531..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for triple output formats
- * 
- *
- */
-public abstract class AbstractTripleOutputFormatTests extends AbstractNodeTupleOutputFormatTests<Triple, TripleWritable> {
-
-    @Override
-    protected Iterator<TripleWritable> generateTuples(int num) {
-        List<TripleWritable> ts = new ArrayList<TripleWritable>();
-        for (int i = 0; i < num; i++) {
-            Triple t = new Triple(NodeFactory.createURI("http://example.org/subjects/" + i), NodeFactory.createURI("http://example.org/predicate"), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
-            ts.add(new TripleWritable(t));
-        }
-        return ts.iterator();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
deleted file mode 100644
index c6784a5..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * 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>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
deleted file mode 100644
index d157409..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * 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>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
deleted file mode 100644
index 1a7ffa4..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for NQuads output format
- * 
- * 
- * 
- */
-public class NQuadsOutputTest extends AbstractQuadOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".nq";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new NQuadsOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
deleted file mode 100644
index ad9be56..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for NTriples output format
- * 
- * 
- * 
- */
-public class NTriplesOutputTest extends AbstractTripleOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".nt";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new NTriplesOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
deleted file mode 100644
index 833f89b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for RDF/JSON output
- * 
- * 
- * 
- */
-public class RdfJsonOutputTest extends AbstractTripleOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".rj";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFJSON;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new RdfJsonOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
deleted file mode 100644
index 40bc937..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for RDF/XML output
- * 
- * 
- * 
- */
-public class RdfXmlOutputTest extends AbstractTripleOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".rdf";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFXML;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new RdfXmlOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputTest.java
deleted file mode 100644
index 91509da..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputTest.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Tests for Thrift output format
- */
-public class ThriftQuadOutputTest extends AbstractQuadOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new ThriftQuadOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputTest.java
deleted file mode 100644
index cf50330..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputTest.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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 org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Tests for Thrift output format
- */
-public class ThriftTripleOutputTest extends AbstractTripleOutputFormatTests {
-
-    @Override
-    protected String getFileExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
-        return new ThriftTripleOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
deleted file mode 100644
index fd886a3..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.util.Arrays;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-
-/**
- * Tests for TriG output
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class BatchedTriGOutputTest extends AbstractQuadOutputFormatTests {
-
-    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-    static long $bs2 = 1000;
-    static long $bs3 = 100;
-    static long $bs4 = 1;
-
-    /**
-     * @return Test parameters
-     */
-    @Parameters
-    public static Collection<Object[]> data() {
-        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
-    }
-
-    private final long batchSize;
-
-    /**
-     * Creates new tests
-     * 
-     * @param batchSize
-     *            Batch size
-     */
-    public BatchedTriGOutputTest(long batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trig";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIG;
-    }
-
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
-        return config;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new BatchedTriGOutputFormat<NullWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
deleted file mode 100644
index 9b2b669..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.util.Arrays;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-
-/**
- * Tests for Turtle output
- * 
- * 
- * 
- */
-@RunWith(Parameterized.class)
-public class StreamedTriGOutputTest extends AbstractQuadOutputFormatTests {
-
-    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
-    static long $bs2 = 1000;
-    static long $bs3 = 100;
-    static long $bs4 = 1;
-
-    /**
-     * @return Test parameters
-     */
-    @Parameters
-    public static Collection<Object[]> data() {
-        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
-    }
-
-    private final long batchSize;
-
-    /**
-     * Creates new tests
-     * 
-     * @param batchSize
-     *            Batch size
-     */
-    public StreamedTriGOutputTest(long batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".trig";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIG;
-    }
-    
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
-        return config;
-    }
-
-    @Override
-    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
-        return new TriGOutputFormat<NullWritable>();
-    }
-
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapperTest.java
deleted file mode 100644
index fdac378..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapperTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Tests for the {@link TripleNodeCountMapper}
- * 
- * 
- * 
- */
-public class TripleNodeCountMapperTest extends AbstractNodeTupleNodeCountTests<Triple, TripleWritable> {
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, NodeWritable, LongWritable> getInstance() {
-        return new TripleNodeCountMapper<LongWritable>();
-    }
-
-    @Override
-    protected TripleWritable createValue(int i) {
-        return new TripleWritable(
-                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        Triple t = tuple.get();
-        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
-                new NodeWritable(t.getObject()) };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterTests.java
deleted file mode 100644
index 23c5661..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterTests.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
-import org.apache.jena.hadoop.rdf.mapreduce.filter.AbstractNodeTupleFilterMapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.junit.Test;
-
-
-/**
- * Abstract tests for {@link AbstractNodeTupleFilterMapper} implementations
- * which filter based on the validity of tuples
- * 
- * 
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleFilterTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        AbstractMapperTests<LongWritable, T, LongWritable, T> {
-
-    protected final void generateData(MapDriver<LongWritable, T, LongWritable, T> driver, int num) {
-        for (int i = 0; i < num; i++) {
-            LongWritable key = new LongWritable(i);
-            if (i % 2 == 0 && !this.noValidInputs()) {
-                T value = this.createValidValue(i);
-                driver.addInput(key, value);
-                if (!this.isInverted())
-                    driver.addOutput(key, value);
-            } else {
-                T value = this.createInvalidValue(i);
-                driver.addInput(key, value);
-                if (this.isInverted())
-                    driver.addOutput(key, value);
-            }
-        }
-    }
-
-    /**
-     * Method that may be overridden for testing filters where all the generated
-     * data will be rejected as invalid
-     * 
-     * @return True if there are no valid inputs, false otherwise (default)
-     */
-    protected boolean noValidInputs() {
-        return false;
-    }
-
-    /**
-     * Method that may be overridden for testing filters with inverted mode
-     * enabled i.e. where normally valid input is considered invalid and vice
-     * versa
-     * 
-     * @return True if inverted, false otherwise (default)
-     */
-    protected boolean isInverted() {
-        return false;
-    }
-
-    /**
-     * Creates an invalid value
-     * 
-     * @param i
-     *            Key
-     * @return Invalid value
-     */
-    protected abstract T createInvalidValue(int i);
-
-    /**
-     * Creates a valid value
-     * 
-     * @param i
-     *            Key
-     * @return Valid value
-     */
-    protected abstract T createValidValue(int i);
-
-    protected final void testFilterValid(int num) throws IOException {
-        MapDriver<LongWritable, T, LongWritable, T> driver = this.getMapDriver();
-        this.generateData(driver, num);
-        driver.runTest();
-    }
-
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void filter_valid_01() throws IOException {
-        this.testFilterValid(1);
-    }
-
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void filter_valid_02() throws IOException {
-        this.testFilterValid(100);
-    }
-
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void filter_valid_03() throws IOException {
-        this.testFilterValid(1000);
-    }
-
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void filter_valid_04() throws IOException {
-        this.testFilterValid(2500);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadValidityFilterTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadValidityFilterTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadValidityFilterTests.java
deleted file mode 100644
index 9458f83..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadValidityFilterTests.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for triple filter mappers that check triple validity
- * 
- * 
- * 
- */
-public abstract class AbstractQuadValidityFilterTests extends AbstractNodeTupleFilterTests<Quad, QuadWritable> {
-
-    @Override
-    protected QuadWritable createValidValue(int i) {
-        return new QuadWritable(
-                new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
-                        NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
-                                XSDDatatype.XSDinteger)));
-    }
-
-    @Override
-    protected QuadWritable createInvalidValue(int i) {
-        switch (i % 8) {
-        case 0:
-            // Invalid to use Literal as Graph
-            return new QuadWritable(new Quad(NodeFactory.createLiteral("invalid"), NodeFactory.createURI("http://subjects/" + i),
-                    NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
-                            XSDDatatype.XSDinteger)));
-        case 1:
-            // Invalid to use Variable as Graph
-            return new QuadWritable(new Quad(NodeFactory.createVariable("invalid"),
-                    NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-        case 2:
-            // Invalid to use Literal as Subject
-            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createLiteral("invalid"),
-                    NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
-                            XSDDatatype.XSDinteger)));
-        case 3:
-            // Invalid to use Variable as Subject
-            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createVariable("invalid"),
-                    NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
-                            XSDDatatype.XSDinteger)));
-        case 4:
-            // Invalid to use Blank Node as Predicate
-            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
-                    NodeFactory.createAnon(), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-        case 5:
-            // Invalid to use Literal as Predicate
-            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
-                    NodeFactory.createLiteral("invalid"), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-        case 6:
-            // Invalid to use Variable as Predicate
-            return new QuadWritable(
-                    new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
-                            NodeFactory.createVariable("invalid"), NodeFactory.createLiteral(Integer.toString(i),
-                                    XSDDatatype.XSDinteger)));
-        default:
-            // Invalid to use Variable as Object
-            return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://subjects/" + i),
-                    NodeFactory.createURI("http://predicate"), NodeFactory.createVariable("invalid")));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleValidityFilterTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleValidityFilterTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleValidityFilterTests.java
deleted file mode 100644
index 3c41710..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleValidityFilterTests.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for triple filter mappers that check triple validity
- * 
- * 
- * 
- */
-public abstract class AbstractTripleValidityFilterTests extends AbstractNodeTupleFilterTests<Triple, TripleWritable> {
-
-    @Override
-    protected TripleWritable createValidValue(int i) {
-        return new TripleWritable(
-                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-    @Override
-    protected TripleWritable createInvalidValue(int i) {
-        switch (i % 6) {
-        case 0:
-            // Invalid to use Literal as Subject
-            return new TripleWritable(new Triple(NodeFactory.createLiteral("invalid"), NodeFactory.createURI("http://predicate"),
-                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-        case 1:
-            // Invalid to use Variable as Subject
-            return new TripleWritable(new Triple(NodeFactory.createVariable("invalid"),
-                    NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral(Integer.toString(i),
-                            XSDDatatype.XSDinteger)));
-        case 2:
-            // Invalid to use Blank Node as Predicate
-            return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createAnon(),
-                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-        case 3:
-            // Invalid to use Literal as Predicate
-            return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i),
-                    NodeFactory.createLiteral("invalid"), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-        case 4:
-            // Invalid to use Variable as Predicate
-            return new TripleWritable(
-                    new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createVariable("invalid"),
-                            NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-        default:
-            // Invalid to use Variable as Object
-            return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i),
-                    NodeFactory.createURI("http://predicate"), NodeFactory.createVariable("invalid")));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByNoPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByNoPredicateMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByNoPredicateMapperTest.java
deleted file mode 100644
index 4c9fb5a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByNoPredicateMapperTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
-
-/**
- * Tests for the {@link TripleFilterByPredicateUriMapper} where there are no
- * predicates and thus all data must be invalid
- * 
- * 
- * 
- */
-public class TripleFilterByNoPredicateMapperTest extends TripleFilterByPredicateMapperTest {
-
-    private static final String[] EMPTY_PREDICATE_POOL = new String[0];
-
-    /**
-     * Gets the pool of predicates considered valid
-     * 
-     * @return Predicate pool
-     */
-    @Override
-    protected String[] getPredicatePool() {
-        return EMPTY_PREDICATE_POOL;
-    }
-
-    @Override
-    protected boolean noValidInputs() {
-        return true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByPredicateMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByPredicateMapperTest.java
deleted file mode 100644
index add363e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleFilterByPredicateMapperTest.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.vocabulary.RDF;
-import com.hp.hpl.jena.vocabulary.RDFS;
-
-/**
- * Tests for the {@link TripleFilterByPredicateUriMapper}
- * 
- * 
- * 
- */
-public class TripleFilterByPredicateMapperTest extends AbstractNodeTupleFilterTests<Triple, TripleWritable> {
-
-    private static final String[] DEFAULT_PREDICATE_POOL = new String[] { RDF.type.getURI(), RDFS.range.getURI(),
-            RDFS.domain.getURI() };
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, LongWritable, TripleWritable> getInstance() {
-        return new TripleFilterByPredicateUriMapper<LongWritable>();
-    }
-
-    @Override
-    protected void configureDriver(MapDriver<LongWritable, TripleWritable, LongWritable, TripleWritable> driver) {
-        super.configureDriver(driver);
-        driver.getContext().getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, this.getPredicatePool());
-    }
-
-    /**
-     * Gets the pool of predicates considered valid
-     * 
-     * @return Predicate pool
-     */
-    protected String[] getPredicatePool() {
-        return DEFAULT_PREDICATE_POOL;
-    }
-
-    @Override
-    protected TripleWritable createInvalidValue(int i) {
-        return new TripleWritable(
-                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-    @Override
-    protected TripleWritable createValidValue(int i) {
-        String[] predicates = this.getPredicatePool();
-        if (predicates.length == 0) return this.createInvalidValue(i);
-        return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI(predicates[i
-                % predicates.length]), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByNoPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByNoPredicateMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByNoPredicateMapperTest.java
deleted file mode 100644
index fb7dd25..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByNoPredicateMapperTest.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
-
-/**
- * Tests for the {@link TripleFilterByPredicateUriMapper} where there are no
- * predicates and thus all data must be invalid
- * 
- * 
- * 
- */
-public class TripleInvertedFilterByNoPredicateMapperTest extends TripleInvertedFilterByPredicateMapperTest {
-
-    private static final String[] EMPTY_PREDICATE_POOL = new String[0];
-
-    /**
-     * Gets the pool of predicates considered valid
-     * 
-     * @return Predicate pool
-     */
-    @Override
-    protected String[] getPredicatePool() {
-        return EMPTY_PREDICATE_POOL;
-    }
-
-    @Override
-    protected boolean noValidInputs() {
-        return true;
-    }
-    
-    @Override
-    protected boolean isInverted() {
-        return true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByPredicateMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByPredicateMapperTest.java
deleted file mode 100644
index b24e9c2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/filter/TripleInvertedFilterByPredicateMapperTest.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-import org.apache.jena.hadoop.rdf.mapreduce.filter.positional.TripleFilterByPredicateUriMapper;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.vocabulary.RDF;
-import com.hp.hpl.jena.vocabulary.RDFS;
-
-/**
- * Tests for the {@link TripleFilterByPredicateUriMapper}
- * 
- * 
- * 
- */
-public class TripleInvertedFilterByPredicateMapperTest extends AbstractNodeTupleFilterTests<Triple, TripleWritable> {
-
-    private static final String[] DEFAULT_PREDICATE_POOL = new String[] { RDF.type.getURI(), RDFS.range.getURI(),
-            RDFS.domain.getURI() };
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, LongWritable, TripleWritable> getInstance() {
-        return new TripleFilterByPredicateUriMapper<LongWritable>();
-    }
-
-    @Override
-    protected void configureDriver(MapDriver<LongWritable, TripleWritable, LongWritable, TripleWritable> driver) {
-        super.configureDriver(driver);
-        driver.getContext().getConfiguration().setStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS, this.getPredicatePool());
-        driver.getContext().getConfiguration().setBoolean(RdfMapReduceConstants.FILTER_INVERT, true);
-    }
-
-    @Override
-    protected boolean isInverted() {
-        return true;
-    }
-
-    /**
-     * Gets the pool of predicates considered valid
-     * 
-     * @return Predicate pool
-     */
-    protected String[] getPredicatePool() {
-        return DEFAULT_PREDICATE_POOL;
-    }
-
-    @Override
-    protected TripleWritable createInvalidValue(int i) {
-        return new TripleWritable(
-                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-    @Override
-    protected TripleWritable createValidValue(int i) {
-        String[] predicates = this.getPredicatePool();
-        if (predicates.length == 0)
-            return this.createInvalidValue(i);
-        return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI(predicates[i
-                % predicates.length]), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-}

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

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

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingTests.java
deleted file mode 100644
index 1e362d1..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingTests.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
-import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.junit.Test;
-
-
-/**
- * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
- * implementations
- * 
- * 
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleGroupingTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        AbstractMapperTests<LongWritable, T, NodeWritable, T> {
-
-    /**
-     * Generates data for use in tests
-     * 
-     * @param driver
-     *            Driver
-     * @param num
-     *            Number of tuples to generate
-     */
-    protected void generateData(MapDriver<LongWritable, T, NodeWritable, T> driver, int num) {
-        for (int i = 0; i < num; i++) {
-            LongWritable inputKey = new LongWritable(i);
-            T value = this.createValue(i);
-            NodeWritable outputKey = this.getOutputKey(value);
-
-            driver.addInput(inputKey, value);
-            driver.addOutput(outputKey, value);
-        }
-    }
-
-    protected abstract T createValue(int i);
-
-    protected abstract NodeWritable getOutputKey(T tuple);
-
-    protected final void testGrouping(int num) throws IOException {
-        MapDriver<LongWritable, T, NodeWritable, T> driver = this.getMapDriver();
-        this.generateData(driver, num);
-        driver.runTest();
-    }
-
-    /**
-     * Test grouping tuples by nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void grouping_01() throws IOException {
-        this.testGrouping(1);
-    }
-    
-    /**
-     * Test grouping tuples by nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void grouping_02() throws IOException {
-        this.testGrouping(100);
-    }
-    
-    /**
-     * Test grouping tuples by nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void grouping_03() throws IOException {
-        this.testGrouping(1000);
-    }
-    
-    /**
-     * Test grouping tuples by nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void grouping_04() throws IOException {
-        this.testGrouping(2500);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingTests.java
deleted file mode 100644
index 562512f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingTests.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.mapreduce.group.AbstractQuadGroupingMapper;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for {@link AbstractQuadGroupingMapper} implementations
- * 
- * 
- * 
- */
-public abstract class AbstractQuadGroupingTests extends AbstractNodeTupleGroupingTests<Quad, QuadWritable> {
-
-    @Override
-    protected QuadWritable createValue(int i) {
-        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
-                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingTests.java
deleted file mode 100644
index 8e3d33c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingTests.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.mapreduce.group.AbstractTripleGroupingMapper;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for {@link AbstractTripleGroupingMapper} implementations
- * 
- *
- */
-public abstract class AbstractTripleGroupingTests extends AbstractNodeTupleGroupingTests<Triple, TripleWritable> {
-
-    @Override
-    protected TripleWritable createValue(int i) {
-        return new TripleWritable(
-                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapperTest.java
deleted file mode 100644
index 370f820..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapperTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupByGraphMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Tests for the {@link QuadGroupByGraphMapper}
- * 
- * 
- * 
- */
-public class QuadGroupByGraphMapperTest extends AbstractQuadGroupingTests {
-
-    @Override
-    protected NodeWritable getOutputKey(QuadWritable tuple) {
-        return new NodeWritable(tuple.get().getGraph());
-    }
-
-    @Override
-    protected Mapper<LongWritable, QuadWritable, NodeWritable, QuadWritable> getInstance() {
-        return new QuadGroupByGraphMapper<LongWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapperTest.java
deleted file mode 100644
index 919696d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapperTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupByObjectMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Tests for the {@link QuadGroupByObjectMapper}
- * 
- * 
- * 
- */
-public class QuadGroupByObjectMapperTest extends AbstractQuadGroupingTests {
-
-    @Override
-    protected NodeWritable getOutputKey(QuadWritable tuple) {
-        return new NodeWritable(tuple.get().getObject());
-    }
-
-    @Override
-    protected Mapper<LongWritable, QuadWritable, NodeWritable, QuadWritable> getInstance() {
-        return new QuadGroupByObjectMapper<LongWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapperTest.java
deleted file mode 100644
index 2a1b520..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapperTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupByPredicateMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Tests for the {@link QuadGroupByPredicateMapper}
- * 
- * 
- * 
- */
-public class QuadGroupByPredicateMapperTest extends AbstractQuadGroupingTests {
-
-    @Override
-    protected NodeWritable getOutputKey(QuadWritable tuple) {
-        return new NodeWritable(tuple.get().getPredicate());
-    }
-
-    @Override
-    protected Mapper<LongWritable, QuadWritable, NodeWritable, QuadWritable> getInstance() {
-        return new QuadGroupByPredicateMapper<LongWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapperTest.java
deleted file mode 100644
index 3b0bb1a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapperTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.QuadGroupBySubjectMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Tests for the {@link QuadGroupBySubjectMapper}
- * 
- * 
- * 
- */
-public class QuadGroupBySubjectMapperTest extends AbstractQuadGroupingTests {
-
-    @Override
-    protected NodeWritable getOutputKey(QuadWritable tuple) {
-        return new NodeWritable(tuple.get().getSubject());
-    }
-
-    @Override
-    protected Mapper<LongWritable, QuadWritable, NodeWritable, QuadWritable> getInstance() {
-        return new QuadGroupBySubjectMapper<LongWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapperTest.java
deleted file mode 100644
index c769bb4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapperTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupByObjectMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Tests for the {@link TripleGroupByObjectMapper}
- * 
- * 
- * 
- */
-public class TripleGroupByObjectMapperTest extends AbstractTripleGroupingTests {
-
-    @Override
-    protected NodeWritable getOutputKey(TripleWritable tuple) {
-        return new NodeWritable(tuple.get().getObject());
-    }
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, NodeWritable, TripleWritable> getInstance() {
-        return new TripleGroupByObjectMapper<LongWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapperTest.java
deleted file mode 100644
index e41cf50..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapperTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupByPredicateMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Tests for the {@link TripleGroupByPredicateMapper}
- * 
- * 
- * 
- */
-public class TripleGroupByPredicateMapperTest extends AbstractTripleGroupingTests {
-
-    @Override
-    protected NodeWritable getOutputKey(TripleWritable tuple) {
-        return new NodeWritable(tuple.get().getPredicate());
-    }
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, NodeWritable, TripleWritable> getInstance() {
-        return new TripleGroupByPredicateMapper<LongWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapperTest.java
deleted file mode 100644
index 64335f4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapperTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupBySubjectMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Tests for the {@link TripleGroupBySubjectMapper}
- * 
- * 
- * 
- */
-public class TripleGroupBySubjectMapperTest extends AbstractTripleGroupingTests {
-
-    @Override
-    protected NodeWritable getOutputKey(TripleWritable tuple) {
-        return new NodeWritable(tuple.get().getSubject());
-    }
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, NodeWritable, TripleWritable> getInstance() {
-        return new TripleGroupBySubjectMapper<LongWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesTests.java
deleted file mode 100644
index 1de39cd..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesTests.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
-import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.junit.Test;
-
-
-/**
- * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
- * implementations
- * 
- * 
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleSplitToNodesTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        AbstractMapperTests<LongWritable, T, LongWritable, NodeWritable> {
-
-    /**
-     * Generates data for use in tests
-     * 
-     * @param driver
-     *            Driver
-     * @param num
-     *            Number of tuples to generate
-     */
-    protected void generateData(MapDriver<LongWritable, T, LongWritable, NodeWritable> driver, int num) {
-        for (int i = 0; i < num; i++) {
-            LongWritable key = new LongWritable(i);
-            T value = this.createValue(i);
-            NodeWritable[] nodes = this.getNodes(value);
-
-            driver.addInput(key, value);
-            for (NodeWritable n : nodes) {
-                driver.addOutput(key, n);
-            }
-        }
-    }
-
-    protected abstract T createValue(int i);
-
-    protected abstract NodeWritable[] getNodes(T tuple);
-
-    protected final void testSplitToNodes(int num) throws IOException {
-        MapDriver<LongWritable, T, LongWritable, NodeWritable> driver = this.getMapDriver();
-        this.generateData(driver, num);
-        driver.runTest();
-    }
-
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void split_to_nodes_01() throws IOException {
-        this.testSplitToNodes(1);
-    }
-    
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void split_to_nodes_02() throws IOException {
-        this.testSplitToNodes(100);
-    }
-    
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void split_to_nodes_03() throws IOException {
-        this.testSplitToNodes(1000);
-    }
-    
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void split_to_nodes_04() throws IOException {
-        this.testSplitToNodes(2500);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesTests.java
deleted file mode 100644
index 71136c2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesTests.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
-import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitWithNodesMapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.junit.Test;
-
-
-/**
- * Abstract tests for {@link AbstractNodeTupleSplitWithNodesMapper}
- * implementations
- * 
- * 
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleSplitWithNodesTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        AbstractMapperTests<LongWritable, T, T, NodeWritable> {
-
-    /**
-     * Generates data for use in tests
-     * 
-     * @param driver
-     *            Driver
-     * @param num
-     *            Number of tuples to generate
-     */
-    protected void generateData(MapDriver<LongWritable, T, T, NodeWritable> driver, int num) {
-        for (int i = 0; i < num; i++) {
-            LongWritable key = new LongWritable(i);
-            T value = this.createValue(i);
-            NodeWritable[] nodes = this.getNodes(value);
-
-            driver.addInput(key, value);
-            for (NodeWritable n : nodes) {
-                driver.addOutput(value, n);
-            }
-        }
-    }
-
-    protected abstract T createValue(int i);
-
-    protected abstract NodeWritable[] getNodes(T tuple);
-
-    protected final void testSplitToNodes(int num) throws IOException {
-        MapDriver<LongWritable, T, T, NodeWritable> driver = this.getMapDriver();
-        this.generateData(driver, num);
-        driver.runTest();
-    }
-
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void split_to_nodes_01() throws IOException {
-        this.testSplitToNodes(1);
-    }
-    
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void split_to_nodes_02() throws IOException {
-        this.testSplitToNodes(100);
-    }
-    
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void split_to_nodes_03() throws IOException {
-        this.testSplitToNodes(1000);
-    }
-    
-    /**
-     * Test splitting tuples into their constituent nodes
-     * 
-     * @throws IOException
-     */
-    @Test
-    public final void split_to_nodes_04() throws IOException {
-        this.testSplitToNodes(2500);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitToNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitToNodesTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitToNodesTests.java
deleted file mode 100644
index 07192c6..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitToNodesTests.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
- * implementations that work on Quads
- * 
- * 
- * 
- */
-public abstract class AbstractQuadSplitToNodesTests extends AbstractNodeTupleSplitToNodesTests<Quad, QuadWritable> {
-
-    @Override
-    protected QuadWritable createValue(int i) {
-        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
-                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
-    }
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        Quad q = tuple.get();
-        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
-                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitWithNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitWithNodesTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitWithNodesTests.java
deleted file mode 100644
index 80517b2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractQuadSplitWithNodesTests.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
- * implementations that work on Quads
- * 
- * 
- * 
- */
-public abstract class AbstractQuadSplitWithNodesTests extends AbstractNodeTupleSplitWithNodesTests<Quad, QuadWritable> {
-
-    @Override
-    protected QuadWritable createValue(int i) {
-        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
-                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
-    }
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        Quad q = tuple.get();
-        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
-                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitToNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitToNodesTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitToNodesTests.java
deleted file mode 100644
index 7e497ab..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitToNodesTests.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
- * implementations that work on Triples
- * 
- * 
- * 
- */
-public abstract class AbstractTripleSplitToNodesTests extends AbstractNodeTupleSplitToNodesTests<Triple, TripleWritable> {
-
-    @Override
-    protected TripleWritable createValue(int i) {
-        return new TripleWritable(
-                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        Triple t = tuple.get();
-        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
-                new NodeWritable(t.getObject()) };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitWithNodesTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitWithNodesTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitWithNodesTests.java
deleted file mode 100644
index babcad1..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractTripleSplitWithNodesTests.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import org.apache.jena.hadoop.rdf.mapreduce.split.AbstractNodeTupleSplitToNodesMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for {@link AbstractNodeTupleSplitToNodesMapper}
- * implementations that work on Triples
- * 
- * 
- * 
- */
-public abstract class AbstractTripleSplitWithNodesTests extends AbstractNodeTupleSplitWithNodesTests<Triple, TripleWritable> {
-
-    @Override
-    protected TripleWritable createValue(int i) {
-        return new TripleWritable(
-                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        Triple t = tuple.get();
-        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
-                new NodeWritable(t.getObject()) };
-    }
-
-}

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

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

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


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java b/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
deleted file mode 100644
index a70dfb0..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
+++ /dev/null
@@ -1,406 +0,0 @@
-/*
- * 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.types;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.hadoop.io.WritableComparable;
-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.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Tests for the various RDF types defined by the
- * {@link org.apache.jena.hadoop.rdf.types} package
- * 
- * 
- * 
- */
-public class RdfTypesTest {
-
-    private static final Logger LOG = LoggerFactory.getLogger(RdfTypesTest.class);
-
-    private ByteArrayOutputStream outputStream;
-    private ByteArrayInputStream inputStream;
-
-    /**
-     * Prepare for output
-     * 
-     * @return Data output
-     */
-    private DataOutput prepareOutput() {
-        this.outputStream = new ByteArrayOutputStream();
-        return new DataOutputStream(this.outputStream);
-    }
-
-    /**
-     * Prepare for input from the previously written output
-     * 
-     * @return Data Input
-     */
-    private DataInput prepareInput() {
-        this.inputStream = new ByteArrayInputStream(this.outputStream.toByteArray());
-        return new DataInputStream(this.inputStream);
-    }
-
-    /**
-     * Prepare for input from the given data
-     * 
-     * @param data
-     *            Data
-     * @return Data Input
-     */
-    @SuppressWarnings("unused")
-    private DataInput prepareInput(byte[] data) {
-        this.inputStream = new ByteArrayInputStream(data);
-        return new DataInputStream(this.inputStream);
-    }
-
-    @SuppressWarnings({ "unchecked", "rawtypes" })
-    private <T extends WritableComparable> void testWriteRead(T writable, T expected) throws IOException, InstantiationException, IllegalAccessException,
-            ClassNotFoundException {
-        // Write out data
-        DataOutput output = this.prepareOutput();
-        writable.write(output);
-
-        // Read back in data
-        DataInput input = this.prepareInput();
-        T actual = (T) Class.forName(writable.getClass().getName()).newInstance();
-        actual.readFields(input);
-
-        LOG.info("Original = " + writable.toString());
-        LOG.info("Round Tripped = " + actual.toString());
-
-        // Check equivalent
-        Assert.assertEquals(0, expected.compareTo(actual));
-    }
-    
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_null() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = null;
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-    
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    @Ignore
-    public void node_writable_variable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createVariable("x");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-    
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    @Ignore
-    public void node_writable_variable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createVariable("really-log-variable-name-asddsfr4545egfdgdfgfdgdtgvdg-dfgfdgdfgdfgdfg4-dfvdfgdfgdfgfdgfdgdfgdfgfdg");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_uri_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createURI("http://example.org");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_uri_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createURI("http://user:password@example.org/some/path?key=value#id");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("simple");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("language", "en", null);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_03() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("string", XSDDatatype.XSDstring);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_04() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("1234", XSDDatatype.XSDinteger);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_05() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("123.4", XSDDatatype.XSDdecimal);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_06() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("12.3e4", XSDDatatype.XSDdouble);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_07() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("true", XSDDatatype.XSDboolean);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_bnode_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createAnon();
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_bnode_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createAnon();
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-        NodeWritable nw2 = new NodeWritable(n);
-        testWriteRead(nw2, nw2);
-
-        Assert.assertEquals(0, nw.compareTo(nw2));
-    }
-
-    /**
-     * Basic triple writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void triple_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Triple t = new Triple(NodeFactory.createURI("http://example"), NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral("value"));
-        TripleWritable tw = new TripleWritable(t);
-        testWriteRead(tw, tw);
-    }
-    
-    /**
-     * Basic triple writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void triple_writable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Triple t = new Triple(NodeFactory.createAnon(), NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral("value"));
-        TripleWritable tw = new TripleWritable(t);
-        testWriteRead(tw, tw);
-    }
-
-    /**
-     * Basic quad writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void quad_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Quad q = new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://example"), NodeFactory.createURI("http://predicate"),
-                NodeFactory.createLiteral("value"));
-        QuadWritable qw = new QuadWritable(q);
-        testWriteRead(qw, qw);
-    }
-    
-    /**
-     * Basic quad writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void quad_writable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Quad q = new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createAnon(), NodeFactory.createURI("http://predicate"),
-                NodeFactory.createLiteral("value"));
-        QuadWritable qw = new QuadWritable(q);
-        testWriteRead(qw, qw);
-    }
-
-    /**
-     * Basic tuple writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void tuple_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Tuple<Node> t = Tuple.createTuple(NodeFactory.createURI("http://one"), NodeFactory.createURI("http://two"), NodeFactory.createLiteral("value"),
-                NodeFactory.createLiteral("foo"), NodeFactory.createURI("http://three"));
-        NodeTupleWritable tw = new NodeTupleWritable(t);
-        testWriteRead(tw, tw);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/pom.xml b/jena-hadoop-rdf/jena-elephas-io/pom.xml
deleted file mode 100644
index 2be37f9..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/pom.xml
+++ /dev/null
@@ -1,67 +0,0 @@
-<!--
-   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.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.jena</groupId>
-    <artifactId>jena-elephas</artifactId>
-    <version>0.9.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>jena-elephas-io</artifactId>
-  <name>Apache Jena - Elephas - I/O</name>
-  <description>RDF Input/Output formats library for Hadoop</description>
-
-	<!-- Note that versions are managed by parent POMs -->
-  <dependencies>
-		<!-- Internal Project Dependencies -->
-    <dependency>
-      <groupId>org.apache.jena</groupId>
-      <artifactId>jena-hadoop-rdf-common</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-
-	<!-- Hadoop Dependencies -->
-	<!-- Note these will be provided on the Hadoop cluster hence the provided 
-		scope -->
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-common</artifactId>
-      <scope>provided</scope>
-    </dependency>
-
-	<!-- Jena dependencies -->
-    <dependency>
-      <groupId>org.apache.jena</groupId>
-      <artifactId>jena-arq</artifactId>
-    </dependency>
-
-	<!-- Test Dependencies -->
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java
deleted file mode 100644
index 5c1b41c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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;
-
-/**
- * Hadoop IO related constants
- * 
- * 
- * 
- */
-public class HadoopIOConstants {
-
-    /**
-     * Private constructor prevents instantiation
-     */
-    private HadoopIOConstants() {
-    }
-
-    /**
-     * Map Reduce configuration setting for max line length
-     */
-    public static final String MAX_LINE_LENGTH = "mapreduce.input.linerecordreader.line.maxlength";
-
-    /**
-     * Run ID
-     */
-    public static final String RUN_ID = "runId";
-    
-    /**
-     * Compression codecs to use
-     */
-    public static final String IO_COMPRESSION_CODECS = "io.compression.codecs";
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
deleted file mode 100644
index 27c2bb2..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-
-/**
- * RDF IO related constants
- * 
- * 
- * 
- */
-public class RdfIOConstants {
-
-    /**
-     * Private constructor prevents instantiation
-     */
-    private RdfIOConstants() {
-    }
-
-    /**
-     * Configuration key used to set whether bad tuples are ignored. This is the
-     * default behaviour, when explicitly set to {@code false} bad tuples will
-     * result in {@link IOException} being thrown by the relevant record
-     * readers.
-     */
-    public static final String INPUT_IGNORE_BAD_TUPLES = "rdf.io.input.ignore-bad-tuples";
-
-    /**
-     * Configuration key used to set the batch size used for RDF output formats
-     * that take a batched writing approach. Default value is given by the
-     * constant {@link #DEFAULT_OUTPUT_BATCH_SIZE}.
-     */
-    public static final String OUTPUT_BATCH_SIZE = "rdf.io.output.batch-size";
-
-    /**
-     * Default batch size for batched output formats
-     */
-    public static final long DEFAULT_OUTPUT_BATCH_SIZE = 10000;
-
-    /**
-     * Configuration key used to control behaviour with regards to how blank
-     * nodes are handled.
-     * <p>
-     * The default behaviour is that blank nodes are file scoped which is what
-     * the RDF specifications require.
-     * </p>
-     * <p>
-     * However in the case of a multi-stage pipeline this behaviour can cause
-     * blank nodes to diverge over several jobs and introduce spurious blank
-     * nodes over time. This is described in <a
-     * href="https://issues.apache.org/jira/browse/JENA-820">JENA-820</a> and
-     * enabling this flag for jobs in your pipeline allow you to work around
-     * this problem.
-     * </p>
-     * <h3>Warning</h3> You should only enable this flag for jobs that take in
-     * RDF output originating from previous jobs since our normal blank node
-     * allocation policy ensures that blank nodes will be file scoped and unique
-     * over all files (barring unfortunate hasing collisions). If you enable
-     * this for jobs that take in RDF originating from other sources you may
-     * incorrectly conflate blank nodes that are supposed to distinct and
-     * separate nodes.
-     */
-    public static final String GLOBAL_BNODE_IDENTITY = "rdf.io.input.bnodes.global-identity";
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
deleted file mode 100644
index 1fcb030..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Abstract line based input format that reuses the machinery from
- * {@link NLineInputFormat} to calculate the splits
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Value type
- */
-public abstract class AbstractNLineFileInputFormat<TKey, TValue> extends FileInputFormat<TKey, TValue> {
-    
-    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractNLineFileInputFormat.class);
-
-    /**
-     * Logically splits the set of input files for the job, splits N lines of
-     * the input as one split.
-     * 
-     * @see FileInputFormat#getSplits(JobContext)
-     */
-    public final List<InputSplit> getSplits(JobContext job) throws IOException {
-        boolean debug = LOGGER.isDebugEnabled();
-        if (debug && FileInputFormat.getInputDirRecursive(job)) {
-            LOGGER.debug("Recursive searching for input data is enabled");
-        }
-        
-        List<InputSplit> splits = new ArrayList<InputSplit>();
-        int numLinesPerSplit = NLineInputFormat.getNumLinesPerSplit(job);
-        for (FileStatus status : listStatus(job)) {
-            if (debug) {
-                LOGGER.debug("Determining how to split input file/directory {}", status.getPath());
-            }
-            splits.addAll(NLineInputFormat.getSplitsForFile(status, job.getConfiguration(), numLinesPerSplit));
-        }
-        return splits;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
deleted file mode 100644
index e561cdb..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.fs.Path;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-
-/**
- * Abstract implementation of a while file input format where each file is a
- * single split
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Value type
- */
-public abstract class AbstractWholeFileInputFormat<TKey, TValue> extends FileInputFormat<TKey, TValue> {
-
-    @Override
-    protected final boolean isSplitable(JobContext context, Path filename) {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
deleted file mode 100644
index b8fdbd5..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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 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.QuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * RDF input format that can handle any RDF quads format that ARQ supports
- * selecting the format to use for each file based upon the file extension
- * 
- * 
- * 
- */
-public class QuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new QuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
deleted file mode 100644
index 03f394a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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 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.TriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-/**
- * RDF input format that can handle any RDF triples format that ARQ supports
- * selecting the format to use for each file based upon the file extension
- */
-public class TriplesInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TriplesReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
deleted file mode 100644
index bfd643e..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 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.TriplesOrQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * RDF input format that can handle any RDF triple/quads format that ARQ
- * supports selecting the format to use for each file based upon the file
- * extension. Triples are converted into quads in the default graph.
- * 
- * 
- * 
- */
-public class TriplesOrQuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TriplesOrQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
deleted file mode 100644
index 2464946..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.jsonld;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
deleted file mode 100644
index 0e08a4b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.jsonld;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
deleted file mode 100644
index 6829c4d..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.nquads;
-
-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.AbstractNLineFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.nquads.BlockedNQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * NTriples input format where files are processed as blocks of lines rather
- * than in a line based manner as with the {@link NQuadsInputFormat} or as
- * whole files with the {@link WholeFileNQuadsInputFormat}
- * <p>
- * This provides a compromise between the higher parser setup of creating more
- * parsers and the benefit of being able to split input files over multiple
- * mappers.
- * </p>
- * 
- * 
- * 
- */
-public class BlockedNQuadsInputFormat extends AbstractNLineFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new BlockedNQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
deleted file mode 100644
index 802fbea..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.nquads;
-
-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.AbstractNLineFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.nquads.NQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * NQuads input format
- * 
- * 
- * 
- */
-public class NQuadsInputFormat extends AbstractNLineFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit arg0, TaskAttemptContext arg1)
-            throws IOException, InterruptedException {
-        return new NQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java
deleted file mode 100644
index 128d079..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.nquads;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.nquads.WholeFileNQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * NQuads input format where files are processed as complete files rather than
- * in a line based manner as with the {@link NQuadsInputFormat}
- * <p>
- * This has the advantage of less parser setup overhead but the disadvantage
- * that the input cannot be split over multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class WholeFileNQuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new WholeFileNQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
deleted file mode 100644
index 292167b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.ntriples;
-
-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.AbstractNLineFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.BlockedNTriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * NTriples input format where files are processed as blocks of lines rather
- * than in a line based manner as with the {@link NTriplesInputFormat} or as
- * whole files with the {@link WholeFileNTriplesInputFormat}
- * <p>
- * This provides a compromise between the higher parser setup of creating more
- * parsers and the benefit of being able to split input files over multiple
- * mappers.
- * </p>
- * 
- * 
- * 
- */
-public class BlockedNTriplesInputFormat extends AbstractNLineFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new BlockedNTriplesReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
deleted file mode 100644
index 1694c87..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.ntriples;
-
-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.AbstractNLineFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.NTriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * NTriples input format
- * 
- * 
- * 
- */
-public class NTriplesInputFormat extends AbstractNLineFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit inputSplit, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new NTriplesReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java
deleted file mode 100644
index 31c1252..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.ntriples;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.WholeFileNTriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * NTriples input format where files are processed as complete files rather than
- * in a line based manner as with the {@link NTriplesInputFormat}
- * <p>
- * This has the advantage of less parser setup overhead but the disadvantage
- * that the input cannot be split over multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class WholeFileNTriplesInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new WholeFileNTriplesReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
deleted file mode 100644
index e5a7940..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.rdfjson;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.rdfjson.RdfJsonReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * RDF/JSON input format
- * 
- * 
- * 
- */
-public class RdfJsonInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new RdfJsonReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
deleted file mode 100644
index 4deb925..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.rdfxml;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.rdfxml.RdfXmlReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * RDF/XML input format
- * 
- * 
- * 
- */
-public class RdfXmlInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new RdfXmlReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
deleted file mode 100644
index 56d031e..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
+++ /dev/null
@@ -1,344 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.util.BlockInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFDataMgr;
-import org.apache.jena.riot.ReaderRIOT;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-import org.apache.jena.riot.lang.PipedRDFStream;
-import org.apache.jena.riot.system.ParserProfile;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract implementation for a record reader that reads records from blocks
- * of files, this is a hybrid between {@link AbstractLineBasedNodeTupleReader}
- * and {@link AbstractWholeFileNodeTupleReader} in that it can only be used by
- * formats which can be split by lines but reduces the overhead by parsing the
- * split as a whole rather than as individual lines.
- * <p>
- * The keys produced are the approximate position in the file at which a tuple
- * was found and the values will be node tuples. Positions are approximate
- * because they are recorded after the point at which the most recent tuple was
- * parsed from the input thus they reflect the approximate position in the
- * stream immediately after which the triple was found.
- * </p>
- * 
- * 
- * 
- * @param <TValue>
- *            Value type
- * @param <T>
- *            Tuple type
- */
-public abstract class AbstractBlockBasedNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractBlockBasedNodeTupleReader.class);
-    private CompressionCodec compressionCodecs;
-    private TrackableInputStream input;
-    private LongWritable key;
-    private long start, length;
-    private T tuple;
-    private TrackedPipedRDFStream<TValue> stream;
-    private PipedRDFIterator<TValue> iter;
-    private Thread parserThread;
-    private boolean finished = false;
-    private boolean ignoreBadTuples = true;
-    private boolean parserFinished = false;
-    private Throwable parserError = null;
-
-    @Override
-    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
-        LOG.debug("initialize({}, {})", genericSplit, context);
-
-        // Assuming file split
-        if (!(genericSplit instanceof FileSplit))
-            throw new IOException("This record reader only supports FileSplit inputs");
-        FileSplit split = (FileSplit) genericSplit;
-
-        // Configuration
-        Configuration config = context.getConfiguration();
-        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
-        if (this.ignoreBadTuples)
-            LOG.warn(
-                    "Configured to ignore bad tuples, parsing errors will be logged and further parsing aborted but no user visible errors will be thrown.  Consider setting {} to false to disable this behaviour",
-                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
-
-        // Figure out what portion of the file to read
-        start = split.getStart();
-        long end = start + split.getLength();
-        final Path file = split.getPath();
-        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
-        boolean readToEnd = end == totalLength;
-        CompressionCodecFactory factory = new CompressionCodecFactory(config);
-        this.compressionCodecs = factory.getCodec(file);
-
-        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { start, split.getLength(), totalLength }));
-
-        // Open the file and prepare the input stream
-        FileSystem fs = file.getFileSystem(config);
-        FSDataInputStream fileIn = fs.open(file);
-        this.length = split.getLength();
-        if (start > 0)
-            fileIn.seek(start);
-
-        if (this.compressionCodecs != null) {
-            // Compressed input
-            // For compressed input NLineInputFormat will have failed to find
-            // any line breaks and will give us a split from 0 -> (length - 1)
-            // Add 1 and re-verify readToEnd so we can abort correctly if ever
-            // given a partial split of a compressed file
-            end++;
-            readToEnd = end == totalLength;
-            if (start > 0 || !readToEnd)
-                throw new IOException("This record reader can only be used with compressed input where the split is a whole file");
-            input = new TrackedInputStream(this.compressionCodecs.createInputStream(fileIn));
-        } else {
-            // Uncompressed input
-
-            if (readToEnd) {
-                input = new TrackedInputStream(fileIn);
-            } else {
-                // Need to limit the portion of the file we are reading
-                input = new BlockInputStream(fileIn, split.getLength());
-            }
-        }
-
-        // Set up background thread for parser
-        iter = this.getPipedIterator();
-        this.stream = this.getPipedStream(iter, this.input);
-        ParserProfile profile = RdfIOUtils.createParserProfile(context, file);
-        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage(), profile);
-        this.parserThread = new Thread(parserRunnable);
-        this.parserThread.setDaemon(true);
-        this.parserThread.start();
-    }
-
-    /**
-     * Gets the RDF iterator to use
-     * 
-     * @return Iterator
-     */
-    protected abstract PipedRDFIterator<TValue> getPipedIterator();
-
-    /**
-     * Gets the RDF stream to parse to
-     * 
-     * @param iterator
-     *            Iterator
-     * @return RDF stream
-     */
-    protected abstract TrackedPipedRDFStream<TValue> getPipedStream(PipedRDFIterator<TValue> iterator, TrackableInputStream input);
-
-    /**
-     * Gets the RDF language to use for parsing
-     * 
-     * @return
-     */
-    protected abstract Lang getRdfLanguage();
-
-    /**
-     * Creates the runnable upon which the parsing will run
-     * 
-     * @param input
-     *            Input
-     * @param stream
-     *            Stream
-     * @param lang
-     *            Language to use for parsing
-     * @return Parser runnable
-     */
-    private Runnable createRunnable(@SuppressWarnings("rawtypes") final AbstractBlockBasedNodeTupleReader reader, final InputStream input,
-            final PipedRDFStream<TValue> stream, final Lang lang, final ParserProfile profile) {
-        return new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    ReaderRIOT riotReader = RDFDataMgr.createReader(lang);
-                    riotReader.setParserProfile(profile);
-                    riotReader.read(input, null, lang.getContentType(), stream, null);
-                    //RDFDataMgr.parse(stream, input, null, lang);
-                    reader.setParserFinished(null);
-                } catch (Throwable e) {
-                    reader.setParserFinished(e);
-                }
-            }
-        };
-    }
-
-    /**
-     * Sets the parser thread finished state
-     * 
-     * @param e
-     *            Error (if any)
-     */
-    private void setParserFinished(Throwable e) {
-        synchronized (this.parserThread) {
-            this.parserError = e;
-            this.parserFinished = true;
-        }
-    }
-
-    /**
-     * Waits for the parser thread to have reported as finished
-     * 
-     * @throws InterruptedException
-     */
-    private void waitForParserFinished() throws InterruptedException {
-        do {
-            synchronized (this.parserThread) {
-                if (this.parserFinished)
-                    return;
-            }
-            Thread.sleep(50);
-        } while (true);
-    }
-
-    /**
-     * Creates an instance of a writable tuple from the given tuple value
-     * 
-     * @param tuple
-     *            Tuple value
-     * @return Writable tuple
-     */
-    protected abstract T createInstance(TValue tuple);
-
-    @Override
-    public boolean nextKeyValue() throws IOException, InterruptedException {
-        // Reuse key for efficiency
-        if (key == null) {
-            key = new LongWritable();
-        }
-
-        if (this.finished)
-            return false;
-
-        try {
-            if (this.iter.hasNext()) {
-                // Position will be relative to the start for the split we're
-                // processing
-                Long l = this.start + this.stream.getPosition();
-                if (l != null) {
-                    this.key.set(l);
-                    // For compressed input the actual length from which we
-                    // calculate progress is likely less than the actual
-                    // uncompressed length so we need to increment the
-                    // length as we go along
-                    // We always add 1 more than the current length because we
-                    // don't want to report 100% progress until we really have
-                    // finished
-                    if (this.compressionCodecs != null && l > this.length)
-                        this.length = l + 1;
-                }
-                this.tuple = this.createInstance(this.iter.next());
-                return true;
-            } else {
-                // Need to ensure that the parser thread has finished in order
-                // to determine whether we finished without error
-                this.waitForParserFinished();
-                if (this.parserError != null) {
-                    LOG.error("Error parsing block, aborting further parsing", this.parserError);
-                    if (!this.ignoreBadTuples)
-                        throw new IOException("Error parsing block at position " + (this.start + this.input.getBytesRead()) + ", aborting further parsing",
-                                this.parserError);
-                }
-
-                this.key = null;
-                this.tuple = null;
-                this.finished = true;
-                // This is necessary so that when compressed input is used we
-                // report 100% progress once we've reached the genuine end of
-                // the stream
-                if (this.compressionCodecs != null)
-                    this.length--;
-                return false;
-            }
-        } catch (IOException e) {
-            throw e;
-        } catch (Throwable e) {
-            // Failed to read the tuple on this line
-            LOG.error("Error parsing block, aborting further parsing", e);
-            if (!this.ignoreBadTuples) {
-                this.iter.close();
-                throw new IOException("Error parsing block at position " + (this.start + this.input.getBytesRead()) + ", aborting further parsing", e);
-            }
-            this.key = null;
-            this.tuple = null;
-            this.finished = true;
-            return false;
-        }
-    }
-
-    @Override
-    public LongWritable getCurrentKey() throws IOException, InterruptedException {
-        return this.key;
-    }
-
-    @Override
-    public T getCurrentValue() throws IOException, InterruptedException {
-        return this.tuple;
-    }
-
-    @Override
-    public float getProgress() throws IOException, InterruptedException {
-        float progress = 0.0f;
-        if (this.key == null) {
-            // We've either not started or we've finished
-            progress = (this.finished ? 1.0f : 0.0f);
-        } else if (this.key.get() == Long.MIN_VALUE) {
-            // We don't have a position so we've either in-progress or finished
-            progress = (this.finished ? 1.0f : 0.5f);
-        } else {
-            // We're some way through the file
-            progress = (this.key.get() - this.start) / (float) this.length;
-        }
-        LOG.debug("getProgress() --> {}", progress);
-        return progress;
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.iter.close();
-        this.input.close();
-        this.finished = true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
deleted file mode 100644
index 2279444..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedQuadsStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract record reader for whole file triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractBlockBasedQuadReader extends AbstractBlockBasedNodeTupleReader<Quad, QuadWritable> {
-
-    @Override
-    protected PipedRDFIterator<Quad> getPipedIterator() {
-        return new PipedRDFIterator<Quad>();
-    }
-
-    @Override
-    protected TrackedPipedRDFStream<Quad> getPipedStream(PipedRDFIterator<Quad> iterator, TrackableInputStream input) {
-        return new TrackedPipedQuadsStream(iterator, input);
-    }
-
-    @Override
-    protected QuadWritable createInstance(Quad tuple) {
-        return new QuadWritable(tuple);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
deleted file mode 100644
index 2afd329..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedTriplesStream;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * An abstract record reader for whole file triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractBlockBasedTripleReader extends AbstractBlockBasedNodeTupleReader<Triple, TripleWritable> {
-
-    @Override
-    protected PipedRDFIterator<Triple> getPipedIterator() {
-        return new PipedRDFIterator<Triple>();
-    }
-
-    @Override
-    protected TrackedPipedRDFStream<Triple> getPipedStream(PipedRDFIterator<Triple> iterator, TrackableInputStream input) {
-        return new TrackedPipedTriplesStream(iterator, input);
-    }
-
-    @Override
-    protected TripleWritable createInstance(Triple tuple) {
-        return new TripleWritable(tuple);
-    }
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java
new file mode 100644
index 0000000..d6ac375
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A triple filter which selects triples which have matching subjects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleFilterBySubjectUriMapper<TKey> extends AbstractTripleFilterByPositionMapper<TKey> {
+
+    private List<Node> subjects = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the subject URIs we are filtering on
+        String[] subjectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_SUBJECT_URIS);
+        if (subjectUris != null) {
+            for (String subjectUri : subjectUris) {
+                this.subjects.add(NodeFactory.createURI(subjectUri));
+            }
+        }
+    }
+
+    @Override
+    protected boolean acceptsSubject(Node subject) {
+        if (this.subjects.size() == 0)
+            return false;
+        return this.subjects.contains(subject);
+    }
+
+    @Override
+    protected boolean acceptsAllPredicates() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllObjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java
new file mode 100644
index 0000000..e3d51e4
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * Abstract mapper implementation which helps in grouping tuples by assigning
+ * them a {@link NodeWritable} key in place of their existing key. Derived
+ * implementations of this may select the key based on some component of the
+ * tuple or by other custom logic.
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleGroupingMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, NodeWritable, T> {
+
+    @Override
+    protected final void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        NodeWritable newKey = this.selectKey(value);
+        context.write(newKey, value);
+    }
+
+    /**
+     * Gets the key to associated with the tuple
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Node to use as key
+     */
+    protected abstract NodeWritable selectKey(T tuple);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java
new file mode 100644
index 0000000..2b96110
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract mapper implementation which helps in grouping quads by assigning
+ * them a {@link NodeWritable} key in place of their existing key. Derived
+ * implementations of this may select the key based on some component of the
+ * quad or by other custom logic.
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public abstract class AbstractQuadGroupingMapper<TKey> extends AbstractNodeTupleGroupingMapper<TKey, Quad, QuadWritable> {
+
+    protected final NodeWritable selectKey(QuadWritable tuple) {
+        return this.selectKey(tuple.get());
+    }
+
+    /**
+     * Selects the key to use
+     * 
+     * @param quad
+     *            Quad
+     * @return Key to use
+     */
+    protected abstract NodeWritable selectKey(Quad quad);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java
new file mode 100644
index 0000000..3f44eb0
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract mapper implementation which helps in grouping triples by assigning
+ * them a {@link NodeWritable} key in place of their existing key. Derived
+ * implementations of this may select the key based on some component of the
+ * triple or by other custom logic.
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public abstract class AbstractTripleGroupingMapper<TKey> extends AbstractNodeTupleGroupingMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected final NodeWritable selectKey(TripleWritable tuple) {
+        return this.selectKey(tuple.get());
+    }
+    
+    protected abstract NodeWritable selectKey(Triple triple);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java
new file mode 100644
index 0000000..3b9fd8d
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which assists in grouping quads by graph by reassigning their keys
+ * to be their graphs
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class QuadGroupByGraphMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Quad quad) {
+        return new NodeWritable(quad.getGraph());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java
new file mode 100644
index 0000000..eb26e0b
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which assists in grouping quads by object by reassigning their keys
+ * to be their objects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class QuadGroupByObjectMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Quad quad) {
+        return new NodeWritable(quad.getObject());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java
new file mode 100644
index 0000000..2670cf4
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which assists in grouping quads by predicate by reassigning their keys
+ * to be their predicates
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class QuadGroupByPredicateMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Quad quad) {
+        return new NodeWritable(quad.getPredicate());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java
new file mode 100644
index 0000000..73809e8
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which assists in grouping quads by subject by reassigning their keys
+ * to be their subjects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class QuadGroupBySubjectMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Quad quad) {
+        return new NodeWritable(quad.getSubject());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java
new file mode 100644
index 0000000..9fde939
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which assists in grouping triples by object by reassigning their
+ * keys to be their objects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleGroupByObjectMapper<TKey> extends AbstractTripleGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Triple triple) {
+        return new NodeWritable(triple.getObject());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java
new file mode 100644
index 0000000..dd15ef5
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which assists in grouping triples by predicate by reassigning their
+ * keys to be their predicates
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleGroupByPredicateMapper<TKey> extends AbstractTripleGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Triple triple) {
+        return new NodeWritable(triple.getPredicate());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java
new file mode 100644
index 0000000..f1116c1
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.group;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which assists in grouping triples by subject by reassigning their
+ * keys to be their subjects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleGroupBySubjectMapper<TKey> extends AbstractTripleGroupingMapper<TKey> {
+
+    @Override
+    protected NodeWritable selectKey(Triple triple) {
+        return new NodeWritable(triple.getSubject());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java
new file mode 100644
index 0000000..840d78c
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * Abstract mapper implementation which splits the tuples into their constituent
+ * nodes preserving the keys as-is
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleSplitToNodesMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, TKey, NodeWritable> {
+
+    @Override
+    protected final void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        NodeWritable[] ns = this.split(value);
+        for (NodeWritable n : ns) {
+            context.write(key, n);
+        }
+    }
+
+    /**
+     * Splits the node tuple type into the individual nodes
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] split(T tuple);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java
new file mode 100644
index 0000000..7dc85fd
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * Abstract mapper implementation which splits the tuples into their constituent
+ * nodes using the tuples as the keys and the nodes as the values
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleSplitWithNodesMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, T, NodeWritable> {
+
+    @Override
+    protected void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        NodeWritable[] ns = this.split(value);
+        for (NodeWritable n : ns) {
+            context.write(value, n);
+        }
+    }
+
+    /**
+     * Splits the node tuple type into the individual nodes
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] split(T tuple);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java
new file mode 100644
index 0000000..c993810
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which splits quads into their constituent nodes preserving the
+ * existing keys as-is
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadSplitToNodesMapper<TKey> extends AbstractNodeTupleSplitToNodesMapper<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected NodeWritable[] split(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java
new file mode 100644
index 0000000..09caef6
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which splits quads into their constituent nodes using the quad as
+ * the key and the nodes as the values
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadSplitWithNodesMapper<TKey> extends AbstractNodeTupleSplitWithNodesMapper<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected NodeWritable[] split(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java
new file mode 100644
index 0000000..0ef02d9
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which splits triples into their constituent nodes
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleSplitToNodesMapper<TKey> extends AbstractNodeTupleSplitToNodesMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected NodeWritable[] split(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java
new file mode 100644
index 0000000..7b18f55
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.split;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which splits triples into their constituent nodes
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleSplitWithNodesMapper<TKey> extends AbstractNodeTupleSplitWithNodesMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected NodeWritable[] split(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java
new file mode 100644
index 0000000..76137fe
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.transform;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract mapper which transforms triples into quads. Derived
+ * implementations may choose how the graph to which triples are assigned is
+ * decided.
+ * <p>
+ * Keys are left as is by this mapper.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public abstract class AbstractTriplesToQuadsMapper<TKey> extends Mapper<TKey, TripleWritable, TKey, QuadWritable> {
+
+    @Override
+    protected final void map(TKey key, TripleWritable value, Context context) throws IOException, InterruptedException {
+        Triple triple = value.get();
+        Node graphNode = this.selectGraph(triple);
+        context.write(key, new QuadWritable(new Quad(graphNode, triple)));
+    }
+
+    /**
+     * Selects the graph name to use for converting the given triple into a quad
+     * 
+     * @param triple
+     *            Triple
+     * @return Tuple
+     */
+    protected abstract Node selectGraph(Triple triple);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java
new file mode 100644
index 0000000..048e669
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.transform;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * A mapper which transforms quads into triples
+ * <p>
+ * Keys are left as is by this mapper.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadsToTriplesMapper<TKey> extends Mapper<TKey, QuadWritable, TKey, TripleWritable> {
+
+    @Override
+    protected void map(TKey key, QuadWritable value, Context context) throws IOException, InterruptedException {
+        context.write(key, new TripleWritable(value.get().asTriple()));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java
new file mode 100644
index 0000000..394d5fd
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.transform;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper which converts triples into quads using the subjects of the triples
+ * as the graph nodes
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * 
+ */
+public class TriplesToQuadsBySubjectMapper<TKey> extends AbstractTriplesToQuadsMapper<TKey> {
+
+    @Override
+    protected final Node selectGraph(Triple triple) {
+        return triple.getSubject();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapper.java
new file mode 100644
index 0000000..ef19edf
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapper.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.transform;
+
+import java.io.IOException;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper which converts triples to quads where all triples are placed in the
+ * same graph
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TriplesToQuadsConstantGraphMapper<TKey> extends AbstractTriplesToQuadsMapper<TKey> {
+
+    private Node graphNode;
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.graphNode = this.getGraphNode();
+    }
+
+    /**
+     * Gets the graph node that will be used for all quads, this will be called
+     * once and only once during the
+     * {@link #setup(org.apache.hadoop.mapreduce.Mapper.Context)} method and the
+     * value returned cached for use throughout the lifetime of this mapper.
+     * <p>
+     * This implementation always used the default graph as the graph for
+     * generated quads. You can override this method in your own derived
+     * implementation to put triples into a different graph than the default
+     * graph.
+     * </p>
+     * <p>
+     * If instead you wanted to select different graphs for each triple you
+     * should extend {@link AbstractTriplesToQuadsMapper} instead and override
+     * the {@link #selectGraph(Triple)} method which is sealed in this
+     * implementation.
+     * </p>
+     * 
+     * @return
+     */
+    protected Node getGraphNode() {
+        return Quad.defaultGraphNodeGenerated;
+    }
+
+    @Override
+    protected final Node selectGraph(Triple triple) {
+        return this.graphNode;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapReduceTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapReduceTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapReduceTests.java
new file mode 100644
index 0000000..32c40f7
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapReduceTests.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
+
+/**
+ * Abstract tests for mappers
+ * 
+ * 
+ * @param <TKey>
+ *            Mapper input key type
+ * @param <TValue>
+ *            Mapper input value type
+ * @param <TIntermediateKey>
+ *            Mapper output/Reducer input key type
+ * @param <TIntermediateValue>
+ *            Mapper output/Reducer input value type
+ * @param <TReducedKey>
+ *            Reducer output key type
+ * @param <TReducedValue>
+ *            Reducer output value type
+ * 
+ * 
+ */
+public abstract class AbstractMapReduceTests<TKey, TValue, TIntermediateKey, TIntermediateValue, TReducedKey, TReducedValue> {
+
+    /**
+     * Gets the mapper instance to test
+     * 
+     * @return Mapper instance
+     */
+    protected abstract Mapper<TKey, TValue, TIntermediateKey, TIntermediateValue> getMapperInstance();
+
+    /**
+     * Gets the reducer instance to test
+     * 
+     * @return Reducer instance
+     */
+    protected abstract Reducer<TIntermediateKey, TIntermediateValue, TReducedKey, TReducedValue> getReducerInstance();
+
+    /**
+     * Gets a map reduce driver that can be used to create a test case
+     * 
+     * @return Map reduce driver
+     */
+    protected MapReduceDriver<TKey, TValue, TIntermediateKey, TIntermediateValue, TReducedKey, TReducedValue> getMapReduceDriver() {
+        return new MapReduceDriver<TKey, TValue, TIntermediateKey, TIntermediateValue, TReducedKey, TReducedValue>(
+                this.getMapperInstance(), this.getReducerInstance());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapperTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapperTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapperTests.java
new file mode 100644
index 0000000..ce6ab9d
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapperTests.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mrunit.mapreduce.MapDriver;
+
+/**
+ * Abstract tests for mappers
+ * 
+ * 
+ * @param <TKeyIn>
+ *            Input key type
+ * @param <TValueIn>
+ *            Input value type
+ * @param <TKeyOut>
+ *            Output key type
+ * @param <TValueOut>
+ *            Output value type
+ * 
+ */
+public abstract class AbstractMapperTests<TKeyIn, TValueIn, TKeyOut, TValueOut> {
+
+    /**
+     * Gets the mapper instance to test
+     * 
+     * @return Mapper instance
+     */
+    protected abstract Mapper<TKeyIn, TValueIn, TKeyOut, TValueOut> getInstance();
+
+    /**
+     * Gets a map driver that can be used to create a test case
+     * 
+     * @return Map driver
+     */
+    protected MapDriver<TKeyIn, TValueIn, TKeyOut, TValueOut> getMapDriver() {
+        MapDriver<TKeyIn, TValueIn, TKeyOut, TValueOut> driver = new MapDriver<TKeyIn, TValueIn, TKeyOut, TValueOut>(
+                this.getInstance());
+        this.configureDriver(driver);
+        return driver;
+    }
+
+    /**
+     * Method that may be overridden by test harnesses which need to configure
+     * the driver in more detail e.g. add configuration keys
+     * 
+     * @param driver
+     *            Driver
+     */
+    protected void configureDriver(MapDriver<TKeyIn, TValueIn, TKeyOut, TValueOut> driver) {
+        // Does nothing
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/TestDistinctTriples.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/TestDistinctTriples.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/TestDistinctTriples.java
new file mode 100644
index 0000000..af32dac
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/TestDistinctTriples.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+public class TestDistinctTriples
+        extends
+        AbstractMapReduceTests<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> {
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, TripleWritable, NullWritable> getMapperInstance() {
+        return new ValuePlusNullMapper<LongWritable, TripleWritable>();
+    }
+
+    @Override
+    protected Reducer<TripleWritable, NullWritable, NullWritable, TripleWritable> getReducerInstance() {
+        return new NullPlusKeyReducer<TripleWritable, NullWritable>();
+    }
+
+    @Test
+    public void distinct_triples_01() throws IOException {
+        MapReduceDriver<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> driver = this
+                .getMapReduceDriver();
+
+        Triple t = new Triple(NodeFactory.createURI("urn:s"), NodeFactory.createURI("urn:p"),
+                NodeFactory.createLiteral("1"));
+        TripleWritable tw = new TripleWritable(t);
+        driver.addInput(new LongWritable(1), tw);
+        driver.addOutput(NullWritable.get(), tw);
+
+        driver.runTest();
+    }
+
+    @Test
+    public void distinct_triples_02() throws IOException {
+        MapReduceDriver<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> driver = this
+                .getMapReduceDriver();
+
+        Triple t = new Triple(NodeFactory.createURI("urn:s"), NodeFactory.createURI("urn:p"),
+                NodeFactory.createLiteral("1"));
+        TripleWritable tw = new TripleWritable(t);
+        for (int i = 0; i < 100; i++) {
+            driver.addInput(new LongWritable(i), tw);
+        }
+        driver.addOutput(NullWritable.get(), tw);
+
+        driver.runTest();
+    }
+
+    @Test
+    public void distinct_triples_03() throws IOException {
+        MapReduceDriver<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> driver = this
+                .getMapReduceDriver();
+
+        Triple t = new Triple(NodeFactory.createURI("urn:s"), NodeFactory.createURI("urn:p"),
+                NodeFactory.createLiteral("1"));
+        Triple t2 = new Triple(t.getSubject(), t.getPredicate(), NodeFactory.createLiteral("2"));
+        Assert.assertNotEquals(t, t2);
+
+        TripleWritable tw = new TripleWritable(t);
+        TripleWritable tw2 = new TripleWritable(t2);
+        Assert.assertNotEquals(tw, tw2);
+
+        driver.addInput(new LongWritable(1), tw);
+        driver.addInput(new LongWritable(2), tw2);
+        driver.addOutput(NullWritable.get(), tw);
+        driver.addOutput(NullWritable.get(), tw2);
+
+        driver.runTest(false);
+    }
+
+    @Test
+    public void distinct_triples_04() throws IOException {
+        MapReduceDriver<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> driver = this
+                .getMapReduceDriver();
+
+        Node s1 = NodeFactory.createURI("urn:nf#cbf2b2c7-109e-4097-bbea-f67f272c7fcc");
+        Node s2 = NodeFactory.createURI("urn:nf#bb08b75c-1ad2-47ef-acd2-eb2d92b94b89");
+        Node p = NodeFactory.createURI("urn:p");
+        Node o = NodeFactory.createURI("urn:66.230.159.118");
+        Assert.assertNotEquals(s1, s2);
+
+        Triple t1 = new Triple(s1, p, o);
+        Triple t2 = new Triple(s2, p, o);
+        Assert.assertNotEquals(t1, t2);
+
+        TripleWritable tw1 = new TripleWritable(t1);
+        TripleWritable tw2 = new TripleWritable(t2);
+        Assert.assertNotEquals(tw1, tw2);
+        Assert.assertNotEquals(0, tw1.compareTo(tw2));
+
+        driver.addInput(new LongWritable(1), tw1);
+        driver.addInput(new LongWritable(2), tw2);
+        driver.addOutput(NullWritable.get(), tw1);
+        driver.addOutput(NullWritable.get(), tw2);
+
+        driver.runTest(false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducerTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducerTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducerTests.java
new file mode 100644
index 0000000..b2d0b92
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducerTests.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapReduceTests;
+import org.apache.jena.hadoop.rdf.mapreduce.characteristics.AbstractCharacteristicSetGeneratingReducer;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
+import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.junit.Test;
+
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * Abstract tests for the {@link AbstractCharacteristicSetGeneratingReducer}
+ * 
+ * 
+ * 
+ * @param <TValue>
+ * @param <T>
+ */
+public abstract class AbstractCharacteristicSetGeneratingReducerTests<TValue, T extends AbstractNodeTupleWritable<TValue>>
+        extends AbstractMapReduceTests<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> {
+
+    /**
+     * Create a tuple
+     * 
+     * @param i
+     *            Key to use in creating the subject
+     * @param predicateUri
+     *            Predicate URI string
+     * @return Tuple
+     */
+    protected abstract T createTuple(int i, String predicateUri);
+
+    /**
+     * Creates a set consisting of the given predicates
+     * 
+     * @param predicates
+     *            Predicates
+     * @return Set
+     */
+    protected CharacteristicSetWritable createSet(MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver, int occurrences, String... predicates) {
+        CharacteristicSetWritable set = new CharacteristicSetWritable();
+        for (String predicateUri : predicates) {
+            set.add(new CharacteristicWritable(NodeFactory.createURI(predicateUri)));
+        }
+        for (int i = 1; i <= occurrences; i++) {
+            driver.addOutput(set, NullWritable.get());
+        }
+        return set;
+    }
+
+    /**
+     * Test basic characteristic set computation
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_generating_reducer_01() throws IOException {
+        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+        T tuple = this.createTuple(1, "http://predicate");
+        driver.addInput(new LongWritable(1), tuple);
+
+        this.createSet(driver, 1, "http://predicate");
+
+        driver.runTest(false);
+    }
+
+    /**
+     * Test basic characteristic set computation
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_generating_reducer_02() throws IOException {
+        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+        T tuple = this.createTuple(1, "http://predicate");
+        driver.addInput(new LongWritable(1), tuple);
+        driver.addInput(new LongWritable(1), tuple);
+
+        this.createSet(driver, 1, "http://predicate");
+
+        driver.runTest(false);
+    }
+
+    /**
+     * Test basic characteristic set computation
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_generating_reducer_03() throws IOException {
+        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+        T tuple = this.createTuple(1, "http://predicate");
+        driver.addInput(new LongWritable(1), tuple);
+        tuple = this.createTuple(2, "http://predicate");
+        driver.addInput(new LongWritable(2), tuple);
+
+        this.createSet(driver, 2, "http://predicate");
+
+        driver.runTest(false);
+    }
+
+    /**
+     * Test basic characteristic set computation
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_generating_reducer_04() throws IOException {
+        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+        T tuple = this.createTuple(1, "http://predicate");
+        driver.addInput(new LongWritable(1), tuple);
+        tuple = this.createTuple(1, "http://other");
+        driver.addInput(new LongWritable(1), tuple);
+
+        // Single entry sets
+        this.createSet(driver, 1, "http://predicate");
+        this.createSet(driver, 1, "http://other");
+        
+        // Two entry sets
+        this.createSet(driver, 1, "http://predicate", "http://other");
+
+        driver.runTest(false);
+    }
+
+    /**
+     * Test basic characteristic set computation
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_generating_reducer_05() throws IOException {
+        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+        T tuple = this.createTuple(1, "http://predicate");
+        driver.addInput(new LongWritable(1), tuple);
+        tuple = this.createTuple(1, "http://other");
+        driver.addInput(new LongWritable(2), tuple);
+        tuple = this.createTuple(1, "http://third");
+        driver.addInput(new LongWritable(3), tuple);
+
+        // Single entry sets
+        this.createSet(driver, 1, "http://predicate");
+        this.createSet(driver, 1, "http://other");
+        this.createSet(driver, 1, "http://third");
+
+        // Two entry sets
+        this.createSet(driver, 1, "http://predicate", "http://other");
+        this.createSet(driver, 1, "http://predicate", "http://third");
+        this.createSet(driver, 1, "http://other", "http://third");
+        
+        // Three entry sets
+        this.createSet(driver, 1, "http://predicate", "http://other", "http://third");
+
+        driver.runTest(false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducerTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducerTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducerTest.java
new file mode 100644
index 0000000..30da730
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducerTest.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.mapreduce.characteristics;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
+import org.apache.hadoop.mrunit.types.Pair;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapReduceTests;
+import org.apache.jena.hadoop.rdf.mapreduce.characteristics.CharacteristicSetReducer;
+import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
+import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * Abstract tests for the {@link CharacteristicSetReducer}
+ * 
+ * 
+ */
+public class CharacteristicSetReducerTest
+        extends
+        AbstractMapReduceTests<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> {
+
+    @Override
+    protected final Mapper<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable> getMapperInstance() {
+        // Identity mapper
+        return new Mapper<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable>();
+    }
+
+    @Override
+    protected final Reducer<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> getReducerInstance() {
+        return new CharacteristicSetReducer();
+    }
+
+    /**
+     * Creates a set consisting of the given predicates
+     * 
+     * @param predicates
+     *            Predicates
+     * @return Set
+     */
+    protected CharacteristicSetWritable createSet(
+            MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver,
+            int inputOccurrences, int outputOccurrences, String... predicates) {
+        CharacteristicSetWritable set = new CharacteristicSetWritable();
+        for (String predicateUri : predicates) {
+            set.add(new CharacteristicWritable(NodeFactory.createURI(predicateUri)));
+        }
+        for (int i = 1; i <= inputOccurrences; i++) {
+            driver.addInput(set, set);
+        }
+        for (int i = 1; i <= outputOccurrences; i++) {
+            driver.addOutput(set, NullWritable.get());
+        }
+        return set;
+    }
+
+    /**
+     * Test characteristic set reduction
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_reducer_01() throws IOException {
+        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+
+        this.createSet(driver, 1, 1, "http://predicate");
+
+        driver.runTest(false);
+    }
+
+    /**
+     * Test characteristic set reduction
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_reducer_02() throws IOException {
+        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+
+        this.createSet(driver, 2, 1, "http://predicate");
+
+        driver.runTest(false);
+
+        List<Pair<CharacteristicSetWritable, NullWritable>> results = driver.run();
+        CharacteristicSetWritable cw = results.get(0).getFirst();
+        Assert.assertEquals(2, cw.getCount().get());
+    }
+
+    /**
+     * Test characteristic set reduction
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_reducer_03() throws IOException {
+        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+
+        this.createSet(driver, 1, 1, "http://predicate");
+        this.createSet(driver, 1, 1, "http://other");
+
+        driver.runTest(false);
+    }
+
+    /**
+     * Test characteristic set reduction
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_reducer_04() throws IOException {
+        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+
+        this.createSet(driver, 2, 1, "http://predicate");
+        this.createSet(driver, 1, 1, "http://other");
+
+        driver.runTest(false);
+
+        List<Pair<CharacteristicSetWritable, NullWritable>> results = driver.run();
+        for (Pair<CharacteristicSetWritable, NullWritable> pair : results) {
+            CharacteristicSetWritable cw = pair.getFirst();
+            boolean expectTwo = cw.getCharacteristics().next().getNode().get().hasURI("http://predicate");
+            Assert.assertEquals(expectTwo ? 2 : 1, cw.getCount().get());
+        }
+    }
+
+    /**
+     * Test characteristic set reduction
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_reducer_05() throws IOException {
+        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+
+        this.createSet(driver, 1, 1, "http://predicate", "http://other");
+        this.createSet(driver, 1, 1, "http://other");
+
+        driver.runTest(false);
+    }
+
+    /**
+     * Test characteristic set reduction
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_reducer_06() throws IOException {
+        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
+                .getMapReduceDriver();
+
+        this.createSet(driver, 2, 1, "http://predicate", "http://other");
+        this.createSet(driver, 1, 1, "http://other");
+
+        driver.runTest(false);
+
+        List<Pair<CharacteristicSetWritable, NullWritable>> results = driver.run();
+        for (Pair<CharacteristicSetWritable, NullWritable> pair : results) {
+            CharacteristicSetWritable cw = pair.getFirst();
+            boolean expectTwo = cw.hasCharacteristic("http://predicate");
+            Assert.assertEquals(expectTwo ? 2 : 1, cw.getCount().get());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducerTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducerTest.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducerTest.java
new file mode 100644
index 0000000..e647b68
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducerTest.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.jena.hadoop.rdf.mapreduce.characteristics.TripleCharacteristicSetGeneratingReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupBySubjectMapper;
+import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Tests for the {@link TripleCharacteristicSetGeneratingReducer}
+ * 
+ * 
+ * 
+ */
+public class TripleCharacteristicSetGeneratingReducerTest extends AbstractCharacteristicSetGeneratingReducerTests<Triple, TripleWritable> {
+
+    @Override
+    protected Mapper<LongWritable, TripleWritable, NodeWritable, TripleWritable> getMapperInstance() {
+        return new TripleGroupBySubjectMapper<LongWritable>();
+    }
+
+    @Override
+    protected Reducer<NodeWritable, TripleWritable, CharacteristicSetWritable, NullWritable> getReducerInstance() {
+        return new TripleCharacteristicSetGeneratingReducer();
+    }
+
+    @Override
+    protected TripleWritable createTuple(int i, String predicateUri) {
+        return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI(predicateUri),
+                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountReducedTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountReducedTests.java b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountReducedTests.java
new file mode 100644
index 0000000..ebdbcde
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountReducedTests.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
+import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapReduceTests;
+import org.apache.jena.hadoop.rdf.mapreduce.count.AbstractNodeTupleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.junit.Test;
+
+
+/**
+ * Abstract tests for mappers derived from
+ * {@link AbstractNodeTupleNodeCountMapper}
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleNodeCountReducedTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        AbstractMapReduceTests<LongWritable, T, NodeWritable, LongWritable, NodeWritable, LongWritable> {
+
+    /**
+     * Generates tuples for the tests
+     * 
+     * @param driver
+     *            Driver
+     * @param num
+     *            Number of tuples to generate
+     */
+    protected void generateData(MapReduceDriver<LongWritable, T, NodeWritable, LongWritable, NodeWritable, LongWritable> driver, int num) {
+        Map<NodeWritable, Long> counts = new HashMap<NodeWritable, Long>();
+        for (int i = 0; i < num; i++) {
+            LongWritable key = new LongWritable(i);
+            T value = this.createValue(i);
+            NodeWritable[] nodes = this.getNodes(value);
+
+            driver.addInput(key, value);
+            for (NodeWritable n : nodes) {
+                if (counts.containsKey(n)) {
+                    counts.put(n, counts.get(n) + 1);
+                } else {
+                    counts.put(n, 1l);
+                }
+            }
+        }
+        
+        for (Entry<NodeWritable, Long> kvp : counts.entrySet()) {
+            driver.addOutput(kvp.getKey(), new LongWritable(kvp.getValue()));
+        }
+    }
+
+    /**
+     * Creates a tuple value
+     * 
+     * @param i
+     *            Index
+     * @return Tuple value
+     */
+    protected abstract T createValue(int i);
+
+    /**
+     * Splits the tuple value into its constituent nodes
+     * 
+     * @param tuple
+     *            Tuple value
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] getNodes(T tuple);
+
+    /**
+     * Runs a node count test
+     * 
+     * @param num
+     *            Number of tuples to generate
+     * @throws IOException
+     */
+    protected void testNodeCount(int num) throws IOException {
+        MapReduceDriver<LongWritable, T, NodeWritable, LongWritable, NodeWritable, LongWritable> driver = this.getMapReduceDriver();
+        this.generateData(driver, num);
+        driver.runTest(false);
+    }
+
+    /**
+     * Tests node counting
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void node_count_01() throws IOException {
+        this.testNodeCount(1);
+    }
+
+    /**
+     * Tests node counting
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void node_count_02() throws IOException {
+        this.testNodeCount(100);
+    }
+
+    /**
+     * Tests node counting
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void node_count_03() throws IOException {
+        this.testNodeCount(1000);
+    }
+
+    /**
+     * Tests node counting
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void node_count_04() throws IOException {
+        this.testNodeCount(2500);
+    }
+}


[05/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
new file mode 100644
index 0000000..4bb0939
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
@@ -0,0 +1,636 @@
+/*
+ * 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.bnodes;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
+import java.nio.file.attribute.FileAttribute;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.log4j.BasicConfigurator;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * Test case that embodies the scenario described in JENA-820
+ */
+@SuppressWarnings("unused")
+public abstract class AbstractBlankNodeTests<T, TValue extends AbstractNodeTupleWritable<T>> {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractBlankNodeTests.class);
+
+    @BeforeClass
+    public static void setup() {
+        // Enable if you need to diagnose test failures
+        // Useful since it includes printing the file names of the temporary
+        // files being used
+        // BasicConfigurator.resetConfiguration();
+        // BasicConfigurator.configure();
+    }
+
+    /**
+     * Gets the extension for the initial input files
+     * 
+     * @return Extension including the {@code .}
+     */
+    protected abstract String getInitialInputExtension();
+
+    /**
+     * Creates a tuple
+     * 
+     * @param s
+     *            Subject
+     * @param p
+     *            Predicate
+     * @param o
+     *            Object
+     * @return Tuple
+     */
+    protected abstract T createTuple(Node s, Node p, Node o);
+
+    /**
+     * Writes out the given tuples to the given file
+     * 
+     * @param f
+     *            File
+     * @param tuples
+     *            Tuples
+     * @throws FileNotFoundException
+     */
+    protected abstract void writeTuples(File f, List<T> tuples) throws FileNotFoundException;
+
+    /**
+     * Creates the input format for reading the initial inputs
+     * 
+     * @return Input format
+     */
+    protected abstract InputFormat<LongWritable, TValue> createInitialInputFormat();
+
+    /**
+     * Creates the output format for writing the intermediate output
+     * 
+     * @return Output format
+     */
+    protected abstract OutputFormat<LongWritable, TValue> createIntermediateOutputFormat();
+
+    /**
+     * Creates the input format for reading the intermediate outputs back in
+     * 
+     * @return Input format
+     */
+    protected abstract InputFormat<LongWritable, TValue> createIntermediateInputFormat();
+
+    /**
+     * Gets the subject of the tuple
+     * 
+     * @param value
+     *            Tuple
+     * @return Subject
+     */
+    protected abstract Node getSubject(T value);
+
+    /**
+     * Gets whether the format being tested respects the RIOT
+     * {@link ParserProfile}
+     * 
+     * @return True if parser profile is respected, false otherwise
+     */
+    protected boolean respectsParserProfile() {
+        return true;
+    }
+
+    /**
+     * Gets whether the format being tested preserves blank node identity
+     * 
+     * @return True if identity is presereved, false otherwise
+     */
+    protected boolean preservesBlankNodeIdentity() {
+        return false;
+    }
+
+    /**
+     * Test that starts with two blank nodes with the same identity in a single
+     * file, splits them over two files and checks that we can workaround
+     * JENA-820 successfully by setting the
+     * {@link RdfIOConstants#GLOBAL_BNODE_IDENTITY} flag for our subsequent job
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public final void blank_node_divergence_01() throws IOException, InterruptedException {
+        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
+        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
+        
+        // Temporary files
+        File a = File.createTempFile("bnode_divergence", getInitialInputExtension());
+        File intermediateOutputDir = Files.createTempDirectory("bnode_divergence", new FileAttribute[0]).toFile();
+
+        try {
+            // Prepare the input data
+            // Two mentions of the same blank node in the same file
+            List<T> tuples = new ArrayList<>();
+            Node bnode = NodeFactory.createAnon();
+            Node pred = NodeFactory.createURI("http://example.org/predicate");
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
+            writeTuples(a, tuples);
+
+            // Set up fake job which will process the file as a single split
+            Configuration config = new Configuration(true);
+            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
+            Job job = Job.getInstance(config);
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()));
+            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            List<InputSplit> splits = inputFormat.getSplits(context);
+            Assert.assertEquals(1, splits.size());
+
+            for (InputSplit split : splits) {
+                // Initialize the input reading
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        createAttemptID(1, 1, 1));
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                // Copy the input to the output - each triple goes to a separate
+                // output file
+                // This is how we force multiple files to be produced
+                int taskID = 1;
+                while (reader.nextKeyValue()) {
+                    // Prepare the output writing
+                    OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
+                    TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                            createAttemptID(1, ++taskID, 1));
+                    RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
+
+                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
+                    writer.close(outputTaskContext);
+                }
+            }
+
+            // Promote outputs from temporary status
+            promoteInputs(intermediateOutputDir);
+
+            // Now we need to create a subsequent job that reads the
+            // intermediate outputs
+            // As described in JENA-820 at this point the blank nodes are
+            // consistent, however when we read them from different files they
+            // by default get treated as different nodes and so the blank nodes
+            // diverge which is incorrect and undesirable behaviour in
+            // multi-stage pipelines
+            System.out.println(intermediateOutputDir.getAbsolutePath());
+            job = Job.getInstance(config);
+            inputFormat = createIntermediateInputFormat();
+            job.setInputFormatClass(inputFormat.getClass());
+            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
+
+            // Enabling this flag works around the JENA-820 issue
+            job.getConfiguration().setBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, true);
+            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            splits = inputFormat.getSplits(context);
+            Assert.assertEquals(2, splits.size());
+
+            // Expect to end up with a single blank node
+            Set<Node> nodes = new HashSet<Node>();
+            for (InputSplit split : splits) {
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        new TaskAttemptID());
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                while (reader.nextKeyValue()) {
+                    nodes.add(getSubject(reader.getCurrentValue().get()));
+                }
+            }
+            // Nodes should not have diverged
+            Assert.assertEquals(1, nodes.size());
+
+        } finally {
+            a.delete();
+            deleteDirectory(intermediateOutputDir);
+        }
+    }
+
+    /**
+     * Test that starts with two blank nodes with the same identity in a single
+     * file, splits them over two files and shows that they diverge in the
+     * subsequent job when the JENA-820 workaround is not enabled
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void blank_node_divergence_02() throws IOException, InterruptedException {
+        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
+        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
+        
+        // Temporary files
+        File a = File.createTempFile("bnode_divergence", getInitialInputExtension());
+        File intermediateOutputDir = Files.createTempDirectory("bnode_divergence", new FileAttribute[0]).toFile();
+
+        try {
+            // Prepare the input data
+            // Two mentions of the same blank node in the same file
+            List<T> tuples = new ArrayList<>();
+            Node bnode = NodeFactory.createAnon();
+            Node pred = NodeFactory.createURI("http://example.org/predicate");
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
+            writeTuples(a, tuples);
+
+            // Set up fake job which will process the file as a single split
+            Configuration config = new Configuration(true);
+            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
+            Job job = Job.getInstance(config);
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()));
+            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            List<InputSplit> splits = inputFormat.getSplits(context);
+            Assert.assertEquals(1, splits.size());
+
+            for (InputSplit split : splits) {
+                // Initialize the input reading
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        createAttemptID(1, 1, 1));
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                // Copy the input to the output - each triple goes to a separate
+                // output file
+                // This is how we force multiple files to be produced
+                int taskID = 1;
+                while (reader.nextKeyValue()) {
+                    // Prepare the output writing
+                    OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
+                    TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                            createAttemptID(1, ++taskID, 1));
+                    RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
+
+                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
+                    writer.close(outputTaskContext);
+                }
+            }
+
+            // Promote outputs from temporary status
+            promoteInputs(intermediateOutputDir);
+
+            // Now we need to create a subsequent job that reads the
+            // intermediate outputs
+            // As described in JENA-820 at this point the blank nodes are
+            // consistent, however when we read them from different files they
+            // by default get treated as different nodes and so the blank nodes
+            // diverge which is incorrect and undesirable behaviour in
+            // multi-stage pipelines. However it is the default behaviour
+            // because when we start from external inputs we want them to be
+            // file scoped.
+            System.out.println(intermediateOutputDir.getAbsolutePath());
+            job = Job.getInstance(config);
+            inputFormat = createIntermediateInputFormat();
+            job.setInputFormatClass(inputFormat.getClass());
+            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
+
+            // Make sure JENA-820 flag is disabled
+            job.getConfiguration().setBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, false);
+            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            splits = inputFormat.getSplits(context);
+            Assert.assertEquals(2, splits.size());
+
+            // Expect to end up with a single blank node
+            Set<Node> nodes = new HashSet<Node>();
+            for (InputSplit split : splits) {
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        new TaskAttemptID());
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                while (reader.nextKeyValue()) {
+                    nodes.add(getSubject(reader.getCurrentValue().get()));
+                }
+            }
+            // Nodes should have diverged
+            Assert.assertEquals(2, nodes.size());
+
+        } finally {
+            a.delete();
+            deleteDirectory(intermediateOutputDir);
+        }
+    }
+
+    /**
+     * Test that starts with two blank nodes in two different files and checks
+     * that writing them to a single file does not conflate them
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void blank_node_identity_01() throws IOException, InterruptedException {
+        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
+        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
+        
+        // Temporary files
+        File a = File.createTempFile("bnode_identity", getInitialInputExtension());
+        File b = File.createTempFile("bnode_identity", getInitialInputExtension());
+        File intermediateOutputDir = Files.createTempDirectory("bnode_identity", new FileAttribute[0]).toFile();
+
+        try {
+            // Prepare the input data
+            // Different blank nodes in different files
+            List<T> tuples = new ArrayList<>();
+            Node bnode1 = NodeFactory.createAnon();
+            Node bnode2 = NodeFactory.createAnon();
+            Node pred = NodeFactory.createURI("http://example.org/predicate");
+
+            tuples.add(createTuple(bnode1, pred, NodeFactory.createLiteral("first")));
+            writeTuples(a, tuples);
+
+            tuples.clear();
+            tuples.add(createTuple(bnode2, pred, NodeFactory.createLiteral("second")));
+            writeTuples(b, tuples);
+
+            // Set up fake job which will process the two files
+            Configuration config = new Configuration(true);
+            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
+            Job job = Job.getInstance(config);
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()), new Path(b.getAbsolutePath()));
+            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            List<InputSplit> splits = inputFormat.getSplits(context);
+            Assert.assertEquals(2, splits.size());
+
+            // Prepare the output writing - putting all output to a single file
+            OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
+            TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(), createAttemptID(
+                    1, 2, 1));
+            RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
+
+            for (InputSplit split : splits) {
+                // Initialize the input reading
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        createAttemptID(1, 1, 1));
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                // Copy the input to the output - all triples go to a single
+                // output
+                while (reader.nextKeyValue()) {
+                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
+                }
+            }
+            writer.close(outputTaskContext);
+
+            // Promote outputs from temporary status
+            promoteInputs(intermediateOutputDir);
+
+            // Now we need to create a subsequent job that reads the
+            // intermediate outputs
+            // The Blank nodes should have been given separate identities so we
+            // should not be conflating them, this is the opposite problem to
+            // that described in JENA-820
+            System.out.println(intermediateOutputDir.getAbsolutePath());
+            job = Job.getInstance(config);
+            inputFormat = createIntermediateInputFormat();
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            splits = inputFormat.getSplits(context);
+            Assert.assertEquals(1, splits.size());
+
+            // Expect to end up with a single blank node
+            Set<Node> nodes = new HashSet<Node>();
+            for (InputSplit split : splits) {
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        new TaskAttemptID());
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                while (reader.nextKeyValue()) {
+                    nodes.add(getSubject(reader.getCurrentValue().get()));
+                }
+            }
+            // Nodes must not have converged
+            Assert.assertEquals(2, nodes.size());
+
+        } finally {
+            a.delete();
+            b.delete();
+            deleteDirectory(intermediateOutputDir);
+        }
+    }
+
+    /**
+     * Test that starts with two blank nodes in two different files and checks
+     * that writing them to a single file does not conflate them
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void blank_node_identity_02() throws IOException, InterruptedException {
+        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
+        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
+        
+        // Temporary files
+        File a = File.createTempFile("bnode_identity", getInitialInputExtension());
+        File b = File.createTempFile("bnode_identity", getInitialInputExtension());
+        File intermediateOutputDir = Files.createTempDirectory("bnode_identity", new FileAttribute[0]).toFile();
+
+        try {
+            // Prepare the input data
+            // Same blank node but in different files so must be treated as
+            // different blank nodes and not converge
+            List<T> tuples = new ArrayList<>();
+            Node bnode = NodeFactory.createAnon();
+            Node pred = NodeFactory.createURI("http://example.org/predicate");
+
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
+            writeTuples(a, tuples);
+
+            tuples.clear();
+            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
+            writeTuples(b, tuples);
+
+            // Set up fake job which will process the two files
+            Configuration config = new Configuration(true);
+            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
+            Job job = Job.getInstance(config);
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()), new Path(b.getAbsolutePath()));
+            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            List<InputSplit> splits = inputFormat.getSplits(context);
+            Assert.assertEquals(2, splits.size());
+
+            // Prepare the output writing - putting all output to a single file
+            OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
+            TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(), createAttemptID(
+                    1, 2, 1));
+            RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
+
+            for (InputSplit split : splits) {
+                // Initialize the input reading
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        createAttemptID(1, 1, 1));
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                // Copy the input to the output - all triples go to a single
+                // output
+                while (reader.nextKeyValue()) {
+                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
+                }
+            }
+            writer.close(outputTaskContext);
+
+            // Promote outputs from temporary status
+            promoteInputs(intermediateOutputDir);
+
+            // Now we need to create a subsequent job that reads the
+            // intermediate outputs
+            // The Blank nodes should have been given separate identities so we
+            // should not be conflating them, this is the opposite problem to
+            // that described in JENA-820
+            System.out.println(intermediateOutputDir.getAbsolutePath());
+            job = Job.getInstance(config);
+            inputFormat = createIntermediateInputFormat();
+            job.setInputFormatClass(inputFormat.getClass());
+            NLineInputFormat.setNumLinesPerSplit(job, 100);
+            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
+            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+
+            // Get the splits
+            splits = inputFormat.getSplits(context);
+            Assert.assertEquals(1, splits.size());
+
+            // Expect to end up with a single blank node
+            Set<Node> nodes = new HashSet<Node>();
+            for (InputSplit split : splits) {
+                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
+                        new TaskAttemptID());
+                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
+                reader.initialize(split, inputTaskContext);
+
+                while (reader.nextKeyValue()) {
+                    nodes.add(getSubject(reader.getCurrentValue().get()));
+                }
+            }
+            // Nodes must not diverge
+            Assert.assertEquals(2, nodes.size());
+
+        } finally {
+            a.delete();
+            b.delete();
+            deleteDirectory(intermediateOutputDir);
+        }
+    }
+
+    private TaskAttemptID createAttemptID(int jobID, int taskID, int id) {
+        return new TaskAttemptID("outputTest", jobID, TaskType.MAP, taskID, 1);
+    }
+
+    private void promoteInputs(File baseDir) throws IOException {
+        for (File f : baseDir.listFiles()) {
+            if (f.isDirectory()) {
+                promoteInputs(baseDir, f);
+            }
+        }
+    }
+
+    private void promoteInputs(File targetDir, File dir) throws IOException {
+        java.nio.file.Path target = Paths.get(targetDir.toURI());
+        for (File f : dir.listFiles()) {
+            if (f.isDirectory()) {
+                promoteInputs(targetDir, f);
+            } else {
+                LOGGER.debug("Moving {} to {}", f.getAbsolutePath(), target.resolve(f.getName()));
+                Files.move(Paths.get(f.toURI()), target.resolve(f.getName()), StandardCopyOption.REPLACE_EXISTING);
+            }
+        }
+
+        // Remove defunct sub-directory
+        dir.delete();
+    }
+
+    private void deleteDirectory(File dir) throws IOException {
+        for (File f : dir.listFiles()) {
+            if (f.isFile())
+                f.delete();
+            if (f.isDirectory())
+                deleteDirectory(f);
+        }
+        dir.delete();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
new file mode 100644
index 0000000..bbd6742
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
@@ -0,0 +1,65 @@
+/*
+ * 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.bnodes;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.util.List;
+
+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.Node;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.graph.GraphFactory;
+
+/**
+ *
+ */
+public abstract class AbstractTripleBlankNodeTests extends AbstractBlankNodeTests<Triple, TripleWritable> {
+    
+    /**
+     * Gets the language to use
+     * 
+     * @return Language
+     */
+    protected abstract Lang getLanguage();
+
+    @Override
+    protected Triple createTuple(Node s, Node p, Node o) {
+        return new Triple(s, p, o);
+    }
+
+    @Override
+    protected void writeTuples(File f, List<Triple> tuples) throws FileNotFoundException {
+        Graph g = GraphFactory.createGraphMem();
+        for (Triple t : tuples) {
+            g.add(t);
+        }
+        RDFDataMgr.write(new FileOutputStream(f), g, getLanguage());
+    }
+
+    @Override
+    protected Node getSubject(Triple value) {
+        return value.getSubject();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
new file mode 100644
index 0000000..f234127
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDTripleInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.jsonld.JsonLDTripleOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link RdfJsonInputFormat}
+ */
+public class JsonLdTripleBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.JSONLD;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".jsonld";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new JsonLDTripleInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new JsonLDTripleOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new JsonLDTripleInputFormat();
+    }
+
+    @Override
+    protected boolean respectsParserProfile() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
new file mode 100644
index 0000000..4c350c7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.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.input.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link NTriplesInputFormat}
+ */
+public class NTriplesBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".nt";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new NTriplesInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new NTriplesOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new NTriplesInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
new file mode 100644
index 0000000..2be1e0e
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.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.input.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.rdfjson.RdfJsonOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link RdfJsonInputFormat}
+ */
+public class RdfJsonBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.RDFJSON;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".rj";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new RdfJsonInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new RdfJsonOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new RdfJsonInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
new file mode 100644
index 0000000..d6f32a2
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.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.input.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftTripleInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.thrift.ThriftTripleOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+/**
+ * Tests blank node divergence when using the {@link RdfThriftInputFormat}
+ */
+public class RdfThriftBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".trdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new ThriftTripleInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new ThriftTripleOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new ThriftTripleInputFormat();
+    }
+
+    @Override
+    protected boolean respectsParserProfile() {
+        return false;
+    }
+    
+    @Override
+    protected boolean preservesBlankNodeIdentity() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
new file mode 100644
index 0000000..da70007
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.rdfxml.RdfXmlInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.rdfxml.RdfXmlOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link RdfXmlInputFormat}
+ */
+public class RdfXmlBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.RDFXML;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".rdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new RdfXmlInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new RdfXmlOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new RdfXmlInputFormat();
+    }
+
+    @Override
+    protected boolean respectsParserProfile() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
new file mode 100644
index 0000000..146c836
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.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.input.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.turtle.TurtleInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.turtle.TurtleOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link TurtleInputFormat}
+ */
+public class TurtleBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.TURTLE;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".ttl";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new TurtleInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new TurtleOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new TurtleInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
new file mode 100644
index 0000000..1f18a95
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.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;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+
+
+/**
+ * 
+ * 
+ * @param <TValue>
+ * @param <T>
+ */
+public abstract class AbstractCompressedNodeTupleInputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        AbstractNodeTupleInputFormatTests<TValue, T> {
+    
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
+        return config;
+    }
+
+    @Override
+    protected OutputStream getOutputStream(File f) throws IOException {
+        CompressionCodec codec = this.getCompressionCodec();
+        if (codec instanceof Configurable) {
+            ((Configurable) codec).setConf(this.prepareConfiguration());
+        }
+        FileOutputStream fileOutput = new FileOutputStream(f, false);
+        return codec.createOutputStream(fileOutput);
+    }
+
+    /**
+     * Gets the compression codec to use
+     * 
+     * @return Compression codec
+     */
+    protected abstract CompressionCodec getCompressionCodec();
+
+    /**
+     * Indicates whether inputs can be split, defaults to false for compressed
+     * input tests
+     */
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
new file mode 100644
index 0000000..312aae7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.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.input.compressed;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for Quad input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedQuadsInputFormatTests extends
+        AbstractCompressedNodeTupleInputFormatTests<Quad, QuadWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected void generateTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateBadTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write("<http://broken\n".getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
+        boolean bad = false;
+        for (int i = 0; i < num; i++, bad = !bad) {
+            if (bad) {
+                output.write("<http://broken\n".getBytes(utf8));
+            } else {
+                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
+            }
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
new file mode 100644
index 0000000..f0f0caf
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.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.input.compressed;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for Triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedTriplesInputFormatTests extends
+        AbstractCompressedNodeTupleInputFormatTests<Triple, TripleWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected void generateTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateBadTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write("<http://broken\n".getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
+        boolean bad = false;
+        for (int i = 0; i < num; i++, bad = !bad) {
+            if (bad) {
+                output.write("<http://broken\n".getBytes(utf8));
+            } else {
+                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
+            }
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
new file mode 100644
index 0000000..be2b1d7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
@@ -0,0 +1,150 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
+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.query.Dataset;
+import com.hp.hpl.jena.query.DatasetFactory;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Property;
+import com.hp.hpl.jena.rdf.model.Resource;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for compressed whole file quad formats
+ * 
+ * 
+ */
+public abstract class AbstractCompressedWholeFileQuadInputFormatTests extends
+        AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
+        return config;
+    }
+
+    @Override
+    protected OutputStream getOutputStream(File f) throws IOException {
+        CompressionCodec codec = this.getCompressionCodec();
+        if (codec instanceof Configurable) {
+            ((Configurable) codec).setConf(this.prepareConfiguration());
+        }
+        FileOutputStream fileOutput = new FileOutputStream(f, false);
+        return codec.createOutputStream(fileOutput);
+    }
+
+    /**
+     * Gets the compression codec to use
+     * 
+     * @return Compression codec
+     */
+    protected abstract CompressionCodec getCompressionCodec();
+
+    /**
+     * Indicates whether inputs can be split, defaults to false for compressed
+     * input tests
+     */
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+
+    private void writeTuples(Dataset ds, OutputStream output) {
+        RDFDataMgr.write(output, ds, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
+    }
+
+    /**
+     * Gets the RDF language to write out generated tuples in
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+    private void writeGoodTuples(OutputStream output, int num) throws IOException {
+        Dataset ds = DatasetFactory.createMem();
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num; i++) {
+            if (i % 100 == 0) {
+                ds.addNamedModel("http://example.org/graphs/" + (i / 100), m);
+                m = ModelFactory.createDefaultModel();
+            }
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        if (!m.isEmpty()) {
+            ds.addNamedModel("http://example.org/graphs/extra", m);
+        }
+        this.writeTuples(ds, output);
+    }
+
+    @Override
+    protected final void generateTuples(OutputStream output, int num) throws IOException {
+        this.writeGoodTuples(output, num);
+        output.close();
+    }
+
+    @Override
+    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
+        // Write good data
+        this.writeGoodTuples(output, num / 2);
+
+        // Write junk data
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num / 2; i++) {
+            output.write(junk);
+        }
+
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
new file mode 100644
index 0000000..56dd8ca
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
@@ -0,0 +1,144 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
+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.Triple;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Property;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * Abstract tests for compressed whole file triple formats
+ * 
+ * 
+ */
+public abstract class AbstractCompressedWholeFileTripleInputFormatTests extends
+        AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
+
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
+        return config;
+    }
+
+    @Override
+    protected OutputStream getOutputStream(File f) throws IOException {
+        CompressionCodec codec = this.getCompressionCodec();
+        if (codec instanceof Configurable) {
+            ((Configurable) codec).setConf(this.prepareConfiguration());
+        }
+        FileOutputStream fileOutput = new FileOutputStream(f, false);
+        return codec.createOutputStream(fileOutput);
+    }
+
+    /**
+     * Gets the compression codec to use
+     * 
+     * @return Compression codec
+     */
+    protected abstract CompressionCodec getCompressionCodec();
+
+    /**
+     * Indicates whether inputs can be split, defaults to false for compressed
+     * input tests
+     */
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+
+    private void writeTuples(Model m, OutputStream output) {
+        RDFDataMgr.write(output, m, this.getRdfLanguage());
+    }
+
+    /**
+     * Gets the RDF language to write out generated tuples in
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+    @Override
+    protected final void generateTuples(OutputStream output, int num) throws IOException {
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num; i++) {
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        this.writeTuples(m, output);
+        output.close();
+    }
+
+    @Override
+    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
+        // Write good data
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num / 2; i++) {
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        this.writeTuples(m, output);
+
+        // Write junk data
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num / 2; i++) {
+            output.write(junk);
+        }
+
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
new file mode 100644
index 0000000..d118f29
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDQuadInputFormat;
+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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
new file mode 100644
index 0000000..acb9e08
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDTripleInputFormat;
+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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-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/jena-elephas-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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-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/jena-elephas-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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-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/jena-elephas-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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-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/jena-elephas-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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-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/jena-elephas-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());
+    }
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapperTest.java
deleted file mode 100644
index 3b71f40..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapperTest.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.split.TripleSplitToNodesMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.split.TripleSplitWithNodesMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Tests for the {@link TripleSplitToNodesMapper}
- * 
- * 
- * 
- */
-public class TripleSplitWithNodesMapperTest extends AbstractTripleSplitWithNodesTests {
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, TripleWritable, NodeWritable> getInstance() {
-        return new TripleSplitWithNodesMapper<LongWritable>();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapperTest.java
deleted file mode 100644
index 51b29cb..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapperTest.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.transform;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
-import org.apache.jena.hadoop.rdf.mapreduce.transform.QuadsToTriplesMapper;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.junit.Test;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Tests for the {@link QuadsToTriplesMapper}
- * 
- * 
- * 
- */
-public class QuadsToTriplesMapperTest extends AbstractMapperTests<LongWritable, QuadWritable, LongWritable, TripleWritable> {
-
-    @Override
-    protected Mapper<LongWritable, QuadWritable, LongWritable, TripleWritable> getInstance() {
-        return new QuadsToTriplesMapper<LongWritable>();
-    }
-
-    protected void generateData(MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver, int num) {
-        for (int i = 0; i < num; i++) {
-            Triple t = new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
-            Quad q = new Quad(Quad.defaultGraphNodeGenerated, t);
-            driver.addInput(new LongWritable(i), new QuadWritable(q));
-            driver.addOutput(new LongWritable(i), new TripleWritable(t));
-        }
-    }
-
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void quads_to_triples_mapper_01() throws IOException {
-        MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver = this.getMapDriver();
-
-        Triple t = new Triple(NodeFactory.createURI("http://s"), NodeFactory.createURI("http://p"),
-                NodeFactory.createLiteral("test"));
-        Quad q = new Quad(Quad.defaultGraphNodeGenerated, t);
-        driver.withInput(new Pair<LongWritable, QuadWritable>(new LongWritable(1), new QuadWritable(q))).withOutput(
-                new Pair<LongWritable, TripleWritable>(new LongWritable(1), new TripleWritable(t)));
-        driver.runTest();
-    }
-    
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void quads_to_triples_mapper_02() throws IOException {
-        MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver = this.getMapDriver();
-        this.generateData(driver, 100);
-        driver.runTest();
-    }
-    
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void quads_to_triples_mapper_03() throws IOException {
-        MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver = this.getMapDriver();
-        this.generateData(driver, 1000);
-        driver.runTest();
-    }
-    
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void quads_to_triples_mapper_04() throws IOException {
-        MapDriver<LongWritable, QuadWritable, LongWritable, TripleWritable> driver = this.getMapDriver();
-        this.generateData(driver, 10000);
-        driver.runTest();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapperTest.java
deleted file mode 100644
index bdf39f5..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapperTest.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.transform;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
-import org.apache.jena.hadoop.rdf.mapreduce.transform.TriplesToQuadsBySubjectMapper;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.junit.Test;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Tests for the {@link TriplesToQuadsBySubjectMapper}
- * 
- * 
- * 
- */
-public class TriplesToQuadsBySubjectMapperTest extends AbstractMapperTests<LongWritable, TripleWritable, LongWritable, QuadWritable> {
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, LongWritable, QuadWritable> getInstance() {
-        return new TriplesToQuadsBySubjectMapper<LongWritable>();
-    }
-
-    protected void generateData(MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver, int num) {
-        for (int i = 0; i < num; i++) {
-            Triple t = new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
-            Quad q = new Quad(t.getSubject(), t);
-            driver.addInput(new LongWritable(i), new TripleWritable(t));
-            driver.addOutput(new LongWritable(i), new QuadWritable(q));
-        }
-    }
-
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void triples_to_quads_mapper_01() throws IOException {
-        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
-
-        Triple t = new Triple(NodeFactory.createURI("http://s"), NodeFactory.createURI("http://p"),
-                NodeFactory.createLiteral("test"));
-        Quad q = new Quad(t.getSubject(), t);
-        driver.withInput(new Pair<LongWritable, TripleWritable>(new LongWritable(1), new TripleWritable(t))).withOutput(
-                new Pair<LongWritable, QuadWritable>(new LongWritable(1), new QuadWritable(q)));
-        driver.runTest();
-    }
-    
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void triples_to_quads_mapper_02() throws IOException {
-        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
-        this.generateData(driver, 100);
-        driver.runTest();
-    }
-    
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void triples_to_quads_mapper_03() throws IOException {
-        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
-        this.generateData(driver, 1000);
-        driver.runTest();
-    }
-    
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void triples_to_quads_mapper_04() throws IOException {
-        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
-        this.generateData(driver, 10000);
-        driver.runTest();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapperTest.java
deleted file mode 100644
index b82f74b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapperTest.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.transform;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
-import org.apache.jena.hadoop.rdf.mapreduce.transform.TriplesToQuadsConstantGraphMapper;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.junit.Test;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Tests for the {@link TriplesToQuadsConstantGraphMapper}
- * 
- * 
- * 
- */
-public class TriplesToQuadsConstantGraphMapperTest extends AbstractMapperTests<LongWritable, TripleWritable, LongWritable, QuadWritable> {
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, LongWritable, QuadWritable> getInstance() {
-        return new TriplesToQuadsConstantGraphMapper<LongWritable>();
-    }
-
-    protected void generateData(MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver, int num) {
-        for (int i = 0; i < num; i++) {
-            Triple t = new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                    NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
-            Quad q = new Quad(Quad.defaultGraphNodeGenerated, t);
-            driver.addInput(new LongWritable(i), new TripleWritable(t));
-            driver.addOutput(new LongWritable(i), new QuadWritable(q));
-        }
-    }
-
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void triples_to_quads_mapper_01() throws IOException {
-        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
-
-        Triple t = new Triple(NodeFactory.createURI("http://s"), NodeFactory.createURI("http://p"),
-                NodeFactory.createLiteral("test"));
-        Quad q = new Quad(Quad.defaultGraphNodeGenerated, t);
-        driver.withInput(new Pair<LongWritable, TripleWritable>(new LongWritable(1), new TripleWritable(t))).withOutput(
-                new Pair<LongWritable, QuadWritable>(new LongWritable(1), new QuadWritable(q)));
-        driver.runTest();
-    }
-    
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void triples_to_quads_mapper_02() throws IOException {
-        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
-        this.generateData(driver, 100);
-        driver.runTest();
-    }
-    
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void triples_to_quads_mapper_03() throws IOException {
-        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
-        this.generateData(driver, 1000);
-        driver.runTest();
-    }
-    
-    /**
-     * Tests quads to triples conversion
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void triples_to_quads_mapper_04() throws IOException {
-        MapDriver<LongWritable, TripleWritable, LongWritable, QuadWritable> driver = this.getMapDriver();
-        this.generateData(driver, 10000);
-        driver.runTest();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/pom.xml b/jena-hadoop-rdf/jena-elephas-common/pom.xml
deleted file mode 100644
index 7dd68a0..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/pom.xml
+++ /dev/null
@@ -1,54 +0,0 @@
-<!--
-   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.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<groupId>org.apache.jena</groupId>
-		<artifactId>jena-elephas</artifactId>
-		<version>0.9.0-SNAPSHOT</version>
-	</parent>
-	<artifactId>jena-elephas-common</artifactId>
-	<name>Apache Jena - Elephas - Common API</name>
-	<description>Common code for RDF on Hadoop such as writable types for RDF primitives</description>
-
-	<!-- Note that versions are managed by parent POMs -->
-	<dependencies>
-		<!-- Hadoop Dependencies -->
-		<!-- Note these will be provided on the Hadoop cluster hence the provided 
-			scope -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Jena dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-arq</artifactId>
-		</dependency>
-
-		<!-- Test Dependencies -->
-		<dependency>
-			<groupId>junit</groupId>
-			<artifactId>junit</artifactId>
-			<scope>test</scope>
-		</dependency>
-	</dependencies>
-</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
deleted file mode 100644
index f0acc09..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableUtils;
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.sparql.util.NodeUtils;
-
-/**
- * A abstract general purpose writable where the actual class represented is
- * composed of a number of {@link Node} instances
- * <p>
- * The binary encoding of this base implementation is just a variable integer
- * indicating the number of nodes present followed by the binary encodings of
- * the {@link NodeWritable} instances. Derived implementations may wish to
- * override the {@link #readFields(DataInput)} and {@link #write(DataOutput)}
- * methods in order to use more specialised encodings.
- * </p>
- * 
- * @param <T>
- *            Tuple type
- */
-public abstract class AbstractNodeTupleWritable<T> implements WritableComparable<AbstractNodeTupleWritable<T>> {
-
-    private T tuple;
-
-    /**
-     * Creates a new empty instance
-     */
-    protected AbstractNodeTupleWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance with the given value
-     * 
-     * @param tuple
-     *            Tuple value
-     */
-    protected AbstractNodeTupleWritable(T tuple) {
-        this.tuple = tuple;
-    }
-
-    /**
-     * Gets the tuple
-     * 
-     * @return Tuple
-     */
-    public T get() {
-        return this.tuple;
-    }
-
-    /**
-     * Sets the tuple
-     * 
-     * @param tuple
-     *            Tuple
-     */
-    public void set(T tuple) {
-        this.tuple = tuple;
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        // Determine how many nodes
-        int size = WritableUtils.readVInt(input);
-        Node[] ns = new Node[size];
-
-        NodeWritable nw = new NodeWritable();
-        for (int i = 0; i < ns.length; i++) {
-            nw.readFields(input);
-            ns[i] = nw.get();
-        }
-
-        // Load the tuple
-        this.tuple = this.createTuple(ns);
-    }
-
-    /**
-     * Creates the actual tuple type from an array of nodes
-     * 
-     * @param ns
-     *            Nodes
-     * @return Tuple
-     */
-    protected abstract T createTuple(Node[] ns);
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        // Determine how many nodes
-        Node[] ns = this.createNodes(this.tuple);
-        WritableUtils.writeVInt(output, ns.length);
-
-        // Write out nodes
-        NodeWritable nw = new NodeWritable();
-        for (int i = 0; i < ns.length; i++) {
-            nw.set(ns[i]);
-            nw.write(output);
-        }
-    }
-
-    /**
-     * Sets the tuple value
-     * <p>
-     * Intended only for internal use i.e. when a derived implementation
-     * overrides {@link #readFields(DataInput)} and needs to set the tuple value
-     * directly i.e. when a derived implementation is using a custom encoding
-     * scheme
-     * </p>
-     * 
-     * @param tuple
-     *            Tuple
-     */
-    protected final void setInternal(T tuple) {
-        this.tuple = tuple;
-    }
-
-    /**
-     * Converts the actual tuple type into an array of nodes
-     * 
-     * @param tuple
-     *            Tuples
-     * @return Nodes
-     */
-    protected abstract Node[] createNodes(T tuple);
-
-    /**
-     * Compares instances node by node
-     * <p>
-     * Derived implementations may wish to override this and substitute native
-     * tuple based comparisons
-     * </p>
-     * 
-     * @param other
-     *            Instance to compare with
-     */
-    @Override
-    public int compareTo(AbstractNodeTupleWritable<T> other) {
-        Node[] ns = this.createNodes(this.tuple);
-        Node[] otherNs = this.createNodes(other.tuple);
-
-        if (ns.length < otherNs.length) {
-            return -1;
-        } else if (ns.length > otherNs.length) {
-            return 1;
-        }
-        // Compare node by node
-        for (int i = 0; i < ns.length; i++) {
-            int c = NodeUtils.compareRDFTerms(ns[i], otherNs[i]);
-            if (c != 0)
-                return c;
-        }
-        return 0;
-    }
-
-    @Override
-    public String toString() {
-        return this.get().toString();
-    }
-
-    @Override
-    public int hashCode() {
-        return this.get().hashCode();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public boolean equals(Object other) {
-        if (!(other instanceof AbstractNodeTupleWritable))
-            return false;
-        return this.compareTo((AbstractNodeTupleWritable<T>) other) == 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
deleted file mode 100644
index f29b156..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
+++ /dev/null
@@ -1,298 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableUtils;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * Represents a characteristic set which is comprised of a count of nodes for
- * which the characteristic is applicable and a set of characteristics which
- * represents the number of usages of predicates with those nodes
- * 
- * 
- * 
- */
-public class CharacteristicSetWritable implements WritableComparable<CharacteristicSetWritable> {
-
-    private Map<NodeWritable, CharacteristicWritable> characteristics = new TreeMap<NodeWritable, CharacteristicWritable>();
-    private LongWritable count = new LongWritable();
-
-    /**
-     * Creates a new empty characteristic set with the default count of 1
-     */
-    public CharacteristicSetWritable() {
-        this(1);
-    }
-
-    /**
-     * Creates a new characteristic set with the default count of 1 and the
-     * given characteristics
-     * 
-     * @param characteristics
-     *            Characteristics
-     */
-    public CharacteristicSetWritable(CharacteristicWritable... characteristics) {
-        this(1, characteristics);
-    }
-
-    /**
-     * Creates an empty characteristic set with the given count
-     * 
-     * @param count
-     *            Count
-     */
-    public CharacteristicSetWritable(long count) {
-        this(count, new CharacteristicWritable[0]);
-    }
-
-    /**
-     * Creates a new characteristic set
-     * 
-     * @param count
-     *            Count
-     * @param characteristics
-     *            Characteristics
-     */
-    public CharacteristicSetWritable(long count, CharacteristicWritable... characteristics) {
-        this.count.set(count);
-        for (CharacteristicWritable characteristic : characteristics) {
-            this.characteristics.put(characteristic.getNode(), characteristic);
-        }
-    }
-
-    /**
-     * Creates a new instance and reads its data from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static CharacteristicSetWritable read(DataInput input) throws IOException {
-        CharacteristicSetWritable set = new CharacteristicSetWritable();
-        set.readFields(input);
-        return set;
-    }
-
-    /**
-     * Gets the count
-     * 
-     * @return Count
-     */
-    public LongWritable getCount() {
-        return this.count;
-    }
-
-    /**
-     * Gets the characteristics
-     * 
-     * @return Characteristics
-     */
-    public Iterator<CharacteristicWritable> getCharacteristics() {
-        return this.characteristics.values().iterator();
-    }
-
-    /**
-     * Gets the size of the characteristic set
-     * 
-     * @return Size
-     */
-    public int size() {
-        return this.characteristics.size();
-    }
-
-    /**
-     * Adds a characteristic to the set merging it into the appropriate existing
-     * characteristic if applicable
-     * 
-     * @param characteristic
-     *            Characteristics
-     */
-    public void add(CharacteristicWritable characteristic) {
-        if (this.characteristics.containsKey(characteristic.getNode())) {
-            this.characteristics.get(characteristic.getNode()).increment(characteristic.getCount().get());
-        } else {
-            this.characteristics.put(characteristic.getNode(), characteristic);
-        }
-    }
-
-    /**
-     * Adds some characteristics to the set merging them with the appropriate
-     * existing characteristics if applicable
-     * 
-     * @param characteristics
-     */
-    public void add(CharacteristicWritable... characteristics) {
-        for (CharacteristicWritable characteristic : characteristics) {
-            this.add(characteristic);
-        }
-    }
-
-    /**
-     * Adds the contents of the other characteristic set to this characteristic
-     * set
-     * 
-     * @param set
-     *            Characteristic set
-     */
-    public void add(CharacteristicSetWritable set) {
-        this.increment(set.getCount().get());
-        Iterator<CharacteristicWritable> iter = set.getCharacteristics();
-        while (iter.hasNext()) {
-            this.add(iter.next());
-        }
-    }
-
-    /**
-     * Gets whether the set contains a characteristic for the given predicate
-     * 
-     * @param uri
-     *            Predicate URI
-     * @return True if contained in the set, false otherwise
-     */
-    public boolean hasCharacteristic(String uri) {
-        return this.hasCharacteristic(NodeFactory.createURI(uri));
-    }
-
-    /**
-     * Gets whether the set contains a characteristic for the given predicate
-     * 
-     * @param n
-     *            Predicate
-     * @return True if contained in the set, false otherwise
-     */
-    public boolean hasCharacteristic(Node n) {
-        return this.hasCharacteristic(new NodeWritable(n));
-    }
-
-    /**
-     * Gets whether the set contains a characteristic for the given predicate
-     * 
-     * @param n
-     *            Predicate
-     * @return True if contained in the set, false otherwise
-     */
-    public boolean hasCharacteristic(NodeWritable n) {
-        return this.characteristics.containsKey(n);
-    }
-
-    /**
-     * Increments the count by the given increment
-     * 
-     * @param l
-     *            Increment
-     */
-    public void increment(long l) {
-        this.count.set(this.count.get() + l);
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        // Write size, then count, then characteristics
-        WritableUtils.writeVInt(output, this.characteristics.size());
-        this.count.write(output);
-        for (CharacteristicWritable characteristic : this.characteristics.values()) {
-            characteristic.write(output);
-        }
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        // Read size, then count, then characteristics
-        int size = WritableUtils.readVInt(input);
-        this.count.readFields(input);
-        this.characteristics.clear();
-        for (int i = 0; i < size; i++) {
-            CharacteristicWritable cw = CharacteristicWritable.read(input);
-            this.characteristics.put(cw.getNode(), cw);
-        }
-    }
-
-    @Override
-    public int compareTo(CharacteristicSetWritable cs) {
-        int size = this.characteristics.size();
-        int otherSize = cs.characteristics.size();
-        if (size < otherSize) {
-            return -1;
-        } else if (size > otherSize) {
-            return 1;
-        } else {
-            // Compare characteristics in turn
-            Iterator<CharacteristicWritable> iter = this.getCharacteristics();
-            Iterator<CharacteristicWritable> otherIter = cs.getCharacteristics();
-
-            int compare = 0;
-            while (iter.hasNext()) {
-                CharacteristicWritable c = iter.next();
-                CharacteristicWritable otherC = otherIter.next();
-                compare = c.compareTo(otherC);
-                if (compare != 0)
-                    return compare;
-            }
-            return compare;
-        }
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        if (!(other instanceof CharacteristicSetWritable))
-            return false;
-        return this.compareTo((CharacteristicSetWritable) other) == 0;
-    }
-
-    @Override
-    public int hashCode() {
-        // Build a hash code from characteristics
-        if (this.characteristics.size() == 0)
-            return 0;
-        Iterator<CharacteristicWritable> iter = this.getCharacteristics();
-        int hash = 17;
-        while (iter.hasNext()) {
-            hash = hash * 31 + iter.next().hashCode();
-        }
-        return hash;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder builder = new StringBuilder();
-        builder.append("{ ");
-        builder.append(this.count.get());
-        Iterator<CharacteristicWritable> iter = this.getCharacteristics();
-        while (iter.hasNext()) {
-            builder.append(" , ");
-            builder.append(iter.next().toString());
-        }
-        builder.append(" }");
-        return builder.toString();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
deleted file mode 100644
index 90fc7db..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.WritableComparable;
-
-import com.hp.hpl.jena.graph.Node;
-
-/**
- * Represents a characteristic for a single node and contains the node and a
- * count associated with that node
- * <p>
- * Note that characteristics are compared based upon only the nodes and not
- * their counts
- * </p>
- * 
- * 
- * 
- */
-public class CharacteristicWritable implements WritableComparable<CharacteristicWritable> {
-
-    private NodeWritable node = new NodeWritable();
-    private LongWritable count = new LongWritable();
-
-    /**
-     * Creates an empty characteristic writable
-     */
-    public CharacteristicWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a characteristic writable with the given node and the default
-     * count of 1
-     * 
-     * @param n
-     *            Node
-     */
-    public CharacteristicWritable(Node n) {
-        this(n, 1);
-    }
-
-    /**
-     * Creates a characteristic writable with the given node and count
-     * 
-     * @param n
-     *            Node
-     * @param count
-     *            Count
-     */
-    public CharacteristicWritable(Node n, long count) {
-        this.node.set(n);
-        this.count.set(count);
-    }
-
-    /**
-     * Creates a new instance and reads in its data from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static CharacteristicWritable read(DataInput input) throws IOException {
-        CharacteristicWritable cw = new CharacteristicWritable();
-        cw.readFields(input);
-        return cw;
-    }
-
-    /**
-     * Gets the node
-     * 
-     * @return Node
-     */
-    public NodeWritable getNode() {
-        return this.node;
-    }
-
-    /**
-     * Gets the count
-     * 
-     * @return Count
-     */
-    public LongWritable getCount() {
-        return this.count;
-    }
-
-    /**
-     * Increments the count by 1
-     */
-    public void increment() {
-        this.increment(1);
-    }
-
-    /**
-     * Increments the count by the given value
-     * 
-     * @param l
-     *            Value to increment by
-     */
-    public void increment(long l) {
-        this.count.set(this.count.get() + l);
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        this.node.write(output);
-        this.count.write(output);
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        this.node.readFields(input);
-        this.count.readFields(input);
-    }
-
-    @Override
-    public int compareTo(CharacteristicWritable o) {
-        return this.node.compareTo(o.node);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        if (!(other instanceof CharacteristicWritable))
-            return false;
-        return this.compareTo((CharacteristicWritable) other) == 0;
-    }
-
-    @Override
-    public int hashCode() {
-        return this.node.hashCode();
-    }
-
-    @Override
-    public String toString() {
-        return "(" + this.node.toString() + ", " + this.count.toString() + ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
deleted file mode 100644
index e06aac4..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.IOException;
-
-import org.apache.jena.atlas.lib.Tuple;
-import com.hp.hpl.jena.graph.Node;
-
-/**
- * A writable RDF tuple
- * <p>
- * Unlike the more specific {@link TripleWritable} and {@link QuadWritable} this
- * class allows for arbitrary length tuples and does not restrict tuples to
- * being of uniform size.
- * </p>
- * 
- * 
- * 
- */
-public class NodeTupleWritable extends AbstractNodeTupleWritable<Tuple<Node>> {
-
-    /**
-     * Creates a new empty instance
-     */
-    public NodeTupleWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance with the given value
-     * 
-     * @param tuple
-     *            Tuple
-     */
-    public NodeTupleWritable(Tuple<Node> tuple) {
-        super(tuple);
-    }
-
-    /**
-     * Creates a new instance from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static NodeTupleWritable read(DataInput input) throws IOException {
-        NodeTupleWritable t = new NodeTupleWritable();
-        t.readFields(input);
-        return t;
-    }
-
-    @Override
-    protected Tuple<Node> createTuple(Node[] ns) {
-        return Tuple.create(ns);
-    }
-
-    @Override
-    protected Node[] createNodes(Tuple<Node> tuple) {
-        return tuple.tuple();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
deleted file mode 100644
index cf00f8d..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
-import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
-import org.apache.jena.riot.thrift.TRDF;
-import org.apache.jena.riot.thrift.ThriftConvert;
-import org.apache.jena.riot.thrift.wire.RDF_Term;
-import org.apache.thrift.TException;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.sparql.util.NodeUtils;
-
-/**
- * A writable for {@link Node} instances
- * <p>
- * This uses <a
- * href="http://afs.github.io/rdf-thrift/rdf-binary-thrift.html">RDF Thrift</a>
- * for the binary encoding of terms. The in-memory storage for this type is both
- * a {@link Node} and a {@link RDF_Term} with lazy conversion between the two
- * forms as necessary.
- * </p>
- */
-public class NodeWritable implements WritableComparable<NodeWritable> {
-
-    static {
-        WritableComparator.define(NodeWritable.class, new SimpleBinaryComparator());
-    }
-
-    private Node node;
-    private RDF_Term term = new RDF_Term();
-
-    /**
-     * Creates an empty writable
-     */
-    public NodeWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static NodeWritable read(DataInput input) throws IOException {
-        NodeWritable nw = new NodeWritable();
-        nw.readFields(input);
-        return nw;
-    }
-
-    /**
-     * Creates a new writable with the given value
-     * 
-     * @param n
-     *            Node
-     */
-    public NodeWritable(Node n) {
-        this.set(n);
-    }
-
-    /**
-     * Gets the node
-     * 
-     * @return Node
-     */
-    public Node get() {
-        // We may not have yet loaded the node
-        if (this.node == null) {
-            // If term is set to undefined then node is supposed to be null
-            if (this.term.isSet() && !this.term.isSetUndefined()) {
-                this.node = ThriftConvert.convert(this.term);
-            }
-        }
-        return this.node;
-    }
-
-    /**
-     * Sets the node
-     * 
-     * @param n
-     *            Node
-     */
-    public void set(Node n) {
-        this.node = n;
-        // Clear the term for now
-        // We only convert the Node to a term as and when we want to write it
-        // out in order to not waste effort if the value is never written out
-        this.term.clear();
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        // Clear previous value
-        this.node = null;
-        this.term.clear();
-
-        // Read in the new value
-        int termLength = input.readInt();
-        byte[] buffer = new byte[termLength];
-        input.readFully(buffer);
-        try {
-            ThriftConverter.fromBytes(buffer, this.term);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-
-        // Note that we don't convert it back into a Node at this time
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        // May not yet have prepared the Thrift term
-        if (!this.term.isSet()) {
-            if (this.node == null) {
-                this.term.setUndefined(TRDF.UNDEF);
-            } else {
-                ThriftConvert.toThrift(this.node, null, this.term, false);
-            }
-        }
-
-        // Write out the Thrift term
-        byte[] buffer;
-        try {
-            buffer = ThriftConverter.toBytes(this.term);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        output.writeInt(buffer.length);
-        output.write(buffer);
-    }
-
-    @Override
-    public int compareTo(NodeWritable other) {
-        // Use get() rather than accessing the field directly because the node
-        // field is lazily instantiated from the Thrift term
-        return NodeUtils.compareRDFTerms(this.get(), other.get());
-    }
-
-    @Override
-    public String toString() {
-        // Use get() rather than accessing the field directly because the node
-        // field is lazily instantiated from the Thrift term
-        Node n = this.get();
-        if (n == null)
-            return "";
-        return n.toString();
-    }
-
-    @Override
-    public int hashCode() {
-        // Use get() rather than accessing the field directly because the node
-        // field is lazily instantiated from the Thrift term
-        Node n = this.get();
-        return n != null ? this.get().hashCode() : 0;
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        if (!(other instanceof NodeWritable))
-            return false;
-        return this.compareTo((NodeWritable) other) == 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
deleted file mode 100644
index 3d9dd00..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
-import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
-import org.apache.jena.riot.thrift.ThriftConvert;
-import org.apache.jena.riot.thrift.wire.RDF_Quad;
-import org.apache.thrift.TException;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A writable quad
- */
-public class QuadWritable extends AbstractNodeTupleWritable<Quad> {
-
-    static {
-        WritableComparator.define(QuadWritable.class, new SimpleBinaryComparator());
-    }
-
-    private RDF_Quad quad = new RDF_Quad();
-
-    /**
-     * Creates a new empty instance
-     */
-    public QuadWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance with the given value
-     * 
-     * @param q
-     *            Quad
-     */
-    public QuadWritable(Quad q) {
-        super(q);
-    }
-
-    /**
-     * Creates a new instance from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static QuadWritable read(DataInput input) throws IOException {
-        QuadWritable q = new QuadWritable();
-        q.readFields(input);
-        return q;
-    }
-
-    @Override
-    public void set(Quad tuple) {
-        super.set(tuple);
-        this.quad.clear();
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        this.quad.clear();
-        int tripleLength = input.readInt();
-        byte[] buffer = new byte[tripleLength];
-        input.readFully(buffer);
-        try {
-            ThriftConverter.fromBytes(buffer, this.quad);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        this.setInternal(new Quad(ThriftConvert.convert(this.quad.getG()), ThriftConvert.convert(this.quad.getS()),
-                ThriftConvert.convert(this.quad.getP()), ThriftConvert.convert(this.quad.getO())));
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        if (this.get() == null)
-            throw new IOException(
-                    "Null quads cannot be written using this class, consider using NodeTupleWritable instead");
-
-        // May not have yet prepared the Thrift triple
-        if (!this.quad.isSetS()) {
-            Quad tuple = this.get();
-            this.quad.setG(ThriftConvert.convert(tuple.getGraph(), false));
-            this.quad.setS(ThriftConvert.convert(tuple.getSubject(), false));
-            this.quad.setP(ThriftConvert.convert(tuple.getPredicate(), false));
-            this.quad.setO(ThriftConvert.convert(tuple.getObject(), false));
-        }
-
-        byte[] buffer;
-        try {
-            buffer = ThriftConverter.toBytes(this.quad);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        output.writeInt(buffer.length);
-        output.write(buffer);
-    }
-
-    @Override
-    protected Quad createTuple(Node[] ns) {
-        if (ns.length != 4)
-            throw new IllegalArgumentException(String.format(
-                    "Incorrect number of nodes to form a quad - got %d but expected 4", ns.length));
-        return new Quad(ns[0], ns[1], ns[2], ns[3]);
-    }
-
-    @Override
-    protected Node[] createNodes(Quad tuple) {
-        return new Node[] { tuple.getGraph(), tuple.getSubject(), tuple.getPredicate(), tuple.getObject() };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
deleted file mode 100644
index a17052b..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
-import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
-import org.apache.jena.riot.thrift.ThriftConvert;
-import org.apache.jena.riot.thrift.wire.RDF_Triple;
-import org.apache.thrift.TException;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A writable triple
- * 
- * 
- * 
- */
-public class TripleWritable extends AbstractNodeTupleWritable<Triple> {
-    
-    static {
-        WritableComparator.define(TripleWritable.class, new SimpleBinaryComparator());
-    }
-
-    private RDF_Triple triple = new RDF_Triple();
-
-    /**
-     * Creates a new instance using the default NTriples node formatter
-     */
-    public TripleWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance with a given value that uses a specific node
-     * formatter
-     * 
-     * @param t
-     *            Triple
-     */
-    public TripleWritable(Triple t) {
-        super(t);
-    }
-
-    /**
-     * Creates a new instance from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static TripleWritable read(DataInput input) throws IOException {
-        TripleWritable t = new TripleWritable();
-        t.readFields(input);
-        return t;
-    }
-
-    @Override
-    public void set(Triple tuple) {
-        super.set(tuple);
-        this.triple.clear();
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        this.triple.clear();
-        int tripleLength = input.readInt();
-        byte[] buffer = new byte[tripleLength];
-        input.readFully(buffer);
-        try {
-            ThriftConverter.fromBytes(buffer, this.triple);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        this.setInternal(new Triple(ThriftConvert.convert(this.triple.getS()),
-                ThriftConvert.convert(this.triple.getP()), ThriftConvert.convert(this.triple.getO())));
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        if (this.get() == null)
-            throw new IOException(
-                    "Null triples cannot be written using this class, consider using NodeTupleWritable instead");
-        
-        // May not have yet prepared the Thrift triple
-        if (!this.triple.isSetS()) {
-            Triple tuple = this.get();
-            this.triple.setS(ThriftConvert.convert(tuple.getSubject(), false));
-            this.triple.setP(ThriftConvert.convert(tuple.getPredicate(), false));
-            this.triple.setO(ThriftConvert.convert(tuple.getObject(), false));
-        }
-
-        byte[] buffer;
-        try {
-            buffer = ThriftConverter.toBytes(this.triple);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        output.writeInt(buffer.length);
-        output.write(buffer);
-    }
-
-    @Override
-    protected Triple createTuple(Node[] ns) {
-        if (ns.length != 3)
-            throw new IllegalArgumentException(String.format(
-                    "Incorrect number of nodes to form a triple - got %d but expected 3", ns.length));
-        return new Triple(ns[0], ns[1], ns[2]);
-    }
-
-    @Override
-    protected Node[] createNodes(Triple tuple) {
-        return new Node[] { tuple.getSubject(), tuple.getPredicate(), tuple.getObject() };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
deleted file mode 100644
index 6c46714..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.types.comparators;
-
-import org.apache.hadoop.io.WritableComparator;
-
-/**
- * A general purpose comparator that may be used with any types which can be
- * compared directly on their binary encodings
- */
-public class SimpleBinaryComparator extends WritableComparator {
-
-    @Override
-    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-        return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
deleted file mode 100644
index 0675afc..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * 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.types.converters;
-
-import java.io.ByteArrayOutputStream;
-
-import org.apache.jena.riot.thrift.wire.RDF_Quad;
-import org.apache.jena.riot.thrift.wire.RDF_Term;
-import org.apache.jena.riot.thrift.wire.RDF_Triple;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.TIOStreamTransport;
-import org.apache.thrift.transport.TMemoryInputTransport;
-import org.apache.thrift.transport.TTransport;
-
-/**
- * Helper for converting between the binary representation of Nodes, Triples and
- * Quads and their Jena API equivalents
- * 
- */
-public class ThriftConverter {
-
-    private static ThreadLocal<TMemoryInputTransport> inputTransports = new ThreadLocal<>();
-    private static ThreadLocal<TProtocol> inputProtocols = new ThreadLocal<>();
-
-    private static ThreadLocal<ByteArrayOutputStream> outputStreams = new ThreadLocal<>();
-    private static ThreadLocal<TTransport> outputTransports = new ThreadLocal<>();
-    private static ThreadLocal<TProtocol> outputProtocols = new ThreadLocal<>();
-
-    private static TMemoryInputTransport getInputTransport() {
-        TMemoryInputTransport transport = inputTransports.get();
-        if (transport != null)
-            return transport;
-
-        transport = new TMemoryInputTransport();
-        inputTransports.set(transport);
-        return transport;
-    }
-
-    private static TProtocol getInputProtocol() {
-        TProtocol protocol = inputProtocols.get();
-        if (protocol != null)
-            return protocol;
-
-        protocol = new TCompactProtocol(getInputTransport());
-        inputProtocols.set(protocol);
-        return protocol;
-    }
-
-    private static ByteArrayOutputStream getOutputStream() {
-        ByteArrayOutputStream output = outputStreams.get();
-        if (output != null)
-            return output;
-
-        output = new ByteArrayOutputStream();
-        outputStreams.set(output);
-        return output;
-    }
-
-    private static TTransport getOutputTransport() {
-        TTransport transport = outputTransports.get();
-        if (transport != null)
-            return transport;
-
-        transport = new TIOStreamTransport(getOutputStream());
-        outputTransports.set(transport);
-        return transport;
-    }
-
-    private static TProtocol getOutputProtocol() {
-        TProtocol protocol = outputProtocols.get();
-        if (protocol != null)
-            return protocol;
-
-        protocol = new TCompactProtocol(getOutputTransport());
-        outputProtocols.set(protocol);
-        return protocol;
-    }
-
-    public static byte[] toBytes(RDF_Term term) throws TException {
-        ByteArrayOutputStream output = getOutputStream();
-        output.reset();
-
-        TProtocol protocol = getOutputProtocol();
-        term.write(protocol);
-
-        return output.toByteArray();
-    }
-
-    public static void fromBytes(byte[] bs, RDF_Term term) throws TException {
-        TMemoryInputTransport transport = getInputTransport();
-        transport.reset(bs);
-        TProtocol protocol = getInputProtocol();
-        term.read(protocol);
-    }
-
-    public static void fromBytes(byte[] buffer, RDF_Triple triple) throws TException {
-        TMemoryInputTransport transport = getInputTransport();
-        transport.reset(buffer);
-        TProtocol protocol = getInputProtocol();
-        triple.read(protocol);
-    }
-
-    public static byte[] toBytes(RDF_Triple triple) throws TException {
-        ByteArrayOutputStream output = getOutputStream();
-        output.reset();
-
-        TProtocol protocol = getOutputProtocol();
-        triple.write(protocol);
-
-        return output.toByteArray();
-    }
-
-    public static void fromBytes(byte[] buffer, RDF_Quad quad) throws TException {
-        TMemoryInputTransport transport = getInputTransport();
-        transport.reset(buffer);
-        TProtocol protocol = getInputProtocol();
-        quad.read(protocol);
-    }
-
-    public static byte[] toBytes(RDF_Quad quad) throws TException {
-        ByteArrayOutputStream output = getOutputStream();
-        output.reset();
-
-        TProtocol protocol = getOutputProtocol();
-        quad.write(protocol);
-
-        return output.toByteArray();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java b/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
deleted file mode 100644
index 7214b14..0000000
--- a/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * 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.types;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
-import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
-import org.junit.Assert;
-import org.junit.Test;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * Tests for {@link CharacteristicWritable} and
- * {@link CharacteristicSetWritable}
- * 
- * 
- * 
- */
-public class CharacteristicTests {
-
-    /**
-     * Checks whether a writable round trips successfully
-     * 
-     * @param cw
-     *            Characteristic writable
-     * @throws IOException
-     */
-    private void checkRoundTrip(CharacteristicWritable cw) throws IOException {
-        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        DataOutputStream output = new DataOutputStream(outputStream);
-        cw.write(output);
-
-        ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray());
-        DataInputStream input = new DataInputStream(inputStream);
-        CharacteristicWritable actual = CharacteristicWritable.read(input);
-        Assert.assertEquals(cw, actual);
-    }
-
-    /**
-     * Tests characteristic round tripping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_writable_01() throws IOException {
-        Node n = NodeFactory.createURI("http://example.org");
-        CharacteristicWritable expected = new CharacteristicWritable(n);
-        Assert.assertEquals(1, expected.getCount().get());
-
-        this.checkRoundTrip(expected);
-    }
-
-    /**
-     * Tests characteristic properties
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_writable_02() throws IOException {
-        Node n = NodeFactory.createURI("http://example.org");
-        CharacteristicWritable cw1 = new CharacteristicWritable(n);
-        CharacteristicWritable cw2 = new CharacteristicWritable(n, 100);
-        this.checkRoundTrip(cw1);
-        this.checkRoundTrip(cw2);
-
-        // Should still be equal since equality is only on the node not the
-        // count
-        Assert.assertEquals(cw1, cw2);
-    }
-
-    /**
-     * Tests characteristic properties
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_writable_03() throws IOException {
-        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
-        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
-        this.checkRoundTrip(cw1);
-        this.checkRoundTrip(cw2);
-
-        // Should not be equal as different nodes
-        Assert.assertNotEquals(cw1, cw2);
-    }
-
-    /**
-     * Checks that a writable round trips
-     * 
-     * @param set
-     *            Characteristic set
-     * @throws IOException
-     */
-    private void checkRoundTrip(CharacteristicSetWritable set) throws IOException {
-        // Test round trip
-        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        DataOutputStream output = new DataOutputStream(outputStream);
-        set.write(output);
-
-        ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray());
-        DataInputStream input = new DataInputStream(inputStream);
-        CharacteristicSetWritable actual = CharacteristicSetWritable.read(input);
-        Assert.assertEquals(set, actual);
-    }
-
-    /**
-     * Checks a characteristic set
-     * 
-     * @param set
-     *            Set
-     * @param expectedItems
-     *            Expected number of characteristics
-     * @param expectedCounts
-     *            Expected counts for characteristics
-     */
-    protected final void checkCharacteristicSet(CharacteristicSetWritable set, int expectedItems, long[] expectedCounts) {
-        Assert.assertEquals(expectedItems, set.size());
-        Assert.assertEquals(expectedItems, expectedCounts.length);
-        Iterator<CharacteristicWritable> iter = set.getCharacteristics();
-        int i = 0;
-        while (iter.hasNext()) {
-            CharacteristicWritable cw = iter.next();
-            Assert.assertEquals(expectedCounts[i], cw.getCount().get());
-            i++;
-        }
-    }
-
-    /**
-     * Tests characteristic sets
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_writable_01() throws IOException {
-        CharacteristicSetWritable set = new CharacteristicSetWritable();
-
-        // Add some characteristics
-        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
-        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
-        set.add(cw1);
-        set.add(cw2);
-        this.checkCharacteristicSet(set, 2, new long[] { 1, 1 });
-        this.checkRoundTrip(set);
-    }
-
-    /**
-     * Tests characteristic sets
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_writable_02() throws IOException {
-        CharacteristicSetWritable set = new CharacteristicSetWritable();
-
-        // Add some characteristics
-        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
-        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"), 2);
-        set.add(cw1);
-        set.add(cw2);
-        this.checkCharacteristicSet(set, 1, new long[] { 3 });
-        this.checkRoundTrip(set);
-    }
-    
-    /**
-     * Tests characteristic sets
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_writable_03() throws IOException {
-        CharacteristicSetWritable set1 = new CharacteristicSetWritable();
-        CharacteristicSetWritable set2 = new CharacteristicSetWritable();
-
-        // Add some characteristics
-        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
-        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
-        set1.add(cw1);
-        set2.add(cw2);
-        this.checkCharacteristicSet(set1, 1, new long[] { 1 });
-        this.checkCharacteristicSet(set2, 1, new long[] { 1 });
-        this.checkRoundTrip(set1);
-        this.checkRoundTrip(set2);
-        
-        Assert.assertNotEquals(set1, set2);
-    }
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java
deleted file mode 100644
index ba47765..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.namespaces;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-import com.hp.hpl.jena.graph.Node;
-
-/**
- * Abstract mapper class for mappers which split node tuple values and extract
- * the namespace URIs they use and outputs pairs of namespaces keys with a long
- * value of 1. Can be used in conjunction with a {@link TextCountReducer} to
- * count the usages of each unique namespace.
- * 
- * 
- * 
- * @param <TKey>
- * @param <TValue>
- * @param <T>
- */
-public abstract class AbstractNodeTupleNamespaceCountMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        Mapper<TKey, T, Text, LongWritable> {
-
-    private LongWritable initialCount = new LongWritable(1);
-    protected static final String NO_NAMESPACE = null;
-
-    @Override
-    protected void map(TKey key, T value, Context context) throws IOException, InterruptedException {
-        NodeWritable[] ns = this.getNodes(value);
-        for (NodeWritable n : ns) {
-            String namespace = this.extractNamespace(n);
-            if (namespace != null) {
-                context.write(new Text(namespace), this.initialCount);
-            }
-        }
-    }
-
-    /**
-     * Extracts the namespace from a node
-     * <p>
-     * Finds the URI for the node (if any) and then invokes
-     * {@link #extractNamespace(String)} to extract the actual namespace URI.
-     * </p>
-     * <p>
-     * Derived classes may override this to change the logic of how namespaces
-     * are extracted.
-     * </p>
-     * 
-     * @param nw
-     *            Node
-     * @return Namespace
-     */
-    protected String extractNamespace(NodeWritable nw) {
-        Node n = nw.get();
-        if (n.isBlank() || n.isVariable())
-            return NO_NAMESPACE;
-        if (n.isLiteral()) {
-            String dtUri = n.getLiteralDatatypeURI();
-            if (dtUri == null)
-                return NO_NAMESPACE;
-            return extractNamespace(dtUri);
-        }
-        return extractNamespace(n.getURI());
-    }
-
-    /**
-     * Extracts the namespace from a URI
-     * <p>
-     * First tries to extract a hash based namespace. If that is not possible it
-     * tries to extract a slash based namespace, if this is not possible then
-     * the full URI is returned.
-     * </p>
-     * <p>
-     * Derived classes may override this to change the logic of how namespaces
-     * are extracted.
-     * </p>
-     * 
-     * @param uri
-     *            URI
-     * @return Namespace
-     */
-    protected String extractNamespace(String uri) {
-        if (uri.contains("#")) {
-            // Extract hash namespace
-            return uri.substring(0, uri.lastIndexOf('#') + 1);
-        } else if (uri.contains("/")) {
-            // Ensure that this is not immediately after the scheme component or
-            // at end of URI
-            int index = uri.lastIndexOf('/');
-            int schemeSepIndex = uri.indexOf(':');
-            if (index - schemeSepIndex <= 2 || index == uri.length() - 1) {
-                // Use full URI
-                return uri;
-            }
-
-            // Otherwise safe to extract slash namespace
-            return uri.substring(0, uri.lastIndexOf('/') + 1);
-        } else {
-            // Use full URI
-            return uri;
-        }
-    }
-
-    /**
-     * Gets the nodes of the tuple whose namespaces are to be counted
-     * 
-     * @param tuple
-     *            Tuple
-     * @return Nodes
-     */
-    protected abstract NodeWritable[] getNodes(T tuple);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java
deleted file mode 100644
index 97ee546..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.namespaces;
-
-import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A mapper for counting namespace usages within quads designed primarily for
- * use in conjunction with the {@link TextCountReducer}
- * 
- * 
- * 
- * @param <TKey>
- */
-public class QuadNamespaceCountMapper<TKey> extends AbstractNodeTupleNamespaceCountMapper<TKey, Quad, QuadWritable> {
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        Quad q = tuple.get();
-        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
-                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java
deleted file mode 100644
index eacef7a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.namespaces;
-
-import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A mapper for counting namespace usages within triples designed primarily for
- * use in conjunction with the {@link TextCountReducer}
- * 
- * 
- * 
- * @param <TKey>
- */
-public class TripleNamespaceCountMapper<TKey> extends AbstractNodeTupleNamespaceCountMapper<TKey, Triple, TripleWritable> {
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        Triple t = tuple.get();
-        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
-                new NodeWritable(t.getObject()) };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java
deleted file mode 100644
index ef19623..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
-
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * A mapper for counting object node usages within quads designed primarily for
- * use in conjunction with {@link NodeCountReducer}
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadObjectCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        return new NodeWritable[] { new NodeWritable(tuple.get().getObject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java
deleted file mode 100644
index 0ef0731..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
-
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * A mapper for counting predicate node usages within quads designed primarily
- * for use in conjunction with {@link NodeCountReducer}
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadPredicateCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        return new NodeWritable[] { new NodeWritable(tuple.get().getPredicate()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java
deleted file mode 100644
index 2b48e79..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
-
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * A mapper for counting subject node usages within quads designed primarily for use
- * in conjunction with {@link NodeCountReducer}
- * 
- * 
- * 
- * @param <TKey> Key type
- */
-public class QuadSubjectCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        return new NodeWritable[] { new NodeWritable(tuple.get().getSubject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java
deleted file mode 100644
index 16250eb..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
-
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * A mapper for counting object node usages within triples designed primarily for use
- * in conjunction with {@link NodeCountReducer}
- * 
- * 
- * 
- * @param <TKey> Key type
- */
-public class TripleObjectCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        return new NodeWritable[] { new NodeWritable(tuple.get().getObject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java
deleted file mode 100644
index de97a9b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
-
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * A mapper for counting predicate node usages within triples designed primarily
- * for use in conjunction with {@link NodeCountReducer}
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class TriplePredicateCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        return new NodeWritable[] { new NodeWritable(tuple.get().getPredicate()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java
deleted file mode 100644
index 5dc3838..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
-
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * A mapper for counting subject node usages within triples designed primarily for use
- * in conjunction with {@link NodeCountReducer}
- * 
- * 
- * 
- * @param <TKey> Key type
- */
-public class TripleSubjectCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        return new NodeWritable[] { new NodeWritable(tuple.get().getSubject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java
deleted file mode 100644
index 1ecec75..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-
-
-/**
- * Abstract mapper implementation which helps in filtering tuples from the
- * input, derived implementations provide an implementation of the
- * {@link #accepts(TKey, T)}
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-@SuppressWarnings("javadoc")
-public abstract class AbstractNodeTupleFilterMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        Mapper<TKey, T, TKey, T> {
-
-    private boolean invert = false;
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.invert = context.getConfiguration().getBoolean(RdfMapReduceConstants.FILTER_INVERT, this.invert);
-    }
-
-    @Override
-    protected final void map(TKey key, T value, Context context) throws IOException, InterruptedException {
-        if (this.accepts(key, value)) {
-            if (!this.invert)
-                context.write(key, value);
-        } else if (this.invert) {
-            context.write(key, value);
-        }
-    }
-
-    /**
-     * Gets whether the mapper accepts the key value pair and will pass it as
-     * output
-     * 
-     * @param key
-     *            Key
-     * @param tuple
-     *            Tuple value
-     * @return True if the mapper accepts the given key value pair, false
-     *         otherwise
-     */
-    protected abstract boolean accepts(TKey key, T tuple);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java
deleted file mode 100644
index 3caf051..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract mapper implementation for filtering quads
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public abstract class AbstractQuadFilterMapper<TKey> extends AbstractNodeTupleFilterMapper<TKey, Quad, QuadWritable> {
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java
deleted file mode 100644
index e99e369..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract mapper implementation for filtering triples
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public abstract class AbstractTripleFilterMapper<TKey> extends AbstractNodeTupleFilterMapper<TKey, Triple, TripleWritable> {
-
-}

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

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

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

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

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
deleted file mode 100644
index f5b5876..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
-
-import org.apache.jena.hadoop.rdf.mapreduce.filter.AbstractQuadFilterMapper;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract triple filter that filters quads based on different criteria for
- * each position (graph, subject, predicate and object) within the quad.
- * <p>
- * By default this implementation eliminates all quads it sees, derived
- * implementations need to override one or more of the specific accept methods
- * in order to actually accept some triples. See
- * {@link QuadFilterByPredicateMapper} for an example implementation.
- * </p>
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public abstract class AbstractQuadFilterByPositionMapper<TKey> extends AbstractQuadFilterMapper<TKey> {
-
-    @Override
-    protected final boolean accepts(Object key, QuadWritable tuple) {
-        Quad q = tuple.get();
-        if (!this.acceptsAllGraphs()) {
-            Node g = q.getGraph();
-            if (!this.acceptsGraph(g))
-                return false;
-        }
-        if (!this.acceptsAllSubjects()) {
-            Node s = q.getSubject();
-            if (!this.acceptsSubject(s))
-                return false;
-        }
-        if (!this.acceptsAllPredicates()) {
-            Node p = q.getPredicate();
-            if (!this.acceptsPredicate(p))
-                return false;
-        }
-        if (!this.acceptsAllObjects()) {
-            Node o = q.getObject();
-            if (!this.acceptsObject(o))
-                return false;
-        }
-
-        return true;
-    }
-
-    /**
-     * Gets whether this filter accepts all graphs, if false then the
-     * {@link #acceptsGraph(Node)} method will be called to determine if a
-     * specific graph is acceptable
-     * <p>
-     * Default behaviour if not overridden is to return {@code false}
-     * </p>
-     * 
-     * @return True if all graphs are accepted, false otherwise
-     */
-    protected boolean acceptsAllGraphs() {
-        return false;
-    }
-
-    /**
-     * Gets whether a specific graph is acceptable
-     * 
-     * @param graph
-     *            Graph
-     * @return True if accepted, false otherwise
-     */
-    protected boolean acceptsGraph(Node graph) {
-        return false;
-    }
-
-    /**
-     * Gets whether this filter accepts all subjects, if false then the
-     * {@link #acceptsSubject(Node)} method will be called to determine if a
-     * specific subject is acceptable
-     * <p>
-     * Default behaviour if not overridden is to return {@code false}
-     * </p>
-     * 
-     * @return True if all subjects are accepted, false otherwise
-     */
-    protected boolean acceptsAllSubjects() {
-        return false;
-    }
-
-    /**
-     * Gets whether a specific subject is acceptable
-     * 
-     * @param subject
-     *            Subject
-     * @return True if accepted, false otherwise
-     */
-    protected boolean acceptsSubject(Node subject) {
-        return false;
-    }
-
-    /**
-     * Gets whether this filter accepts all predicate, if false then the
-     * {@link #acceptsPredicate(Node)} method will be called to determine if a
-     * specific predicate is acceptable
-     * <p>
-     * Default behaviour if not overridden is to return {@code false}
-     * </p>
-     * 
-     * @return True if all predicates are accepted, false otherwise
-     */
-    protected boolean acceptsAllPredicates() {
-        return false;
-    }
-
-    /**
-     * Gets whether a specific predicate is acceptable
-     * 
-     * @param predicate
-     *            Predicate
-     * @return True if accepted, false otherwise
-     */
-    protected boolean acceptsPredicate(Node predicate) {
-        return false;
-    }
-
-    /**
-     * Gets whether this filter accepts all objects, if false then the
-     * {@link #acceptsObject(Node)} method will be called to determine if a
-     * specific object is acceptable
-     * <p>
-     * Default behaviour if not overridden is to return {@code false}
-     * </p>
-     * 
-     * @return True if all objects are accepted, false otherwise
-     */
-    protected boolean acceptsAllObjects() {
-        return false;
-    }
-
-    /**
-     * Gets whether a specific object is acceptable
-     * 
-     * @param object
-     *            Object
-     * @return True if accepted, false otherwise
-     */
-    protected boolean acceptsObject(Node object) {
-        return false;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java
deleted file mode 100644
index 973d651..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
-
-import org.apache.jena.hadoop.rdf.mapreduce.filter.AbstractTripleFilterMapper;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * An abstract triple filter that filters triples based on different criteria
- * for each position (subject, predicate and object) within the triple.
- * <p>
- * By default this implementation eliminates all triples it sees, derived
- * implementations need to override one or more of the specific accept methods
- * in order to actually accept some triples. See
- * {@link TripleFilterByPredicateUriMapper} for an example implementation.
- * </p>
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public abstract class AbstractTripleFilterByPositionMapper<TKey> extends AbstractTripleFilterMapper<TKey> {
-
-    @Override
-    protected final boolean accepts(Object key, TripleWritable tuple) {
-        Triple t = tuple.get();
-        if (!this.acceptsAllSubjects()) {
-            Node s = t.getSubject();
-            if (!this.acceptsSubject(s))
-                return false;
-        }
-        if (!this.acceptsAllPredicates()) {
-            Node p = t.getPredicate();
-            if (!this.acceptsPredicate(p))
-                return false;
-        }
-        if (!this.acceptsAllObjects()) {
-            Node o = t.getObject();
-            if (!this.acceptsObject(o))
-                return false;
-        }
-
-        return true;
-    }
-
-    /**
-     * Gets whether this filter accepts all subjects, if false then the
-     * {@link #acceptsSubject(Node)} method will be called to determine if a
-     * specific subject is acceptable
-     * <p>
-     * Default behaviour if not overridden is to return {@code false}
-     * </p>
-     * 
-     * @return True if all subjects are accepted, false otherwise
-     */
-    protected boolean acceptsAllSubjects() {
-        return false;
-    }
-
-    /**
-     * Gets whether a specific subject is acceptable
-     * 
-     * @param subject
-     *            Subject
-     * @return True if accepted, false otherwise
-     */
-    protected boolean acceptsSubject(Node subject) {
-        return false;
-    }
-
-    /**
-     * Gets whether this filter accepts all predicate, if false then the
-     * {@link #acceptsPredicate(Node)} method will be called to determine if a
-     * specific predicate is acceptable
-     * <p>
-     * Default behaviour if not overridden is to return {@code false}
-     * </p>
-     * 
-     * @return True if all predicates are accepted, false otherwise
-     */
-    protected boolean acceptsAllPredicates() {
-        return false;
-    }
-
-    /**
-     * Gets whether a specific predicate is acceptable
-     * 
-     * @param predicate
-     *            Predicate
-     * @return True if accepted, false otherwise
-     */
-    protected boolean acceptsPredicate(Node predicate) {
-        return false;
-    }
-
-    /**
-     * Gets whether this filter accepts all objects, if false then the
-     * {@link #acceptsObject(Node)} method will be called to determine if a
-     * specific object is acceptable
-     * <p>
-     * Default behaviour if not overridden is to return {@code false}
-     * </p>
-     * 
-     * @return True if all objects are accepted, false otherwise
-     */
-    protected boolean acceptsAllObjects() {
-        return false;
-    }
-
-    /**
-     * Gets whether a specific object is acceptable
-     * 
-     * @param object
-     *            Object
-     * @return True if accepted, false otherwise
-     */
-    protected boolean acceptsObject(Node object) {
-        return false;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
deleted file mode 100644
index 1c2b29e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * A quad filter which selects quads which have matching subjects
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadFilterByGraphUriMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
-
-    private List<Node> graphs = new ArrayList<Node>();
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-
-        // Get the subject URIs we are filtering on
-        String[] graphUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_GRAPH_URIS);
-        if (graphUris != null) {
-            for (String graphUri : graphUris) {
-                this.graphs.add(NodeFactory.createURI(graphUri));
-            }
-        }
-    }
-
-    @Override
-    protected boolean acceptsAllSubjects() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsGraph(Node graph) {
-        if (this.graphs.size() == 0)
-            return false;
-        return this.graphs.contains(graph);
-    }
-
-    @Override
-    protected boolean acceptsAllPredicates() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsAllObjects() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
deleted file mode 100644
index a3be806..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * A quad filter which selects quads which have matching objects
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadFilterByObjectUriMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
-
-    private List<Node> objects = new ArrayList<Node>();
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-
-        // Get the subject URIs we are filtering on
-        String[] objectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_OBJECT_URIS);
-        if (objectUris != null) {
-            for (String objectUri : objectUris) {
-                this.objects.add(NodeFactory.createURI(objectUri));
-            }
-        }
-    }
-    
-    @Override
-    protected boolean acceptsAllGraphs() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsObject(Node object) {
-        if (this.objects.size() == 0)
-            return false;
-        return this.objects.contains(object);
-    }
-
-    @Override
-    protected boolean acceptsAllPredicates() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsAllSubjects() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
deleted file mode 100644
index 1be64bf..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * A quad filter which selects quads which have matching predicates
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadFilterByPredicateMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
-
-    private List<Node> predicates = new ArrayList<Node>();
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-
-        // Get the predicate URIs we are filtering on
-        String[] predicateUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS);
-        if (predicateUris != null) {
-            for (String predicateUri : predicateUris) {
-                this.predicates.add(NodeFactory.createURI(predicateUri));
-            }
-        }
-    }
-    
-    @Override
-    protected boolean acceptsAllGraphs() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsAllSubjects() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsPredicate(Node predicate) {
-        if (this.predicates.size() == 0)
-            return false;
-        return this.predicates.contains(predicate);
-    }
-
-    @Override
-    protected boolean acceptsAllObjects() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
deleted file mode 100644
index 2a4f37b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * A quad filter which selects quads which have matching subjects
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadFilterBySubjectUriMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
-
-    private List<Node> subjects = new ArrayList<Node>();
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-
-        // Get the subject URIs we are filtering on
-        String[] subjectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_SUBJECT_URIS);
-        if (subjectUris != null) {
-            for (String subjectUri : subjectUris) {
-                this.subjects.add(NodeFactory.createURI(subjectUri));
-            }
-        }
-    }
-    
-    @Override
-    protected boolean acceptsAllGraphs() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsSubject(Node subject) {
-        if (this.subjects.size() == 0)
-            return false;
-        return this.subjects.contains(subject);
-    }
-
-    @Override
-    protected boolean acceptsAllPredicates() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsAllObjects() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java
deleted file mode 100644
index 035aabc..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * A triple filter which selects triples which have matching objects
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class TripleFilterByObjectUriMapper<TKey> extends AbstractTripleFilterByPositionMapper<TKey> {
-
-    private List<Node> objects = new ArrayList<Node>();
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-
-        // Get the subject URIs we are filtering on
-        String[] objectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_OBJECT_URIS);
-        if (objectUris != null) {
-            for (String objectUri : objectUris) {
-                this.objects.add(NodeFactory.createURI(objectUri));
-            }
-        }
-    }
-
-    @Override
-    protected boolean acceptsObject(Node object) {
-        if (this.objects.size() == 0)
-            return false;
-        return this.objects.contains(object);
-    }
-
-    @Override
-    protected boolean acceptsAllPredicates() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsAllSubjects() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java
deleted file mode 100644
index 061a3e7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * A triple filter which selects triples which have matching predicates
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class TripleFilterByPredicateUriMapper<TKey> extends AbstractTripleFilterByPositionMapper<TKey> {
-
-    private List<Node> predicates = new ArrayList<Node>();
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-
-        // Get the predicate URIs we are filtering on
-        String[] predicateUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS);
-        if (predicateUris != null) {
-            for (String predicateUri : predicateUris) {
-                this.predicates.add(NodeFactory.createURI(predicateUri));
-            }
-        }
-    }
-
-    @Override
-    protected boolean acceptsAllSubjects() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsPredicate(Node predicate) {
-        if (this.predicates.size() == 0)
-            return false;
-        return this.predicates.contains(predicate);
-    }
-
-    @Override
-    protected boolean acceptsAllObjects() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java
deleted file mode 100644
index d6ac375..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterBySubjectUriMapper.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * A triple filter which selects triples which have matching subjects
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class TripleFilterBySubjectUriMapper<TKey> extends AbstractTripleFilterByPositionMapper<TKey> {
-
-    private List<Node> subjects = new ArrayList<Node>();
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-
-        // Get the subject URIs we are filtering on
-        String[] subjectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_SUBJECT_URIS);
-        if (subjectUris != null) {
-            for (String subjectUri : subjectUris) {
-                this.subjects.add(NodeFactory.createURI(subjectUri));
-            }
-        }
-    }
-
-    @Override
-    protected boolean acceptsSubject(Node subject) {
-        if (this.subjects.size() == 0)
-            return false;
-        return this.subjects.contains(subject);
-    }
-
-    @Override
-    protected boolean acceptsAllPredicates() {
-        return true;
-    }
-
-    @Override
-    protected boolean acceptsAllObjects() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java
deleted file mode 100644
index e3d51e4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractNodeTupleGroupingMapper.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-
-/**
- * Abstract mapper implementation which helps in grouping tuples by assigning
- * them a {@link NodeWritable} key in place of their existing key. Derived
- * implementations of this may select the key based on some component of the
- * tuple or by other custom logic.
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleGroupingMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        Mapper<TKey, T, NodeWritable, T> {
-
-    @Override
-    protected final void map(TKey key, T value, Context context) throws IOException, InterruptedException {
-        NodeWritable newKey = this.selectKey(value);
-        context.write(newKey, value);
-    }
-
-    /**
-     * Gets the key to associated with the tuple
-     * 
-     * @param tuple
-     *            Tuple
-     * @return Node to use as key
-     */
-    protected abstract NodeWritable selectKey(T tuple);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java
deleted file mode 100644
index 2b96110..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractQuadGroupingMapper.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract mapper implementation which helps in grouping quads by assigning
- * them a {@link NodeWritable} key in place of their existing key. Derived
- * implementations of this may select the key based on some component of the
- * quad or by other custom logic.
- * 
- * 
- * 
- * @param <TKey>
- */
-public abstract class AbstractQuadGroupingMapper<TKey> extends AbstractNodeTupleGroupingMapper<TKey, Quad, QuadWritable> {
-
-    protected final NodeWritable selectKey(QuadWritable tuple) {
-        return this.selectKey(tuple.get());
-    }
-
-    /**
-     * Selects the key to use
-     * 
-     * @param quad
-     *            Quad
-     * @return Key to use
-     */
-    protected abstract NodeWritable selectKey(Quad quad);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java
deleted file mode 100644
index 3f44eb0..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/AbstractTripleGroupingMapper.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract mapper implementation which helps in grouping triples by assigning
- * them a {@link NodeWritable} key in place of their existing key. Derived
- * implementations of this may select the key based on some component of the
- * triple or by other custom logic.
- * 
- * 
- * 
- * @param <TKey>
- */
-public abstract class AbstractTripleGroupingMapper<TKey> extends AbstractNodeTupleGroupingMapper<TKey, Triple, TripleWritable> {
-
-    @Override
-    protected final NodeWritable selectKey(TripleWritable tuple) {
-        return this.selectKey(tuple.get());
-    }
-    
-    protected abstract NodeWritable selectKey(Triple triple);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java
deleted file mode 100644
index 3b9fd8d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByGraphMapper.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A mapper which assists in grouping quads by graph by reassigning their keys
- * to be their graphs
- * 
- * 
- * 
- * @param <TKey>
- */
-public class QuadGroupByGraphMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
-
-    @Override
-    protected NodeWritable selectKey(Quad quad) {
-        return new NodeWritable(quad.getGraph());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java
deleted file mode 100644
index eb26e0b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByObjectMapper.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A mapper which assists in grouping quads by object by reassigning their keys
- * to be their objects
- * 
- * 
- * 
- * @param <TKey>
- */
-public class QuadGroupByObjectMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
-
-    @Override
-    protected NodeWritable selectKey(Quad quad) {
-        return new NodeWritable(quad.getObject());
-    }
-}


[22/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
Rebrand to Jena Elephas per community vote


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

Branch: refs/heads/hadoop-rdf
Commit: a6c0fefc7bec24484720748b6c5a2c1cc1b08919
Parents: 657fae5
Author: Rob Vesse <rv...@apache.org>
Authored: Mon Jan 5 14:55:12 2015 +0000
Committer: Rob Vesse <rv...@apache.org>
Committed: Mon Jan 5 14:55:12 2015 +0000

----------------------------------------------------------------------
 jena-hadoop-rdf/hadoop-rdf-common/pom.xml       |  54 --
 .../rdf/types/AbstractNodeTupleWritable.java    | 193 -----
 .../rdf/types/CharacteristicSetWritable.java    | 298 --------
 .../rdf/types/CharacteristicWritable.java       | 160 ----
 .../hadoop/rdf/types/NodeTupleWritable.java     |  80 --
 .../jena/hadoop/rdf/types/NodeWritable.java     | 188 -----
 .../jena/hadoop/rdf/types/QuadWritable.java     | 136 ----
 .../jena/hadoop/rdf/types/TripleWritable.java   | 138 ----
 .../comparators/SimpleBinaryComparator.java     |  34 -
 .../rdf/types/converters/ThriftConverter.java   | 147 ----
 .../rdf/io/types/CharacteristicTests.java       | 210 -----
 .../jena/hadoop/rdf/io/types/RdfTypesTest.java  | 406 ----------
 jena-hadoop-rdf/hadoop-rdf-io/pom.xml           | 100 ---
 .../jena/hadoop/rdf/io/HadoopIOConstants.java   |  49 --
 .../jena/hadoop/rdf/io/RdfIOConstants.java      |  81 --
 .../io/input/AbstractNLineFileInputFormat.java  |  70 --
 .../io/input/AbstractWholeFileInputFormat.java  |  42 -
 .../hadoop/rdf/io/input/QuadsInputFormat.java   |  46 --
 .../hadoop/rdf/io/input/TriplesInputFormat.java |  42 -
 .../rdf/io/input/TriplesOrQuadsInputFormat.java |  47 --
 .../io/input/jsonld/JsonLDQuadInputFormat.java  |  39 -
 .../input/jsonld/JsonLDTripleInputFormat.java   |  39 -
 .../input/nquads/BlockedNQuadsInputFormat.java  |  53 --
 .../rdf/io/input/nquads/NQuadsInputFormat.java  |  46 --
 .../nquads/WholeFileNQuadsInputFormat.java      |  51 --
 .../ntriples/BlockedNTriplesInputFormat.java    |  53 --
 .../io/input/ntriples/NTriplesInputFormat.java  |  46 --
 .../ntriples/WholeFileNTriplesInputFormat.java  |  51 --
 .../io/input/rdfjson/RdfJsonInputFormat.java    |  46 --
 .../rdf/io/input/rdfxml/RdfXmlInputFormat.java  |  46 --
 .../AbstractBlockBasedNodeTupleReader.java      | 344 ---------
 .../readers/AbstractBlockBasedQuadReader.java   |  51 --
 .../readers/AbstractBlockBasedTripleReader.java |  51 --
 .../AbstractLineBasedNodeTupleReader.java       | 265 -------
 .../readers/AbstractLineBasedQuadReader.java    |  50 --
 .../readers/AbstractLineBasedTripleReader.java  |  51 --
 .../rdf/io/input/readers/AbstractRdfReader.java | 108 ---
 .../AbstractWholeFileNodeTupleReader.java       | 328 --------
 .../readers/AbstractWholeFileQuadReader.java    |  51 --
 .../readers/AbstractWholeFileTripleReader.java  |  51 --
 .../rdf/io/input/readers/QuadsReader.java       |  49 --
 .../io/input/readers/TriplesOrQuadsReader.java  |  72 --
 .../rdf/io/input/readers/TriplesReader.java     |  49 --
 .../io/input/readers/TriplesToQuadsReader.java  | 102 ---
 .../input/readers/jsonld/JsonLDQuadReader.java  |  32 -
 .../readers/jsonld/JsonLDTripleReader.java      |  30 -
 .../readers/nquads/BlockedNQuadsReader.java     |  45 --
 .../io/input/readers/nquads/NQuadsReader.java   |  49 --
 .../readers/nquads/WholeFileNQuadsReader.java   |  42 -
 .../readers/ntriples/BlockedNTriplesReader.java |  45 --
 .../input/readers/ntriples/NTriplesReader.java  |  48 --
 .../ntriples/WholeFileNTriplesReader.java       |  42 -
 .../io/input/readers/rdfjson/RdfJsonReader.java |  37 -
 .../io/input/readers/rdfxml/RdfXmlReader.java   |  37 -
 .../input/readers/thrift/ThriftQuadReader.java  |  32 -
 .../readers/thrift/ThriftTripleReader.java      |  30 -
 .../rdf/io/input/readers/trig/TriGReader.java   |  37 -
 .../rdf/io/input/readers/trix/TriXReader.java   |  37 -
 .../io/input/readers/turtle/TurtleReader.java   |  37 -
 .../io/input/thrift/ThriftQuadInputFormat.java  |  39 -
 .../input/thrift/ThriftTripleInputFormat.java   |  39 -
 .../rdf/io/input/trig/TriGInputFormat.java      |  46 --
 .../rdf/io/input/trix/TriXInputFormat.java      |  42 -
 .../rdf/io/input/turtle/TurtleInputFormat.java  |  46 --
 .../rdf/io/input/util/BlockInputStream.java     |  94 ---
 .../hadoop/rdf/io/input/util/RdfIOUtils.java    | 101 ---
 .../rdf/io/input/util/TrackableInputStream.java |  38 -
 .../rdf/io/input/util/TrackedInputStream.java   | 124 ---
 .../io/input/util/TrackedPipedQuadsStream.java  |  55 --
 .../io/input/util/TrackedPipedRDFStream.java    |  64 --
 .../input/util/TrackedPipedTriplesStream.java   |  56 --
 .../AbstractBatchedNodeTupleOutputFormat.java   |  55 --
 .../rdf/io/output/AbstractNodeOutputFormat.java |  94 ---
 .../output/AbstractNodeTupleOutputFormat.java   | 109 ---
 .../AbstractStreamRdfNodeTupleOutputFormat.java |  73 --
 .../hadoop/rdf/io/output/QuadsOutputFormat.java |  64 --
 .../io/output/TriplesOrQuadsOutputFormat.java   |  74 --
 .../rdf/io/output/TriplesOutputFormat.java      |  61 --
 .../output/jsonld/JsonLDQuadOutputFormat.java   |  44 --
 .../output/jsonld/JsonLDTripleOutputFormat.java |  44 --
 .../io/output/nquads/NQuadsOutputFormat.java    |  52 --
 .../ntriples/NTriplesNodeOutputFormat.java      |  45 --
 .../output/ntriples/NTriplesOutputFormat.java   |  52 --
 .../io/output/rdfjson/RdfJsonOutputFormat.java  |  52 --
 .../io/output/rdfxml/RdfXmlOutputFormat.java    |  52 --
 .../output/thrift/ThriftQuadOutputFormat.java   |  51 --
 .../output/thrift/ThriftTripleOutputFormat.java |  52 --
 .../io/output/trig/BatchedTriGOutputFormat.java |  54 --
 .../rdf/io/output/trig/TriGOutputFormat.java    |  58 --
 .../rdf/io/output/trix/TriXOutputFormat.java    |  57 --
 .../turtle/BatchedTurtleOutputFormat.java       |  50 --
 .../io/output/turtle/TurtleOutputFormat.java    |  56 --
 .../writers/AbstractBatchedNodeTupleWriter.java | 113 ---
 .../writers/AbstractBatchedQuadWriter.java      |  80 --
 .../writers/AbstractBatchedTripleWriter.java    |  68 --
 .../AbstractLineBasedNodeTupleWriter.java       | 152 ----
 .../writers/AbstractLineBasedQuadWriter.java    |  71 --
 .../writers/AbstractLineBasedTripleWriter.java  |  68 --
 .../io/output/writers/AbstractNodeWriter.java   | 192 -----
 .../AbstractStreamRdfNodeTupleWriter.java       |  71 --
 .../AbstractWholeFileNodeTupleWriter.java       |  96 ---
 .../writers/AbstractWholeFileQuadWriter.java    |  66 --
 .../writers/AbstractWholeFileTripleWriter.java  |  65 --
 .../io/output/writers/QuadsToTriplesWriter.java |  59 --
 .../io/output/writers/StreamRdfQuadWriter.java  |  45 --
 .../output/writers/StreamRdfTripleWriter.java   |  44 --
 .../output/writers/jsonld/JsonLDQuadWriter.java |  38 -
 .../writers/jsonld/JsonLDTripleWriter.java      |  38 -
 .../io/output/writers/nquads/NQuadsWriter.java  |  57 --
 .../writers/ntriples/NTriplesNodeWriter.java    |  59 --
 .../output/writers/ntriples/NTriplesWriter.java |  58 --
 .../output/writers/rdfjson/RdfJsonWriter.java   |  51 --
 .../io/output/writers/rdfxml/RdfXmlWriter.java  |  51 --
 .../output/writers/thrift/ThriftQuadWriter.java |  38 -
 .../writers/thrift/ThriftTripleWriter.java      |  38 -
 .../output/writers/trig/BatchedTriGWriter.java  |  52 --
 .../writers/turtle/BatchedTurtleWriter.java     |  54 --
 .../rdf/io/registry/HadoopRdfIORegistry.java    | 310 --------
 .../hadoop/rdf/io/registry/ReaderFactory.java   |  83 --
 .../hadoop/rdf/io/registry/WriterFactory.java   |  96 ---
 .../readers/AbstractQuadsOnlyReaderFactory.java |  83 --
 .../registry/readers/AbstractReaderFactory.java |  80 --
 .../AbstractTriplesOnlyReaderFactory.java       |  83 --
 .../registry/readers/JsonLDReaderFactory.java   |  49 --
 .../registry/readers/NQuadsReaderFactory.java   |  42 -
 .../registry/readers/NTriplesReaderFactory.java |  38 -
 .../registry/readers/RdfJsonReaderFactory.java  |  41 -
 .../registry/readers/RdfXmlReaderFactory.java   |  40 -
 .../registry/readers/ThriftReaderFactory.java   |  49 --
 .../io/registry/readers/TriGReaderFactory.java  |  42 -
 .../io/registry/readers/TriXReaderFactory.java  |  41 -
 .../registry/readers/TurtleReaderFactory.java   |  40 -
 .../writers/AbstractQuadsOnlyWriterFactory.java |  86 ---
 .../AbstractTriplesOnlyWriterFactory.java       |  85 ---
 .../registry/writers/AbstractWriterFactory.java |  82 --
 .../registry/writers/JsonLDWriterFactory.java   |  52 --
 .../registry/writers/NQuadsWriterFactory.java   |  44 --
 .../registry/writers/NTriplesWriterFactory.java |  44 --
 .../registry/writers/RdfJsonWriterFactory.java  |  43 --
 .../registry/writers/RdfXmlWriterFactory.java   |  44 --
 .../registry/writers/ThriftWriterFactory.java   |  57 --
 .../io/registry/writers/TriGWriterFactory.java  |  45 --
 .../io/registry/writers/TriXWriterFactory.java  |  47 --
 .../registry/writers/TurtleWriterFactory.java   |  45 --
 ...he.jena.hadoop.rdf.io.registry.ReaderFactory |  10 -
 ...he.jena.hadoop.rdf.io.registry.WriterFactory |  10 -
 .../rdf/io/RdfTriplesInputTestMapper.java       |  47 --
 .../AbstractBlockedQuadInputFormatTests.java    |  33 -
 .../AbstractBlockedTripleInputFormatTests.java  |  33 -
 .../AbstractNodeTupleInputFormatTests.java      | 612 ---------------
 .../io/input/AbstractQuadsInputFormatTests.java |  70 --
 .../input/AbstractTriplesInputFormatTests.java  |  72 --
 .../AbstractWholeFileQuadInputFormatTests.java  | 115 ---
 ...AbstractWholeFileTripleInputFormatTests.java | 108 ---
 .../io/input/bnodes/AbstractBlankNodeTests.java | 636 ----------------
 .../bnodes/AbstractTripleBlankNodeTests.java    |  65 --
 .../input/bnodes/JsonLdTripleBlankNodeTest.java |  63 --
 .../io/input/bnodes/NTriplesBlankNodeTest.java  |  58 --
 .../io/input/bnodes/RdfJsonBlankNodeTest.java   |  58 --
 .../io/input/bnodes/RdfThriftBlankNodeTest.java |  68 --
 .../io/input/bnodes/RdfXmlBlankNodeTest.java    |  62 --
 .../io/input/bnodes/TurtleBlankNodeTest.java    |  58 --
 ...ractCompressedNodeTupleInputFormatTests.java |  74 --
 ...AbstractCompressedQuadsInputFormatTests.java |  71 --
 ...stractCompressedTriplesInputFormatTests.java |  71 --
 ...CompressedWholeFileQuadInputFormatTests.java | 150 ----
 ...mpressedWholeFileTripleInputFormatTests.java | 144 ----
 ...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 -
 ...bstractCompressedNQuadsInputFormatTests.java |  68 --
 ...mpressedWholeFileNQuadsInputFormatTests.java |  75 --
 .../nquads/BZipppedNQuadsInputTest.java         |  38 -
 .../BZipppedWholeFileNQuadsInputTest.java       |  37 -
 .../nquads/DeflatedNQuadsInputTest.java         |  37 -
 .../DeflatedWholeFileNQuadsInputTest.java       |  37 -
 .../nquads/GZippedNQuadsInputTest.java          |  38 -
 .../nquads/GZippedWholeFileNQuadsInputTest.java |  38 -
 ...mpressedBlockedNTriplesInputFormatTests.java |  53 --
 ...tractCompressedNTriplesInputFormatTests.java |  68 --
 ...ressedWholeFileNTriplesInputFormatTests.java |  75 --
 .../ntriples/BZippedBlockedNTriplesInput.java   |  37 -
 .../ntriples/BZippedNTriplesInputTest.java      |  38 -
 .../BZippedWholeFileNTriplesInputTest.java      |  38 -
 .../ntriples/DeflatedBlockedNTriplesInput.java  |  37 -
 .../ntriples/DeflatedNTriplesInputTest.java     |  38 -
 .../DeflatedWholeFileNTriplesInputTest.java     |  38 -
 .../ntriples/GZippedBlockedNTriplesInput.java   |  37 -
 .../ntriples/GZippedNTriplesInputTest.java      |  41 -
 .../GZippedWholeFileNTriplesInputTest.java      |  38 -
 ...stractCompressedRdfJsonInputFormatTests.java |  74 --
 .../rdfjson/BZippedRdfJsonInputTest.java        |  37 -
 .../rdfjson/DeflatedRdfJsonInputTest.java       |  37 -
 .../rdfjson/GZippedRdfJsonInputTest.java        |  37 -
 ...bstractCompressedRdfXmlInputFormatTests.java |  75 --
 .../rdfxml/BZippedRdfXmlInputTest.java          |  37 -
 .../rdfxml/DeflatedRdfXmlInputTest.java         |  37 -
 .../rdfxml/GZippedRdfXmlInputTest.java          |  37 -
 ...actCompressedThriftQuadInputFormatTests.java |  72 --
 ...tCompressedThriftTripleInputFormatTests.java |  72 --
 .../thrift/BZippedThriftQuadInputTest.java      |  34 -
 .../thrift/BZippedThriftTripleInputTest.java    |  34 -
 .../thrift/DeflatedThriftQuadInputTest.java     |  34 -
 .../thrift/DeflatedThriftTripleInputTest.java   |  34 -
 .../thrift/GZippedThriftQuadInputTest.java      |  34 -
 .../thrift/GZippedThriftTripleInputTest.java    |  34 -
 .../AbstractCompressedTriGInputFormatTests.java |  72 --
 .../compressed/trig/BZippedTriGInputTest.java   |  37 -
 .../compressed/trig/DeflatedTriGInputTest.java  |  37 -
 .../compressed/trig/GZippedTriGInputTest.java   |  37 -
 .../AbstractCompressedTriXInputFormatTests.java |  72 --
 .../compressed/trix/BZippedTriXInputTest.java   |  35 -
 .../compressed/trix/DeflatedTriXInputTest.java  |  35 -
 .../compressed/trix/GZippedTriXInputTest.java   |  35 -
 ...bstractCompressedTurtleInputFormatTests.java |  75 --
 .../turtle/BZippedTurtleInputTest.java          |  37 -
 .../turtle/DeflatedTurtleInputTest.java         |  37 -
 .../turtle/GZippedTurtleInputTest.java          |  37 -
 .../io/input/jsonld/JsonLDQuadInputTest.java    |  50 --
 .../io/input/jsonld/JsonLDTripleInputTest.java  |  50 --
 .../io/input/nquads/BlockedNQuadsInputTest.java |  51 --
 .../rdf/io/input/nquads/NQuadsInputTest.java    |  44 --
 .../input/nquads/WholeFileNQuadsInputTest.java  |  51 --
 .../ntriples/BlockedNTriplesInputTest.java      |  50 --
 .../io/input/ntriples/NTriplesInputTest.java    |  44 --
 .../ntriples/WholeFileNTriplesInputTest.java    |  52 --
 .../rdf/io/input/rdfjson/RdfJsonInputTest.java  |  51 --
 .../rdf/io/input/rdfxml/RdfXmlInputTest.java    |  51 --
 .../io/input/thrift/ThriftQuadInputTest.java    |  51 --
 .../io/input/thrift/ThriftTripleInputTest.java  |  51 --
 .../hadoop/rdf/io/input/trig/TriGInputTest.java |  50 --
 .../hadoop/rdf/io/input/trix/TriXInputTest.java |  50 --
 .../rdf/io/input/turtle/TurtleInputTest.java    |  50 --
 .../util/AbstractTrackableInputStreamTests.java | 701 -----------------
 .../rdf/io/input/util/BlockInputStreamTest.java | 240 ------
 .../io/input/util/TrackedInputStreamTest.java   |  39 -
 .../AbstractNodeTupleOutputFormatTests.java     | 255 -------
 .../output/AbstractQuadOutputFormatTests.java   |  51 --
 .../output/AbstractTripleOutputFormatTests.java |  47 --
 .../io/output/jsonld/JsonLdQuadOutputTest.java  |  47 --
 .../output/jsonld/JsonLdTripleOutputTest.java   |  47 --
 .../rdf/io/output/nquads/NQuadsOutputTest.java  |  51 --
 .../io/output/ntriples/NTriplesOutputTest.java  |  51 --
 .../io/output/rdfjson/RdfJsonOutputTest.java    |  51 --
 .../rdf/io/output/rdfxml/RdfXmlOutputTest.java  |  51 --
 .../io/output/thrift/ThriftQuadOutputTest.java  |  48 --
 .../output/thrift/ThriftTripleOutputTest.java   |  48 --
 .../io/output/trig/BatchedTriGOutputTest.java   |  92 ---
 .../io/output/trig/StreamedTriGOutputTest.java  |  92 ---
 .../output/trig/TriGBlankNodeOutputTests.java   | 120 ---
 .../rdf/io/output/trix/TriXOutputTest.java      |  47 --
 .../output/turtle/BatchedTurtleOutputTest.java  |  92 ---
 .../output/turtle/StreamedTurtleOutputTest.java |  92 ---
 .../turtle/TurtleBlankNodeOutputTests.java      | 118 ---
 .../io/registry/TestHadoopRdfIORegistry.java    | 186 -----
 jena-hadoop-rdf/hadoop-rdf-stats/hadoop-job.xml |  46 --
 jena-hadoop-rdf/hadoop-rdf-stats/pom.xml        | 103 ---
 .../apache/jena/hadoop/rdf/stats/RdfStats.java  | 405 ----------
 .../jena/hadoop/rdf/stats/jobs/JobFactory.java  | 757 -------------------
 jena-hadoop-rdf/jena-elephas-common/pom.xml     |  54 ++
 .../rdf/types/AbstractNodeTupleWritable.java    | 193 +++++
 .../rdf/types/CharacteristicSetWritable.java    | 298 ++++++++
 .../rdf/types/CharacteristicWritable.java       | 160 ++++
 .../hadoop/rdf/types/NodeTupleWritable.java     |  80 ++
 .../jena/hadoop/rdf/types/NodeWritable.java     | 188 +++++
 .../jena/hadoop/rdf/types/QuadWritable.java     | 136 ++++
 .../jena/hadoop/rdf/types/TripleWritable.java   | 138 ++++
 .../comparators/SimpleBinaryComparator.java     |  34 +
 .../rdf/types/converters/ThriftConverter.java   | 147 ++++
 .../rdf/io/types/CharacteristicTests.java       | 210 +++++
 .../jena/hadoop/rdf/io/types/RdfTypesTest.java  | 406 ++++++++++
 jena-hadoop-rdf/jena-elephas-io/pom.xml         |  67 ++
 .../jena/hadoop/rdf/io/HadoopIOConstants.java   |  49 ++
 .../jena/hadoop/rdf/io/RdfIOConstants.java      |  81 ++
 .../io/input/AbstractNLineFileInputFormat.java  |  70 ++
 .../io/input/AbstractWholeFileInputFormat.java  |  42 +
 .../hadoop/rdf/io/input/QuadsInputFormat.java   |  46 ++
 .../hadoop/rdf/io/input/TriplesInputFormat.java |  42 +
 .../rdf/io/input/TriplesOrQuadsInputFormat.java |  47 ++
 .../io/input/jsonld/JsonLDQuadInputFormat.java  |  39 +
 .../input/jsonld/JsonLDTripleInputFormat.java   |  39 +
 .../input/nquads/BlockedNQuadsInputFormat.java  |  53 ++
 .../rdf/io/input/nquads/NQuadsInputFormat.java  |  46 ++
 .../nquads/WholeFileNQuadsInputFormat.java      |  51 ++
 .../ntriples/BlockedNTriplesInputFormat.java    |  53 ++
 .../io/input/ntriples/NTriplesInputFormat.java  |  46 ++
 .../ntriples/WholeFileNTriplesInputFormat.java  |  51 ++
 .../io/input/rdfjson/RdfJsonInputFormat.java    |  46 ++
 .../rdf/io/input/rdfxml/RdfXmlInputFormat.java  |  46 ++
 .../AbstractBlockBasedNodeTupleReader.java      | 344 +++++++++
 .../readers/AbstractBlockBasedQuadReader.java   |  51 ++
 .../readers/AbstractBlockBasedTripleReader.java |  51 ++
 .../AbstractLineBasedNodeTupleReader.java       | 265 +++++++
 .../readers/AbstractLineBasedQuadReader.java    |  50 ++
 .../readers/AbstractLineBasedTripleReader.java  |  51 ++
 .../rdf/io/input/readers/AbstractRdfReader.java | 108 +++
 .../AbstractWholeFileNodeTupleReader.java       | 328 ++++++++
 .../readers/AbstractWholeFileQuadReader.java    |  51 ++
 .../readers/AbstractWholeFileTripleReader.java  |  51 ++
 .../rdf/io/input/readers/QuadsReader.java       |  49 ++
 .../io/input/readers/TriplesOrQuadsReader.java  |  72 ++
 .../rdf/io/input/readers/TriplesReader.java     |  49 ++
 .../io/input/readers/TriplesToQuadsReader.java  | 102 +++
 .../input/readers/jsonld/JsonLDQuadReader.java  |  32 +
 .../readers/jsonld/JsonLDTripleReader.java      |  30 +
 .../readers/nquads/BlockedNQuadsReader.java     |  45 ++
 .../io/input/readers/nquads/NQuadsReader.java   |  49 ++
 .../readers/nquads/WholeFileNQuadsReader.java   |  42 +
 .../readers/ntriples/BlockedNTriplesReader.java |  45 ++
 .../input/readers/ntriples/NTriplesReader.java  |  48 ++
 .../ntriples/WholeFileNTriplesReader.java       |  42 +
 .../io/input/readers/rdfjson/RdfJsonReader.java |  37 +
 .../io/input/readers/rdfxml/RdfXmlReader.java   |  37 +
 .../input/readers/thrift/ThriftQuadReader.java  |  32 +
 .../readers/thrift/ThriftTripleReader.java      |  30 +
 .../rdf/io/input/readers/trig/TriGReader.java   |  37 +
 .../rdf/io/input/readers/trix/TriXReader.java   |  37 +
 .../io/input/readers/turtle/TurtleReader.java   |  37 +
 .../io/input/thrift/ThriftQuadInputFormat.java  |  39 +
 .../input/thrift/ThriftTripleInputFormat.java   |  39 +
 .../rdf/io/input/trig/TriGInputFormat.java      |  46 ++
 .../rdf/io/input/trix/TriXInputFormat.java      |  42 +
 .../rdf/io/input/turtle/TurtleInputFormat.java  |  46 ++
 .../rdf/io/input/util/BlockInputStream.java     |  94 +++
 .../hadoop/rdf/io/input/util/RdfIOUtils.java    | 101 +++
 .../rdf/io/input/util/TrackableInputStream.java |  38 +
 .../rdf/io/input/util/TrackedInputStream.java   | 124 +++
 .../io/input/util/TrackedPipedQuadsStream.java  |  55 ++
 .../io/input/util/TrackedPipedRDFStream.java    |  64 ++
 .../input/util/TrackedPipedTriplesStream.java   |  56 ++
 .../AbstractBatchedNodeTupleOutputFormat.java   |  55 ++
 .../rdf/io/output/AbstractNodeOutputFormat.java |  94 +++
 .../output/AbstractNodeTupleOutputFormat.java   | 109 +++
 .../AbstractStreamRdfNodeTupleOutputFormat.java |  73 ++
 .../hadoop/rdf/io/output/QuadsOutputFormat.java |  64 ++
 .../io/output/TriplesOrQuadsOutputFormat.java   |  74 ++
 .../rdf/io/output/TriplesOutputFormat.java      |  61 ++
 .../output/jsonld/JsonLDQuadOutputFormat.java   |  44 ++
 .../output/jsonld/JsonLDTripleOutputFormat.java |  44 ++
 .../io/output/nquads/NQuadsOutputFormat.java    |  52 ++
 .../ntriples/NTriplesNodeOutputFormat.java      |  45 ++
 .../output/ntriples/NTriplesOutputFormat.java   |  52 ++
 .../io/output/rdfjson/RdfJsonOutputFormat.java  |  52 ++
 .../io/output/rdfxml/RdfXmlOutputFormat.java    |  52 ++
 .../output/thrift/ThriftQuadOutputFormat.java   |  51 ++
 .../output/thrift/ThriftTripleOutputFormat.java |  52 ++
 .../io/output/trig/BatchedTriGOutputFormat.java |  54 ++
 .../rdf/io/output/trig/TriGOutputFormat.java    |  58 ++
 .../rdf/io/output/trix/TriXOutputFormat.java    |  57 ++
 .../turtle/BatchedTurtleOutputFormat.java       |  50 ++
 .../io/output/turtle/TurtleOutputFormat.java    |  56 ++
 .../writers/AbstractBatchedNodeTupleWriter.java | 113 +++
 .../writers/AbstractBatchedQuadWriter.java      |  80 ++
 .../writers/AbstractBatchedTripleWriter.java    |  68 ++
 .../AbstractLineBasedNodeTupleWriter.java       | 152 ++++
 .../writers/AbstractLineBasedQuadWriter.java    |  71 ++
 .../writers/AbstractLineBasedTripleWriter.java  |  68 ++
 .../io/output/writers/AbstractNodeWriter.java   | 192 +++++
 .../AbstractStreamRdfNodeTupleWriter.java       |  71 ++
 .../AbstractWholeFileNodeTupleWriter.java       |  96 +++
 .../writers/AbstractWholeFileQuadWriter.java    |  66 ++
 .../writers/AbstractWholeFileTripleWriter.java  |  65 ++
 .../io/output/writers/QuadsToTriplesWriter.java |  59 ++
 .../io/output/writers/StreamRdfQuadWriter.java  |  45 ++
 .../output/writers/StreamRdfTripleWriter.java   |  44 ++
 .../output/writers/jsonld/JsonLDQuadWriter.java |  38 +
 .../writers/jsonld/JsonLDTripleWriter.java      |  38 +
 .../io/output/writers/nquads/NQuadsWriter.java  |  57 ++
 .../writers/ntriples/NTriplesNodeWriter.java    |  59 ++
 .../output/writers/ntriples/NTriplesWriter.java |  58 ++
 .../output/writers/rdfjson/RdfJsonWriter.java   |  51 ++
 .../io/output/writers/rdfxml/RdfXmlWriter.java  |  51 ++
 .../output/writers/thrift/ThriftQuadWriter.java |  38 +
 .../writers/thrift/ThriftTripleWriter.java      |  38 +
 .../output/writers/trig/BatchedTriGWriter.java  |  52 ++
 .../writers/turtle/BatchedTurtleWriter.java     |  54 ++
 .../rdf/io/registry/HadoopRdfIORegistry.java    | 310 ++++++++
 .../hadoop/rdf/io/registry/ReaderFactory.java   |  83 ++
 .../hadoop/rdf/io/registry/WriterFactory.java   |  96 +++
 .../readers/AbstractQuadsOnlyReaderFactory.java |  83 ++
 .../registry/readers/AbstractReaderFactory.java |  80 ++
 .../AbstractTriplesOnlyReaderFactory.java       |  83 ++
 .../registry/readers/JsonLDReaderFactory.java   |  49 ++
 .../registry/readers/NQuadsReaderFactory.java   |  42 +
 .../registry/readers/NTriplesReaderFactory.java |  38 +
 .../registry/readers/RdfJsonReaderFactory.java  |  41 +
 .../registry/readers/RdfXmlReaderFactory.java   |  40 +
 .../registry/readers/ThriftReaderFactory.java   |  49 ++
 .../io/registry/readers/TriGReaderFactory.java  |  42 +
 .../io/registry/readers/TriXReaderFactory.java  |  41 +
 .../registry/readers/TurtleReaderFactory.java   |  40 +
 .../writers/AbstractQuadsOnlyWriterFactory.java |  86 +++
 .../AbstractTriplesOnlyWriterFactory.java       |  85 +++
 .../registry/writers/AbstractWriterFactory.java |  82 ++
 .../registry/writers/JsonLDWriterFactory.java   |  52 ++
 .../registry/writers/NQuadsWriterFactory.java   |  44 ++
 .../registry/writers/NTriplesWriterFactory.java |  44 ++
 .../registry/writers/RdfJsonWriterFactory.java  |  43 ++
 .../registry/writers/RdfXmlWriterFactory.java   |  44 ++
 .../registry/writers/ThriftWriterFactory.java   |  57 ++
 .../io/registry/writers/TriGWriterFactory.java  |  45 ++
 .../io/registry/writers/TriXWriterFactory.java  |  47 ++
 .../registry/writers/TurtleWriterFactory.java   |  45 ++
 ...he.jena.hadoop.rdf.io.registry.ReaderFactory |  10 +
 ...he.jena.hadoop.rdf.io.registry.WriterFactory |  10 +
 .../rdf/io/RdfTriplesInputTestMapper.java       |  47 ++
 .../AbstractBlockedQuadInputFormatTests.java    |  33 +
 .../AbstractBlockedTripleInputFormatTests.java  |  33 +
 .../AbstractNodeTupleInputFormatTests.java      | 612 +++++++++++++++
 .../io/input/AbstractQuadsInputFormatTests.java |  70 ++
 .../input/AbstractTriplesInputFormatTests.java  |  72 ++
 .../AbstractWholeFileQuadInputFormatTests.java  | 115 +++
 ...AbstractWholeFileTripleInputFormatTests.java | 108 +++
 .../io/input/bnodes/AbstractBlankNodeTests.java | 636 ++++++++++++++++
 .../bnodes/AbstractTripleBlankNodeTests.java    |  65 ++
 .../input/bnodes/JsonLdTripleBlankNodeTest.java |  63 ++
 .../io/input/bnodes/NTriplesBlankNodeTest.java  |  58 ++
 .../io/input/bnodes/RdfJsonBlankNodeTest.java   |  58 ++
 .../io/input/bnodes/RdfThriftBlankNodeTest.java |  68 ++
 .../io/input/bnodes/RdfXmlBlankNodeTest.java    |  62 ++
 .../io/input/bnodes/TurtleBlankNodeTest.java    |  58 ++
 ...ractCompressedNodeTupleInputFormatTests.java |  74 ++
 ...AbstractCompressedQuadsInputFormatTests.java |  71 ++
 ...stractCompressedTriplesInputFormatTests.java |  71 ++
 ...CompressedWholeFileQuadInputFormatTests.java | 150 ++++
 ...mpressedWholeFileTripleInputFormatTests.java | 144 ++++
 ...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 +
 ...bstractCompressedNQuadsInputFormatTests.java |  68 ++
 ...mpressedWholeFileNQuadsInputFormatTests.java |  75 ++
 .../nquads/BZipppedNQuadsInputTest.java         |  38 +
 .../BZipppedWholeFileNQuadsInputTest.java       |  37 +
 .../nquads/DeflatedNQuadsInputTest.java         |  37 +
 .../DeflatedWholeFileNQuadsInputTest.java       |  37 +
 .../nquads/GZippedNQuadsInputTest.java          |  38 +
 .../nquads/GZippedWholeFileNQuadsInputTest.java |  38 +
 ...mpressedBlockedNTriplesInputFormatTests.java |  53 ++
 ...tractCompressedNTriplesInputFormatTests.java |  68 ++
 ...ressedWholeFileNTriplesInputFormatTests.java |  75 ++
 .../ntriples/BZippedBlockedNTriplesInput.java   |  37 +
 .../ntriples/BZippedNTriplesInputTest.java      |  38 +
 .../BZippedWholeFileNTriplesInputTest.java      |  38 +
 .../ntriples/DeflatedBlockedNTriplesInput.java  |  37 +
 .../ntriples/DeflatedNTriplesInputTest.java     |  38 +
 .../DeflatedWholeFileNTriplesInputTest.java     |  38 +
 .../ntriples/GZippedBlockedNTriplesInput.java   |  37 +
 .../ntriples/GZippedNTriplesInputTest.java      |  41 +
 .../GZippedWholeFileNTriplesInputTest.java      |  38 +
 ...stractCompressedRdfJsonInputFormatTests.java |  74 ++
 .../rdfjson/BZippedRdfJsonInputTest.java        |  37 +
 .../rdfjson/DeflatedRdfJsonInputTest.java       |  37 +
 .../rdfjson/GZippedRdfJsonInputTest.java        |  37 +
 ...bstractCompressedRdfXmlInputFormatTests.java |  75 ++
 .../rdfxml/BZippedRdfXmlInputTest.java          |  37 +
 .../rdfxml/DeflatedRdfXmlInputTest.java         |  37 +
 .../rdfxml/GZippedRdfXmlInputTest.java          |  37 +
 ...actCompressedThriftQuadInputFormatTests.java |  72 ++
 ...tCompressedThriftTripleInputFormatTests.java |  72 ++
 .../thrift/BZippedThriftQuadInputTest.java      |  34 +
 .../thrift/BZippedThriftTripleInputTest.java    |  34 +
 .../thrift/DeflatedThriftQuadInputTest.java     |  34 +
 .../thrift/DeflatedThriftTripleInputTest.java   |  34 +
 .../thrift/GZippedThriftQuadInputTest.java      |  34 +
 .../thrift/GZippedThriftTripleInputTest.java    |  34 +
 .../AbstractCompressedTriGInputFormatTests.java |  72 ++
 .../compressed/trig/BZippedTriGInputTest.java   |  37 +
 .../compressed/trig/DeflatedTriGInputTest.java  |  37 +
 .../compressed/trig/GZippedTriGInputTest.java   |  37 +
 .../AbstractCompressedTriXInputFormatTests.java |  72 ++
 .../compressed/trix/BZippedTriXInputTest.java   |  35 +
 .../compressed/trix/DeflatedTriXInputTest.java  |  35 +
 .../compressed/trix/GZippedTriXInputTest.java   |  35 +
 ...bstractCompressedTurtleInputFormatTests.java |  75 ++
 .../turtle/BZippedTurtleInputTest.java          |  37 +
 .../turtle/DeflatedTurtleInputTest.java         |  37 +
 .../turtle/GZippedTurtleInputTest.java          |  37 +
 .../io/input/jsonld/JsonLDQuadInputTest.java    |  50 ++
 .../io/input/jsonld/JsonLDTripleInputTest.java  |  50 ++
 .../io/input/nquads/BlockedNQuadsInputTest.java |  51 ++
 .../rdf/io/input/nquads/NQuadsInputTest.java    |  44 ++
 .../input/nquads/WholeFileNQuadsInputTest.java  |  51 ++
 .../ntriples/BlockedNTriplesInputTest.java      |  50 ++
 .../io/input/ntriples/NTriplesInputTest.java    |  44 ++
 .../ntriples/WholeFileNTriplesInputTest.java    |  52 ++
 .../rdf/io/input/rdfjson/RdfJsonInputTest.java  |  51 ++
 .../rdf/io/input/rdfxml/RdfXmlInputTest.java    |  51 ++
 .../io/input/thrift/ThriftQuadInputTest.java    |  51 ++
 .../io/input/thrift/ThriftTripleInputTest.java  |  51 ++
 .../hadoop/rdf/io/input/trig/TriGInputTest.java |  50 ++
 .../hadoop/rdf/io/input/trix/TriXInputTest.java |  50 ++
 .../rdf/io/input/turtle/TurtleInputTest.java    |  50 ++
 .../util/AbstractTrackableInputStreamTests.java | 701 +++++++++++++++++
 .../rdf/io/input/util/BlockInputStreamTest.java | 240 ++++++
 .../io/input/util/TrackedInputStreamTest.java   |  39 +
 .../AbstractNodeTupleOutputFormatTests.java     | 255 +++++++
 .../output/AbstractQuadOutputFormatTests.java   |  51 ++
 .../output/AbstractTripleOutputFormatTests.java |  47 ++
 .../io/output/jsonld/JsonLdQuadOutputTest.java  |  47 ++
 .../output/jsonld/JsonLdTripleOutputTest.java   |  47 ++
 .../rdf/io/output/nquads/NQuadsOutputTest.java  |  51 ++
 .../io/output/ntriples/NTriplesOutputTest.java  |  51 ++
 .../io/output/rdfjson/RdfJsonOutputTest.java    |  51 ++
 .../rdf/io/output/rdfxml/RdfXmlOutputTest.java  |  51 ++
 .../io/output/thrift/ThriftQuadOutputTest.java  |  48 ++
 .../output/thrift/ThriftTripleOutputTest.java   |  48 ++
 .../io/output/trig/BatchedTriGOutputTest.java   |  92 +++
 .../io/output/trig/StreamedTriGOutputTest.java  |  92 +++
 .../output/trig/TriGBlankNodeOutputTests.java   | 120 +++
 .../rdf/io/output/trix/TriXOutputTest.java      |  47 ++
 .../output/turtle/BatchedTurtleOutputTest.java  |  92 +++
 .../output/turtle/StreamedTurtleOutputTest.java |  92 +++
 .../turtle/TurtleBlankNodeOutputTests.java      | 118 +++
 .../io/registry/TestHadoopRdfIORegistry.java    | 186 +++++
 .../jena-elephas-stats/hadoop-job.xml           |  46 ++
 jena-hadoop-rdf/jena-elephas-stats/pom.xml      | 103 +++
 .../apache/jena/hadoop/rdf/stats/RdfStats.java  | 405 ++++++++++
 .../jena/hadoop/rdf/stats/jobs/JobFactory.java  | 757 +++++++++++++++++++
 jena-hadoop-rdf/pom.xml                         |  26 +-
 529 files changed, 20511 insertions(+), 20552 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/pom.xml b/jena-hadoop-rdf/hadoop-rdf-common/pom.xml
deleted file mode 100644
index b5081a6..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/pom.xml
+++ /dev/null
@@ -1,54 +0,0 @@
-<!--
-   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.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<groupId>org.apache.jena</groupId>
-		<artifactId>jena-hadoop-rdf</artifactId>
-		<version>0.9.0-SNAPSHOT</version>
-	</parent>
-	<artifactId>jena-hadoop-rdf-common</artifactId>
-	<name>Apache Jena - RDF Tools for Hadoop - Common API</name>
-	<description>Common code for RDF on Hadoop such as writable types for RDF primitives</description>
-
-	<!-- Note that versions are managed by parent POMs -->
-	<dependencies>
-		<!-- Hadoop Dependencies -->
-		<!-- Note these will be provided on the Hadoop cluster hence the provided 
-			scope -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Jena dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-arq</artifactId>
-		</dependency>
-
-		<!-- Test Dependencies -->
-		<dependency>
-			<groupId>junit</groupId>
-			<artifactId>junit</artifactId>
-			<scope>test</scope>
-		</dependency>
-	</dependencies>
-</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java b/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
deleted file mode 100644
index f0acc09..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableUtils;
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.sparql.util.NodeUtils;
-
-/**
- * A abstract general purpose writable where the actual class represented is
- * composed of a number of {@link Node} instances
- * <p>
- * The binary encoding of this base implementation is just a variable integer
- * indicating the number of nodes present followed by the binary encodings of
- * the {@link NodeWritable} instances. Derived implementations may wish to
- * override the {@link #readFields(DataInput)} and {@link #write(DataOutput)}
- * methods in order to use more specialised encodings.
- * </p>
- * 
- * @param <T>
- *            Tuple type
- */
-public abstract class AbstractNodeTupleWritable<T> implements WritableComparable<AbstractNodeTupleWritable<T>> {
-
-    private T tuple;
-
-    /**
-     * Creates a new empty instance
-     */
-    protected AbstractNodeTupleWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance with the given value
-     * 
-     * @param tuple
-     *            Tuple value
-     */
-    protected AbstractNodeTupleWritable(T tuple) {
-        this.tuple = tuple;
-    }
-
-    /**
-     * Gets the tuple
-     * 
-     * @return Tuple
-     */
-    public T get() {
-        return this.tuple;
-    }
-
-    /**
-     * Sets the tuple
-     * 
-     * @param tuple
-     *            Tuple
-     */
-    public void set(T tuple) {
-        this.tuple = tuple;
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        // Determine how many nodes
-        int size = WritableUtils.readVInt(input);
-        Node[] ns = new Node[size];
-
-        NodeWritable nw = new NodeWritable();
-        for (int i = 0; i < ns.length; i++) {
-            nw.readFields(input);
-            ns[i] = nw.get();
-        }
-
-        // Load the tuple
-        this.tuple = this.createTuple(ns);
-    }
-
-    /**
-     * Creates the actual tuple type from an array of nodes
-     * 
-     * @param ns
-     *            Nodes
-     * @return Tuple
-     */
-    protected abstract T createTuple(Node[] ns);
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        // Determine how many nodes
-        Node[] ns = this.createNodes(this.tuple);
-        WritableUtils.writeVInt(output, ns.length);
-
-        // Write out nodes
-        NodeWritable nw = new NodeWritable();
-        for (int i = 0; i < ns.length; i++) {
-            nw.set(ns[i]);
-            nw.write(output);
-        }
-    }
-
-    /**
-     * Sets the tuple value
-     * <p>
-     * Intended only for internal use i.e. when a derived implementation
-     * overrides {@link #readFields(DataInput)} and needs to set the tuple value
-     * directly i.e. when a derived implementation is using a custom encoding
-     * scheme
-     * </p>
-     * 
-     * @param tuple
-     *            Tuple
-     */
-    protected final void setInternal(T tuple) {
-        this.tuple = tuple;
-    }
-
-    /**
-     * Converts the actual tuple type into an array of nodes
-     * 
-     * @param tuple
-     *            Tuples
-     * @return Nodes
-     */
-    protected abstract Node[] createNodes(T tuple);
-
-    /**
-     * Compares instances node by node
-     * <p>
-     * Derived implementations may wish to override this and substitute native
-     * tuple based comparisons
-     * </p>
-     * 
-     * @param other
-     *            Instance to compare with
-     */
-    @Override
-    public int compareTo(AbstractNodeTupleWritable<T> other) {
-        Node[] ns = this.createNodes(this.tuple);
-        Node[] otherNs = this.createNodes(other.tuple);
-
-        if (ns.length < otherNs.length) {
-            return -1;
-        } else if (ns.length > otherNs.length) {
-            return 1;
-        }
-        // Compare node by node
-        for (int i = 0; i < ns.length; i++) {
-            int c = NodeUtils.compareRDFTerms(ns[i], otherNs[i]);
-            if (c != 0)
-                return c;
-        }
-        return 0;
-    }
-
-    @Override
-    public String toString() {
-        return this.get().toString();
-    }
-
-    @Override
-    public int hashCode() {
-        return this.get().hashCode();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public boolean equals(Object other) {
-        if (!(other instanceof AbstractNodeTupleWritable))
-            return false;
-        return this.compareTo((AbstractNodeTupleWritable<T>) other) == 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java b/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
deleted file mode 100644
index f29b156..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
+++ /dev/null
@@ -1,298 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableUtils;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * Represents a characteristic set which is comprised of a count of nodes for
- * which the characteristic is applicable and a set of characteristics which
- * represents the number of usages of predicates with those nodes
- * 
- * 
- * 
- */
-public class CharacteristicSetWritable implements WritableComparable<CharacteristicSetWritable> {
-
-    private Map<NodeWritable, CharacteristicWritable> characteristics = new TreeMap<NodeWritable, CharacteristicWritable>();
-    private LongWritable count = new LongWritable();
-
-    /**
-     * Creates a new empty characteristic set with the default count of 1
-     */
-    public CharacteristicSetWritable() {
-        this(1);
-    }
-
-    /**
-     * Creates a new characteristic set with the default count of 1 and the
-     * given characteristics
-     * 
-     * @param characteristics
-     *            Characteristics
-     */
-    public CharacteristicSetWritable(CharacteristicWritable... characteristics) {
-        this(1, characteristics);
-    }
-
-    /**
-     * Creates an empty characteristic set with the given count
-     * 
-     * @param count
-     *            Count
-     */
-    public CharacteristicSetWritable(long count) {
-        this(count, new CharacteristicWritable[0]);
-    }
-
-    /**
-     * Creates a new characteristic set
-     * 
-     * @param count
-     *            Count
-     * @param characteristics
-     *            Characteristics
-     */
-    public CharacteristicSetWritable(long count, CharacteristicWritable... characteristics) {
-        this.count.set(count);
-        for (CharacteristicWritable characteristic : characteristics) {
-            this.characteristics.put(characteristic.getNode(), characteristic);
-        }
-    }
-
-    /**
-     * Creates a new instance and reads its data from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static CharacteristicSetWritable read(DataInput input) throws IOException {
-        CharacteristicSetWritable set = new CharacteristicSetWritable();
-        set.readFields(input);
-        return set;
-    }
-
-    /**
-     * Gets the count
-     * 
-     * @return Count
-     */
-    public LongWritable getCount() {
-        return this.count;
-    }
-
-    /**
-     * Gets the characteristics
-     * 
-     * @return Characteristics
-     */
-    public Iterator<CharacteristicWritable> getCharacteristics() {
-        return this.characteristics.values().iterator();
-    }
-
-    /**
-     * Gets the size of the characteristic set
-     * 
-     * @return Size
-     */
-    public int size() {
-        return this.characteristics.size();
-    }
-
-    /**
-     * Adds a characteristic to the set merging it into the appropriate existing
-     * characteristic if applicable
-     * 
-     * @param characteristic
-     *            Characteristics
-     */
-    public void add(CharacteristicWritable characteristic) {
-        if (this.characteristics.containsKey(characteristic.getNode())) {
-            this.characteristics.get(characteristic.getNode()).increment(characteristic.getCount().get());
-        } else {
-            this.characteristics.put(characteristic.getNode(), characteristic);
-        }
-    }
-
-    /**
-     * Adds some characteristics to the set merging them with the appropriate
-     * existing characteristics if applicable
-     * 
-     * @param characteristics
-     */
-    public void add(CharacteristicWritable... characteristics) {
-        for (CharacteristicWritable characteristic : characteristics) {
-            this.add(characteristic);
-        }
-    }
-
-    /**
-     * Adds the contents of the other characteristic set to this characteristic
-     * set
-     * 
-     * @param set
-     *            Characteristic set
-     */
-    public void add(CharacteristicSetWritable set) {
-        this.increment(set.getCount().get());
-        Iterator<CharacteristicWritable> iter = set.getCharacteristics();
-        while (iter.hasNext()) {
-            this.add(iter.next());
-        }
-    }
-
-    /**
-     * Gets whether the set contains a characteristic for the given predicate
-     * 
-     * @param uri
-     *            Predicate URI
-     * @return True if contained in the set, false otherwise
-     */
-    public boolean hasCharacteristic(String uri) {
-        return this.hasCharacteristic(NodeFactory.createURI(uri));
-    }
-
-    /**
-     * Gets whether the set contains a characteristic for the given predicate
-     * 
-     * @param n
-     *            Predicate
-     * @return True if contained in the set, false otherwise
-     */
-    public boolean hasCharacteristic(Node n) {
-        return this.hasCharacteristic(new NodeWritable(n));
-    }
-
-    /**
-     * Gets whether the set contains a characteristic for the given predicate
-     * 
-     * @param n
-     *            Predicate
-     * @return True if contained in the set, false otherwise
-     */
-    public boolean hasCharacteristic(NodeWritable n) {
-        return this.characteristics.containsKey(n);
-    }
-
-    /**
-     * Increments the count by the given increment
-     * 
-     * @param l
-     *            Increment
-     */
-    public void increment(long l) {
-        this.count.set(this.count.get() + l);
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        // Write size, then count, then characteristics
-        WritableUtils.writeVInt(output, this.characteristics.size());
-        this.count.write(output);
-        for (CharacteristicWritable characteristic : this.characteristics.values()) {
-            characteristic.write(output);
-        }
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        // Read size, then count, then characteristics
-        int size = WritableUtils.readVInt(input);
-        this.count.readFields(input);
-        this.characteristics.clear();
-        for (int i = 0; i < size; i++) {
-            CharacteristicWritable cw = CharacteristicWritable.read(input);
-            this.characteristics.put(cw.getNode(), cw);
-        }
-    }
-
-    @Override
-    public int compareTo(CharacteristicSetWritable cs) {
-        int size = this.characteristics.size();
-        int otherSize = cs.characteristics.size();
-        if (size < otherSize) {
-            return -1;
-        } else if (size > otherSize) {
-            return 1;
-        } else {
-            // Compare characteristics in turn
-            Iterator<CharacteristicWritable> iter = this.getCharacteristics();
-            Iterator<CharacteristicWritable> otherIter = cs.getCharacteristics();
-
-            int compare = 0;
-            while (iter.hasNext()) {
-                CharacteristicWritable c = iter.next();
-                CharacteristicWritable otherC = otherIter.next();
-                compare = c.compareTo(otherC);
-                if (compare != 0)
-                    return compare;
-            }
-            return compare;
-        }
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        if (!(other instanceof CharacteristicSetWritable))
-            return false;
-        return this.compareTo((CharacteristicSetWritable) other) == 0;
-    }
-
-    @Override
-    public int hashCode() {
-        // Build a hash code from characteristics
-        if (this.characteristics.size() == 0)
-            return 0;
-        Iterator<CharacteristicWritable> iter = this.getCharacteristics();
-        int hash = 17;
-        while (iter.hasNext()) {
-            hash = hash * 31 + iter.next().hashCode();
-        }
-        return hash;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder builder = new StringBuilder();
-        builder.append("{ ");
-        builder.append(this.count.get());
-        Iterator<CharacteristicWritable> iter = this.getCharacteristics();
-        while (iter.hasNext()) {
-            builder.append(" , ");
-            builder.append(iter.next().toString());
-        }
-        builder.append(" }");
-        return builder.toString();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java b/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
deleted file mode 100644
index 90fc7db..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.WritableComparable;
-
-import com.hp.hpl.jena.graph.Node;
-
-/**
- * Represents a characteristic for a single node and contains the node and a
- * count associated with that node
- * <p>
- * Note that characteristics are compared based upon only the nodes and not
- * their counts
- * </p>
- * 
- * 
- * 
- */
-public class CharacteristicWritable implements WritableComparable<CharacteristicWritable> {
-
-    private NodeWritable node = new NodeWritable();
-    private LongWritable count = new LongWritable();
-
-    /**
-     * Creates an empty characteristic writable
-     */
-    public CharacteristicWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a characteristic writable with the given node and the default
-     * count of 1
-     * 
-     * @param n
-     *            Node
-     */
-    public CharacteristicWritable(Node n) {
-        this(n, 1);
-    }
-
-    /**
-     * Creates a characteristic writable with the given node and count
-     * 
-     * @param n
-     *            Node
-     * @param count
-     *            Count
-     */
-    public CharacteristicWritable(Node n, long count) {
-        this.node.set(n);
-        this.count.set(count);
-    }
-
-    /**
-     * Creates a new instance and reads in its data from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static CharacteristicWritable read(DataInput input) throws IOException {
-        CharacteristicWritable cw = new CharacteristicWritable();
-        cw.readFields(input);
-        return cw;
-    }
-
-    /**
-     * Gets the node
-     * 
-     * @return Node
-     */
-    public NodeWritable getNode() {
-        return this.node;
-    }
-
-    /**
-     * Gets the count
-     * 
-     * @return Count
-     */
-    public LongWritable getCount() {
-        return this.count;
-    }
-
-    /**
-     * Increments the count by 1
-     */
-    public void increment() {
-        this.increment(1);
-    }
-
-    /**
-     * Increments the count by the given value
-     * 
-     * @param l
-     *            Value to increment by
-     */
-    public void increment(long l) {
-        this.count.set(this.count.get() + l);
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        this.node.write(output);
-        this.count.write(output);
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        this.node.readFields(input);
-        this.count.readFields(input);
-    }
-
-    @Override
-    public int compareTo(CharacteristicWritable o) {
-        return this.node.compareTo(o.node);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        if (!(other instanceof CharacteristicWritable))
-            return false;
-        return this.compareTo((CharacteristicWritable) other) == 0;
-    }
-
-    @Override
-    public int hashCode() {
-        return this.node.hashCode();
-    }
-
-    @Override
-    public String toString() {
-        return "(" + this.node.toString() + ", " + this.count.toString() + ")";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java b/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
deleted file mode 100644
index e06aac4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.IOException;
-
-import org.apache.jena.atlas.lib.Tuple;
-import com.hp.hpl.jena.graph.Node;
-
-/**
- * A writable RDF tuple
- * <p>
- * Unlike the more specific {@link TripleWritable} and {@link QuadWritable} this
- * class allows for arbitrary length tuples and does not restrict tuples to
- * being of uniform size.
- * </p>
- * 
- * 
- * 
- */
-public class NodeTupleWritable extends AbstractNodeTupleWritable<Tuple<Node>> {
-
-    /**
-     * Creates a new empty instance
-     */
-    public NodeTupleWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance with the given value
-     * 
-     * @param tuple
-     *            Tuple
-     */
-    public NodeTupleWritable(Tuple<Node> tuple) {
-        super(tuple);
-    }
-
-    /**
-     * Creates a new instance from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static NodeTupleWritable read(DataInput input) throws IOException {
-        NodeTupleWritable t = new NodeTupleWritable();
-        t.readFields(input);
-        return t;
-    }
-
-    @Override
-    protected Tuple<Node> createTuple(Node[] ns) {
-        return Tuple.create(ns);
-    }
-
-    @Override
-    protected Node[] createNodes(Tuple<Node> tuple) {
-        return tuple.tuple();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java b/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
deleted file mode 100644
index cf00f8d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
-import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
-import org.apache.jena.riot.thrift.TRDF;
-import org.apache.jena.riot.thrift.ThriftConvert;
-import org.apache.jena.riot.thrift.wire.RDF_Term;
-import org.apache.thrift.TException;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.sparql.util.NodeUtils;
-
-/**
- * A writable for {@link Node} instances
- * <p>
- * This uses <a
- * href="http://afs.github.io/rdf-thrift/rdf-binary-thrift.html">RDF Thrift</a>
- * for the binary encoding of terms. The in-memory storage for this type is both
- * a {@link Node} and a {@link RDF_Term} with lazy conversion between the two
- * forms as necessary.
- * </p>
- */
-public class NodeWritable implements WritableComparable<NodeWritable> {
-
-    static {
-        WritableComparator.define(NodeWritable.class, new SimpleBinaryComparator());
-    }
-
-    private Node node;
-    private RDF_Term term = new RDF_Term();
-
-    /**
-     * Creates an empty writable
-     */
-    public NodeWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static NodeWritable read(DataInput input) throws IOException {
-        NodeWritable nw = new NodeWritable();
-        nw.readFields(input);
-        return nw;
-    }
-
-    /**
-     * Creates a new writable with the given value
-     * 
-     * @param n
-     *            Node
-     */
-    public NodeWritable(Node n) {
-        this.set(n);
-    }
-
-    /**
-     * Gets the node
-     * 
-     * @return Node
-     */
-    public Node get() {
-        // We may not have yet loaded the node
-        if (this.node == null) {
-            // If term is set to undefined then node is supposed to be null
-            if (this.term.isSet() && !this.term.isSetUndefined()) {
-                this.node = ThriftConvert.convert(this.term);
-            }
-        }
-        return this.node;
-    }
-
-    /**
-     * Sets the node
-     * 
-     * @param n
-     *            Node
-     */
-    public void set(Node n) {
-        this.node = n;
-        // Clear the term for now
-        // We only convert the Node to a term as and when we want to write it
-        // out in order to not waste effort if the value is never written out
-        this.term.clear();
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        // Clear previous value
-        this.node = null;
-        this.term.clear();
-
-        // Read in the new value
-        int termLength = input.readInt();
-        byte[] buffer = new byte[termLength];
-        input.readFully(buffer);
-        try {
-            ThriftConverter.fromBytes(buffer, this.term);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-
-        // Note that we don't convert it back into a Node at this time
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        // May not yet have prepared the Thrift term
-        if (!this.term.isSet()) {
-            if (this.node == null) {
-                this.term.setUndefined(TRDF.UNDEF);
-            } else {
-                ThriftConvert.toThrift(this.node, null, this.term, false);
-            }
-        }
-
-        // Write out the Thrift term
-        byte[] buffer;
-        try {
-            buffer = ThriftConverter.toBytes(this.term);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        output.writeInt(buffer.length);
-        output.write(buffer);
-    }
-
-    @Override
-    public int compareTo(NodeWritable other) {
-        // Use get() rather than accessing the field directly because the node
-        // field is lazily instantiated from the Thrift term
-        return NodeUtils.compareRDFTerms(this.get(), other.get());
-    }
-
-    @Override
-    public String toString() {
-        // Use get() rather than accessing the field directly because the node
-        // field is lazily instantiated from the Thrift term
-        Node n = this.get();
-        if (n == null)
-            return "";
-        return n.toString();
-    }
-
-    @Override
-    public int hashCode() {
-        // Use get() rather than accessing the field directly because the node
-        // field is lazily instantiated from the Thrift term
-        Node n = this.get();
-        return n != null ? this.get().hashCode() : 0;
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        if (!(other instanceof NodeWritable))
-            return false;
-        return this.compareTo((NodeWritable) other) == 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java b/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
deleted file mode 100644
index 3d9dd00..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
-import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
-import org.apache.jena.riot.thrift.ThriftConvert;
-import org.apache.jena.riot.thrift.wire.RDF_Quad;
-import org.apache.thrift.TException;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A writable quad
- */
-public class QuadWritable extends AbstractNodeTupleWritable<Quad> {
-
-    static {
-        WritableComparator.define(QuadWritable.class, new SimpleBinaryComparator());
-    }
-
-    private RDF_Quad quad = new RDF_Quad();
-
-    /**
-     * Creates a new empty instance
-     */
-    public QuadWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance with the given value
-     * 
-     * @param q
-     *            Quad
-     */
-    public QuadWritable(Quad q) {
-        super(q);
-    }
-
-    /**
-     * Creates a new instance from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static QuadWritable read(DataInput input) throws IOException {
-        QuadWritable q = new QuadWritable();
-        q.readFields(input);
-        return q;
-    }
-
-    @Override
-    public void set(Quad tuple) {
-        super.set(tuple);
-        this.quad.clear();
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        this.quad.clear();
-        int tripleLength = input.readInt();
-        byte[] buffer = new byte[tripleLength];
-        input.readFully(buffer);
-        try {
-            ThriftConverter.fromBytes(buffer, this.quad);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        this.setInternal(new Quad(ThriftConvert.convert(this.quad.getG()), ThriftConvert.convert(this.quad.getS()),
-                ThriftConvert.convert(this.quad.getP()), ThriftConvert.convert(this.quad.getO())));
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        if (this.get() == null)
-            throw new IOException(
-                    "Null quads cannot be written using this class, consider using NodeTupleWritable instead");
-
-        // May not have yet prepared the Thrift triple
-        if (!this.quad.isSetS()) {
-            Quad tuple = this.get();
-            this.quad.setG(ThriftConvert.convert(tuple.getGraph(), false));
-            this.quad.setS(ThriftConvert.convert(tuple.getSubject(), false));
-            this.quad.setP(ThriftConvert.convert(tuple.getPredicate(), false));
-            this.quad.setO(ThriftConvert.convert(tuple.getObject(), false));
-        }
-
-        byte[] buffer;
-        try {
-            buffer = ThriftConverter.toBytes(this.quad);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        output.writeInt(buffer.length);
-        output.write(buffer);
-    }
-
-    @Override
-    protected Quad createTuple(Node[] ns) {
-        if (ns.length != 4)
-            throw new IllegalArgumentException(String.format(
-                    "Incorrect number of nodes to form a quad - got %d but expected 4", ns.length));
-        return new Quad(ns[0], ns[1], ns[2], ns[3]);
-    }
-
-    @Override
-    protected Node[] createNodes(Quad tuple) {
-        return new Node[] { tuple.getGraph(), tuple.getSubject(), tuple.getPredicate(), tuple.getObject() };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java b/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
deleted file mode 100644
index a17052b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.types;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
-import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
-import org.apache.jena.riot.thrift.ThriftConvert;
-import org.apache.jena.riot.thrift.wire.RDF_Triple;
-import org.apache.thrift.TException;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A writable triple
- * 
- * 
- * 
- */
-public class TripleWritable extends AbstractNodeTupleWritable<Triple> {
-    
-    static {
-        WritableComparator.define(TripleWritable.class, new SimpleBinaryComparator());
-    }
-
-    private RDF_Triple triple = new RDF_Triple();
-
-    /**
-     * Creates a new instance using the default NTriples node formatter
-     */
-    public TripleWritable() {
-        this(null);
-    }
-
-    /**
-     * Creates a new instance with a given value that uses a specific node
-     * formatter
-     * 
-     * @param t
-     *            Triple
-     */
-    public TripleWritable(Triple t) {
-        super(t);
-    }
-
-    /**
-     * Creates a new instance from the given input
-     * 
-     * @param input
-     *            Input
-     * @return New instance
-     * @throws IOException
-     */
-    public static TripleWritable read(DataInput input) throws IOException {
-        TripleWritable t = new TripleWritable();
-        t.readFields(input);
-        return t;
-    }
-
-    @Override
-    public void set(Triple tuple) {
-        super.set(tuple);
-        this.triple.clear();
-    }
-
-    @Override
-    public void readFields(DataInput input) throws IOException {
-        this.triple.clear();
-        int tripleLength = input.readInt();
-        byte[] buffer = new byte[tripleLength];
-        input.readFully(buffer);
-        try {
-            ThriftConverter.fromBytes(buffer, this.triple);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        this.setInternal(new Triple(ThriftConvert.convert(this.triple.getS()),
-                ThriftConvert.convert(this.triple.getP()), ThriftConvert.convert(this.triple.getO())));
-    }
-
-    @Override
-    public void write(DataOutput output) throws IOException {
-        if (this.get() == null)
-            throw new IOException(
-                    "Null triples cannot be written using this class, consider using NodeTupleWritable instead");
-        
-        // May not have yet prepared the Thrift triple
-        if (!this.triple.isSetS()) {
-            Triple tuple = this.get();
-            this.triple.setS(ThriftConvert.convert(tuple.getSubject(), false));
-            this.triple.setP(ThriftConvert.convert(tuple.getPredicate(), false));
-            this.triple.setO(ThriftConvert.convert(tuple.getObject(), false));
-        }
-
-        byte[] buffer;
-        try {
-            buffer = ThriftConverter.toBytes(this.triple);
-        } catch (TException e) {
-            throw new IOException(e);
-        }
-        output.writeInt(buffer.length);
-        output.write(buffer);
-    }
-
-    @Override
-    protected Triple createTuple(Node[] ns) {
-        if (ns.length != 3)
-            throw new IllegalArgumentException(String.format(
-                    "Incorrect number of nodes to form a triple - got %d but expected 3", ns.length));
-        return new Triple(ns[0], ns[1], ns[2]);
-    }
-
-    @Override
-    protected Node[] createNodes(Triple tuple) {
-        return new Node[] { tuple.getSubject(), tuple.getPredicate(), tuple.getObject() };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java b/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
deleted file mode 100644
index 6c46714..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.types.comparators;
-
-import org.apache.hadoop.io.WritableComparator;
-
-/**
- * A general purpose comparator that may be used with any types which can be
- * compared directly on their binary encodings
- */
-public class SimpleBinaryComparator extends WritableComparator {
-
-    @Override
-    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-        return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java b/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
deleted file mode 100644
index 0675afc..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * 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.types.converters;
-
-import java.io.ByteArrayOutputStream;
-
-import org.apache.jena.riot.thrift.wire.RDF_Quad;
-import org.apache.jena.riot.thrift.wire.RDF_Term;
-import org.apache.jena.riot.thrift.wire.RDF_Triple;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.TIOStreamTransport;
-import org.apache.thrift.transport.TMemoryInputTransport;
-import org.apache.thrift.transport.TTransport;
-
-/**
- * Helper for converting between the binary representation of Nodes, Triples and
- * Quads and their Jena API equivalents
- * 
- */
-public class ThriftConverter {
-
-    private static ThreadLocal<TMemoryInputTransport> inputTransports = new ThreadLocal<>();
-    private static ThreadLocal<TProtocol> inputProtocols = new ThreadLocal<>();
-
-    private static ThreadLocal<ByteArrayOutputStream> outputStreams = new ThreadLocal<>();
-    private static ThreadLocal<TTransport> outputTransports = new ThreadLocal<>();
-    private static ThreadLocal<TProtocol> outputProtocols = new ThreadLocal<>();
-
-    private static TMemoryInputTransport getInputTransport() {
-        TMemoryInputTransport transport = inputTransports.get();
-        if (transport != null)
-            return transport;
-
-        transport = new TMemoryInputTransport();
-        inputTransports.set(transport);
-        return transport;
-    }
-
-    private static TProtocol getInputProtocol() {
-        TProtocol protocol = inputProtocols.get();
-        if (protocol != null)
-            return protocol;
-
-        protocol = new TCompactProtocol(getInputTransport());
-        inputProtocols.set(protocol);
-        return protocol;
-    }
-
-    private static ByteArrayOutputStream getOutputStream() {
-        ByteArrayOutputStream output = outputStreams.get();
-        if (output != null)
-            return output;
-
-        output = new ByteArrayOutputStream();
-        outputStreams.set(output);
-        return output;
-    }
-
-    private static TTransport getOutputTransport() {
-        TTransport transport = outputTransports.get();
-        if (transport != null)
-            return transport;
-
-        transport = new TIOStreamTransport(getOutputStream());
-        outputTransports.set(transport);
-        return transport;
-    }
-
-    private static TProtocol getOutputProtocol() {
-        TProtocol protocol = outputProtocols.get();
-        if (protocol != null)
-            return protocol;
-
-        protocol = new TCompactProtocol(getOutputTransport());
-        outputProtocols.set(protocol);
-        return protocol;
-    }
-
-    public static byte[] toBytes(RDF_Term term) throws TException {
-        ByteArrayOutputStream output = getOutputStream();
-        output.reset();
-
-        TProtocol protocol = getOutputProtocol();
-        term.write(protocol);
-
-        return output.toByteArray();
-    }
-
-    public static void fromBytes(byte[] bs, RDF_Term term) throws TException {
-        TMemoryInputTransport transport = getInputTransport();
-        transport.reset(bs);
-        TProtocol protocol = getInputProtocol();
-        term.read(protocol);
-    }
-
-    public static void fromBytes(byte[] buffer, RDF_Triple triple) throws TException {
-        TMemoryInputTransport transport = getInputTransport();
-        transport.reset(buffer);
-        TProtocol protocol = getInputProtocol();
-        triple.read(protocol);
-    }
-
-    public static byte[] toBytes(RDF_Triple triple) throws TException {
-        ByteArrayOutputStream output = getOutputStream();
-        output.reset();
-
-        TProtocol protocol = getOutputProtocol();
-        triple.write(protocol);
-
-        return output.toByteArray();
-    }
-
-    public static void fromBytes(byte[] buffer, RDF_Quad quad) throws TException {
-        TMemoryInputTransport transport = getInputTransport();
-        transport.reset(buffer);
-        TProtocol protocol = getInputProtocol();
-        quad.read(protocol);
-    }
-
-    public static byte[] toBytes(RDF_Quad quad) throws TException {
-        ByteArrayOutputStream output = getOutputStream();
-        output.reset();
-
-        TProtocol protocol = getOutputProtocol();
-        quad.write(protocol);
-
-        return output.toByteArray();
-    }
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
deleted file mode 100644
index 1cda0bd..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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;
-
-/**
- * Abstract tests for blocked triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractBlockedQuadInputFormatTests extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected boolean canSplitInputs() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
deleted file mode 100644
index 2e1e865..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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;
-
-/**
- * Abstract tests for blocked triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractBlockedTripleInputFormatTests extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected boolean canSplitInputs() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
deleted file mode 100644
index e22650f..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
+++ /dev/null
@@ -1,612 +0,0 @@
-/*
- * 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 java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Abstract node tuple input format tests
- * 
- * 
- * 
- * @param <TValue>
- * @param <T>
- */
-public abstract class AbstractNodeTupleInputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleInputFormatTests.class);
-
-    protected static final int EMPTY_SIZE = 0, SMALL_SIZE = 100, LARGE_SIZE = 10000, BAD_SIZE = 100, MIXED_SIZE = 100;
-    protected static final String EMPTY = "empty";
-    protected static final String SMALL = "small";
-    protected static final String LARGE = "large";
-    protected static final String BAD = "bad";
-    protected static final String MIXED = "mixed";
-
-    /**
-     * Temporary folder for the tests
-     */
-    @Rule
-    public TemporaryFolder folder = new TemporaryFolder();
-
-    protected File empty, small, large, bad, mixed;
-
-    /**
-     * Prepares the inputs for the tests
-     * 
-     * @throws IOException
-     */
-    @Before
-    public void beforeTest() throws IOException {
-        this.prepareInputs();
-    }
-
-    /**
-     * Cleans up the inputs after each test
-     */
-    @After
-    public void afterTest() {
-        // Should be unnecessary since JUnit will clean up the temporary folder
-        // anyway but best to do this regardless
-        if (empty != null)
-            empty.delete();
-        if (small != null)
-            small.delete();
-        if (large != null)
-            large.delete();
-        if (bad != null)
-            bad.delete();
-        if (mixed != null)
-            mixed.delete();
-    }
-
-    /**
-     * Prepares a fresh configuration
-     * 
-     * @return Configuration
-     */
-    protected Configuration prepareConfiguration() {
-        Configuration config = new Configuration(true);
-        // Nothing else to do
-        return config;
-    }
-
-    /**
-     * Prepares the inputs
-     * 
-     * @throws IOException
-     */
-    protected void prepareInputs() throws IOException {
-        String ext = this.getFileExtension();
-        empty = folder.newFile(EMPTY + ext);
-        this.generateTuples(empty, EMPTY_SIZE);
-        small = folder.newFile(SMALL + ext);
-        this.generateTuples(small, SMALL_SIZE);
-        large = folder.newFile(LARGE + ext);
-        this.generateTuples(large, LARGE_SIZE);
-        bad = folder.newFile(BAD + ext);
-        this.generateBadTuples(bad, BAD_SIZE);
-        mixed = folder.newFile(MIXED + ext);
-        this.generateMixedTuples(mixed, MIXED_SIZE);
-    }
-
-    /**
-     * Gets the extra file extension to add to the filenames
-     * 
-     * @return File extension
-     */
-    protected abstract String getFileExtension();
-
-    /**
-     * Generates tuples used for tests
-     * 
-     * @param f
-     *            File
-     * @param num
-     *            Number of tuples to generate
-     * @throws IOException
-     */
-    protected final void generateTuples(File f, int num) throws IOException {
-        this.generateTuples(this.getOutputStream(f), num);
-    }
-
-    /**
-     * Gets the output stream to use for generating tuples
-     * 
-     * @param f
-     *            File
-     * @return Output Stream
-     * @throws IOException
-     */
-    protected OutputStream getOutputStream(File f) throws IOException {
-        return new FileOutputStream(f, false);
-    }
-
-    /**
-     * Generates tuples used for tests
-     * 
-     * @param output
-     *            Output Stream to write to
-     * @param num
-     *            Number of tuples to generate
-     * @throws IOException
-     */
-    protected abstract void generateTuples(OutputStream output, int num) throws IOException;
-
-    /**
-     * Generates bad tuples used for tests
-     * 
-     * @param f
-     *            File
-     * @param num
-     *            Number of bad tuples to generate
-     * @throws IOException
-     */
-    protected final void generateBadTuples(File f, int num) throws IOException {
-        this.generateBadTuples(this.getOutputStream(f), num);
-    }
-
-    /**
-     * Generates bad tuples used for tests
-     * 
-     * @param output
-     *            Output Stream to write to
-     * @param num
-     *            Number of bad tuples to generate
-     * @throws IOException
-     */
-    protected abstract void generateBadTuples(OutputStream output, int num) throws IOException;
-
-    /**
-     * Generates a mixture of good and bad tuples used for tests
-     * 
-     * @param f
-     *            File
-     * @param num
-     *            Number of tuples to generate, they should be a 50/50 mix of
-     *            good and bad tuples
-     * @throws IOException
-     */
-    protected final void generateMixedTuples(File f, int num) throws IOException {
-        this.generateMixedTuples(this.getOutputStream(f), num);
-    }
-
-    /**
-     * Generates a mixture of good and bad tuples used for tests
-     * 
-     * @param output
-     *            Output Stream to write to
-     * @param num
-     *            Number of tuples to generate, they should be a 50/50 mix of
-     *            good and bad tuples
-     * @throws IOException
-     */
-    protected abstract void generateMixedTuples(OutputStream output, int num) throws IOException;
-
-    /**
-     * Adds an input path to the job configuration
-     * 
-     * @param f
-     *            File
-     * @param config
-     *            Configuration
-     * @param job
-     *            Job
-     * @throws IOException
-     */
-    protected void addInputPath(File f, Configuration config, Job job) throws IOException {
-        FileSystem fs = FileSystem.getLocal(config);
-        Path inputPath = fs.makeQualified(new Path(f.getAbsolutePath()));
-        FileInputFormat.addInputPath(job, inputPath);
-    }
-
-    protected final int countTuples(RecordReader<LongWritable, T> reader) throws IOException, InterruptedException {
-        int count = 0;
-
-        // Check initial progress
-        LOG.info(String.format("Initial Reported Progress %f", reader.getProgress()));
-        float progress = reader.getProgress();
-        if (Float.compare(0.0f, progress) == 0) {
-            Assert.assertEquals(0.0d, reader.getProgress(), 0.0d);
-        } else if (Float.compare(1.0f, progress) == 0) {
-            // If reader is reported 1.0 straight away then we expect there to
-            // be no key values
-            Assert.assertEquals(1.0d, reader.getProgress(), 0.0d);
-            Assert.assertFalse(reader.nextKeyValue());
-        } else {
-            Assert.fail(String.format(
-                    "Expected progress of 0.0 or 1.0 before reader has been accessed for first time but got %f",
-                    progress));
-        }
-
-        // Count tuples
-        boolean debug = LOG.isDebugEnabled();
-        while (reader.nextKeyValue()) {
-            count++;
-            progress = reader.getProgress();
-            if (debug)
-                LOG.debug(String.format("Current Reported Progress %f", progress));
-            Assert.assertTrue(String.format("Progress should be in the range 0.0 < p <= 1.0 but got %f", progress),
-                    progress > 0.0f && progress <= 1.0f);
-        }
-        reader.close();
-        LOG.info(String.format("Got %d tuples from this record reader", count));
-
-        // Check final progress
-        LOG.info(String.format("Final Reported Progress %f", reader.getProgress()));
-        Assert.assertEquals(1.0d, reader.getProgress(), 0.0d);
-
-        return count;
-    }
-
-    protected final void checkTuples(RecordReader<LongWritable, T> reader, int expected) throws IOException,
-            InterruptedException {
-        Assert.assertEquals(expected, this.countTuples(reader));
-    }
-
-    /**
-     * Runs a test with a single input
-     * 
-     * @param input
-     *            Input
-     * @param expectedTuples
-     *            Expected tuples
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected final void testSingleInput(File input, int expectedSplits, int expectedTuples) throws IOException,
-            InterruptedException {
-        // Prepare configuration
-        Configuration config = this.prepareConfiguration();
-        this.testSingleInput(config, input, expectedSplits, expectedTuples);
-    }
-
-    /**
-     * Runs a test with a single input
-     * 
-     * @param config
-     *            Configuration
-     * @param input
-     *            Input
-     * @param expectedTuples
-     *            Expected tuples
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected final void testSingleInput(Configuration config, File input, int expectedSplits, int expectedTuples)
-            throws IOException, InterruptedException {
-        // Set up fake job
-        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
-        Job job = Job.getInstance(config);
-        job.setInputFormatClass(inputFormat.getClass());
-        this.addInputPath(input, job.getConfiguration(), job);
-        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-        Assert.assertEquals(1, FileInputFormat.getInputPaths(context).length);
-        NLineInputFormat.setNumLinesPerSplit(job, LARGE_SIZE);
-
-        // Check splits
-        List<InputSplit> splits = inputFormat.getSplits(context);
-        Assert.assertEquals(expectedSplits, splits.size());
-
-        // Check tuples
-        for (InputSplit split : splits) {
-            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
-            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
-            reader.initialize(split, taskContext);
-            this.checkTuples(reader, expectedTuples);
-        }
-    }
-
-    protected abstract InputFormat<LongWritable, T> getInputFormat();
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_01() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(empty, this.canSplitInputs() ? 0 : 1, EMPTY_SIZE);
-    }
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_02() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(small, 1, SMALL_SIZE);
-    }
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_03() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(large, 1, LARGE_SIZE);
-    }
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_04() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(bad, 1, 0);
-    }
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_05() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(mixed, 1, MIXED_SIZE / 2);
-    }
-
-    /**
-     * Tests behaviour when ignoring bad tuples is disabled
-     * 
-     * @throws InterruptedException
-     * @throws IOException
-     */
-    @Test(expected = IOException.class)
-    public final void fail_on_bad_input_01() throws IOException, InterruptedException {
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        Assert.assertFalse(config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true));
-        testSingleInput(config, bad, 1, 0);
-    }
-
-    /**
-     * Tests behaviour when ignoring bad tuples is disabled
-     * 
-     * @throws InterruptedException
-     * @throws IOException
-     */
-    @Test(expected = IOException.class)
-    public final void fail_on_bad_input_02() throws IOException, InterruptedException {
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        Assert.assertFalse(config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true));
-        testSingleInput(config, mixed, 1, MIXED_SIZE / 2);
-    }
-
-    /**
-     * Runs a multiple input test
-     * 
-     * @param inputs
-     *            Inputs
-     * @param expectedSplits
-     *            Number of splits expected
-     * @param expectedTuples
-     *            Number of tuples expected
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected final void testMultipleInputs(File[] inputs, int expectedSplits, int expectedTuples) throws IOException,
-            InterruptedException {
-        // Prepare configuration and inputs
-        Configuration config = this.prepareConfiguration();
-
-        // Set up fake job
-        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
-        Job job = Job.getInstance(config);
-        job.setInputFormatClass(inputFormat.getClass());
-        for (File input : inputs) {
-            this.addInputPath(input, job.getConfiguration(), job);
-        }
-        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-        Assert.assertEquals(inputs.length, FileInputFormat.getInputPaths(context).length);
-        NLineInputFormat.setNumLinesPerSplit(job, expectedTuples);
-
-        // Check splits
-        List<InputSplit> splits = inputFormat.getSplits(context);
-        Assert.assertEquals(expectedSplits, splits.size());
-
-        // Check tuples
-        int count = 0;
-        for (InputSplit split : splits) {
-            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
-            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
-            reader.initialize(split, taskContext);
-            count += this.countTuples(reader);
-        }
-        Assert.assertEquals(expectedTuples, count);
-    }
-
-    /**
-     * tuples test with multiple inputs
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void multiple_inputs_01() throws IOException, InterruptedException, ClassNotFoundException {
-        testMultipleInputs(new File[] { empty, small, large }, this.canSplitInputs() ? 2 : 3, EMPTY_SIZE + SMALL_SIZE
-                + LARGE_SIZE);
-    }
-
-    /**
-     * tuples test with multiple inputs
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void multiple_inputs_02() throws IOException, InterruptedException, ClassNotFoundException {
-        testMultipleInputs(new File[] { folder.getRoot() }, this.canSplitInputs() ? 4 : 5, EMPTY_SIZE + SMALL_SIZE
-                + LARGE_SIZE + (MIXED_SIZE / 2));
-    }
-
-    protected final void testSplitInputs(Configuration config, File[] inputs, int expectedSplits, int expectedTuples)
-            throws IOException, InterruptedException {
-        // Set up fake job
-        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
-        Job job = Job.getInstance(config);
-        job.setInputFormatClass(inputFormat.getClass());
-        for (File input : inputs) {
-            this.addInputPath(input, job.getConfiguration(), job);
-        }
-        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-        Assert.assertEquals(inputs.length, FileInputFormat.getInputPaths(context).length);
-
-        // Check splits
-        List<InputSplit> splits = inputFormat.getSplits(context);
-        Assert.assertEquals(expectedSplits, splits.size());
-
-        // Check tuples
-        int count = 0;
-        for (InputSplit split : splits) {
-            // Validate split
-            Assert.assertTrue(this.isValidSplit(split, config));
-
-            // Read split
-            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
-            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
-            reader.initialize(split, taskContext);
-            count += this.countTuples(reader);
-        }
-        Assert.assertEquals(expectedTuples, count);
-    }
-
-    /**
-     * Determines whether an input split is valid
-     * 
-     * @param split
-     *            Input split
-     * @return True if a valid split, false otherwise
-     * @throws IOException
-     */
-    protected boolean isValidSplit(InputSplit split, Configuration config) throws IOException {
-        return split instanceof FileSplit;
-    }
-
-    /**
-     * Indicates whether inputs can be split, defaults to true
-     * 
-     * @return Whether inputs can be split
-     */
-    protected boolean canSplitInputs() {
-        return true;
-    }
-
-    /**
-     * Tests for input splitting
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public final void split_input_01() throws IOException, InterruptedException, ClassNotFoundException {
-        Assume.assumeTrue(this.canSplitInputs());
-
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
-        this.testSplitInputs(config, new File[] { small }, 100, SMALL_SIZE);
-    }
-
-    /**
-     * Tests for input splitting
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public final void split_input_02() throws IOException, InterruptedException, ClassNotFoundException {
-        Assume.assumeTrue(this.canSplitInputs());
-
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        config.setLong(NLineInputFormat.LINES_PER_MAP, 10);
-        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
-        this.testSplitInputs(config, new File[] { small }, 10, SMALL_SIZE);
-    }
-
-    /**
-     * Tests for input splitting
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public final void split_input_03() throws IOException, InterruptedException, ClassNotFoundException {
-        Assume.assumeTrue(this.canSplitInputs());
-
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        config.setLong(NLineInputFormat.LINES_PER_MAP, 100);
-        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
-        this.testSplitInputs(config, new File[] { large }, 100, LARGE_SIZE);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
deleted file mode 100644
index 78d7f33..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for Quad input formats
- * 
- *
- */
-public abstract class AbstractQuadsInputFormatTests extends AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected void generateTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-    
-    @Override
-    protected void generateBadTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write("<http://broken\n".getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
-        boolean bad = false;
-        for (int i = 0; i < num; i++, bad = !bad) {
-            if (bad) {
-                output.write("<http://broken\n".getBytes(utf8));
-            } else {
-                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
-            }
-        }
-        output.flush();
-        output.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
deleted file mode 100644
index 65a9889..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for Triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractTriplesInputFormatTests extends AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected void generateTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "<http://broken\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
-        boolean bad = false;
-        for (int i = 0; i < num; i++, bad = !bad) {
-            if (bad) {
-                output.write("<http://broken\n".getBytes(utf8));
-            } else {
-                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
-            }
-        }
-        output.flush();
-        output.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
deleted file mode 100644
index 0b6cfde..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-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.query.Dataset;
-import com.hp.hpl.jena.query.DatasetFactory;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ModelFactory;
-import com.hp.hpl.jena.rdf.model.Property;
-import com.hp.hpl.jena.rdf.model.Resource;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for Quad input formats
- * 
- * 
- * 
- */
-public abstract class AbstractWholeFileQuadInputFormatTests extends AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-
-    private void writeTuples(Dataset ds, OutputStream output) {
-        RDFDataMgr.write(output, ds, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
-    }
-
-    /**
-     * Gets the RDF language to write out generate tuples in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-    private void writeGoodTuples(OutputStream output, int num) throws IOException {
-        Dataset ds = DatasetFactory.createMem();
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num; i++) {
-            if (i % 100 == 0) {
-                ds.addNamedModel("http://example.org/graphs/" + (i / 100), m);
-                m = ModelFactory.createDefaultModel();
-            }
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        if (!m.isEmpty()) {
-            ds.addNamedModel("http://example.org/graphs/extra", m);
-        }
-        this.writeTuples(ds, output);
-    }
-
-    @Override
-    protected final void generateTuples(OutputStream output, int num) throws IOException {
-        this.writeGoodTuples(output, num);
-        output.close();
-    }
-
-    @Override
-    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
-        // Write good data
-        this.writeGoodTuples(output, num / 2);
-
-        // Write junk data
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num / 2; i++) {
-            output.write(junk);
-        }
-
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
deleted file mode 100644
index b68d662..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-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.Triple;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ModelFactory;
-import com.hp.hpl.jena.rdf.model.Property;
-import com.hp.hpl.jena.rdf.model.Resource;
-
-/**
- * Abstract tests for Triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractWholeFileTripleInputFormatTests extends AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-    
-    private void writeTuples(Model m, OutputStream output) {
-        RDFDataMgr.write(output, m, this.getRdfLanguage());
-    }
-        
-    /**
-     * Gets the RDF language to write out generate tuples in
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-    
-    @Override
-    protected final void generateTuples(OutputStream output, int num) throws IOException {
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num; i++) {
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        this.writeTuples(m, output);
-        output.close();
-    }
-    
-    @Override
-    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
-        // Write good data
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num / 2; i++) {
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        this.writeTuples(m, output);
-        
-        // Write junk data
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num / 2; i++) {
-            output.write(junk);
-        }
-        
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
deleted file mode 100644
index 4bb0939..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
+++ /dev/null
@@ -1,636 +0,0 @@
-/*
- * 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.bnodes;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.nio.file.StandardCopyOption;
-import java.nio.file.attribute.FileAttribute;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.log4j.BasicConfigurator;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * Test case that embodies the scenario described in JENA-820
- */
-@SuppressWarnings("unused")
-public abstract class AbstractBlankNodeTests<T, TValue extends AbstractNodeTupleWritable<T>> {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractBlankNodeTests.class);
-
-    @BeforeClass
-    public static void setup() {
-        // Enable if you need to diagnose test failures
-        // Useful since it includes printing the file names of the temporary
-        // files being used
-        // BasicConfigurator.resetConfiguration();
-        // BasicConfigurator.configure();
-    }
-
-    /**
-     * Gets the extension for the initial input files
-     * 
-     * @return Extension including the {@code .}
-     */
-    protected abstract String getInitialInputExtension();
-
-    /**
-     * Creates a tuple
-     * 
-     * @param s
-     *            Subject
-     * @param p
-     *            Predicate
-     * @param o
-     *            Object
-     * @return Tuple
-     */
-    protected abstract T createTuple(Node s, Node p, Node o);
-
-    /**
-     * Writes out the given tuples to the given file
-     * 
-     * @param f
-     *            File
-     * @param tuples
-     *            Tuples
-     * @throws FileNotFoundException
-     */
-    protected abstract void writeTuples(File f, List<T> tuples) throws FileNotFoundException;
-
-    /**
-     * Creates the input format for reading the initial inputs
-     * 
-     * @return Input format
-     */
-    protected abstract InputFormat<LongWritable, TValue> createInitialInputFormat();
-
-    /**
-     * Creates the output format for writing the intermediate output
-     * 
-     * @return Output format
-     */
-    protected abstract OutputFormat<LongWritable, TValue> createIntermediateOutputFormat();
-
-    /**
-     * Creates the input format for reading the intermediate outputs back in
-     * 
-     * @return Input format
-     */
-    protected abstract InputFormat<LongWritable, TValue> createIntermediateInputFormat();
-
-    /**
-     * Gets the subject of the tuple
-     * 
-     * @param value
-     *            Tuple
-     * @return Subject
-     */
-    protected abstract Node getSubject(T value);
-
-    /**
-     * Gets whether the format being tested respects the RIOT
-     * {@link ParserProfile}
-     * 
-     * @return True if parser profile is respected, false otherwise
-     */
-    protected boolean respectsParserProfile() {
-        return true;
-    }
-
-    /**
-     * Gets whether the format being tested preserves blank node identity
-     * 
-     * @return True if identity is presereved, false otherwise
-     */
-    protected boolean preservesBlankNodeIdentity() {
-        return false;
-    }
-
-    /**
-     * Test that starts with two blank nodes with the same identity in a single
-     * file, splits them over two files and checks that we can workaround
-     * JENA-820 successfully by setting the
-     * {@link RdfIOConstants#GLOBAL_BNODE_IDENTITY} flag for our subsequent job
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void blank_node_divergence_01() throws IOException, InterruptedException {
-        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
-        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
-        
-        // Temporary files
-        File a = File.createTempFile("bnode_divergence", getInitialInputExtension());
-        File intermediateOutputDir = Files.createTempDirectory("bnode_divergence", new FileAttribute[0]).toFile();
-
-        try {
-            // Prepare the input data
-            // Two mentions of the same blank node in the same file
-            List<T> tuples = new ArrayList<>();
-            Node bnode = NodeFactory.createAnon();
-            Node pred = NodeFactory.createURI("http://example.org/predicate");
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
-            writeTuples(a, tuples);
-
-            // Set up fake job which will process the file as a single split
-            Configuration config = new Configuration(true);
-            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
-            Job job = Job.getInstance(config);
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()));
-            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            List<InputSplit> splits = inputFormat.getSplits(context);
-            Assert.assertEquals(1, splits.size());
-
-            for (InputSplit split : splits) {
-                // Initialize the input reading
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        createAttemptID(1, 1, 1));
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                // Copy the input to the output - each triple goes to a separate
-                // output file
-                // This is how we force multiple files to be produced
-                int taskID = 1;
-                while (reader.nextKeyValue()) {
-                    // Prepare the output writing
-                    OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
-                    TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                            createAttemptID(1, ++taskID, 1));
-                    RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
-
-                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
-                    writer.close(outputTaskContext);
-                }
-            }
-
-            // Promote outputs from temporary status
-            promoteInputs(intermediateOutputDir);
-
-            // Now we need to create a subsequent job that reads the
-            // intermediate outputs
-            // As described in JENA-820 at this point the blank nodes are
-            // consistent, however when we read them from different files they
-            // by default get treated as different nodes and so the blank nodes
-            // diverge which is incorrect and undesirable behaviour in
-            // multi-stage pipelines
-            System.out.println(intermediateOutputDir.getAbsolutePath());
-            job = Job.getInstance(config);
-            inputFormat = createIntermediateInputFormat();
-            job.setInputFormatClass(inputFormat.getClass());
-            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
-
-            // Enabling this flag works around the JENA-820 issue
-            job.getConfiguration().setBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, true);
-            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            splits = inputFormat.getSplits(context);
-            Assert.assertEquals(2, splits.size());
-
-            // Expect to end up with a single blank node
-            Set<Node> nodes = new HashSet<Node>();
-            for (InputSplit split : splits) {
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        new TaskAttemptID());
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                while (reader.nextKeyValue()) {
-                    nodes.add(getSubject(reader.getCurrentValue().get()));
-                }
-            }
-            // Nodes should not have diverged
-            Assert.assertEquals(1, nodes.size());
-
-        } finally {
-            a.delete();
-            deleteDirectory(intermediateOutputDir);
-        }
-    }
-
-    /**
-     * Test that starts with two blank nodes with the same identity in a single
-     * file, splits them over two files and shows that they diverge in the
-     * subsequent job when the JENA-820 workaround is not enabled
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void blank_node_divergence_02() throws IOException, InterruptedException {
-        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
-        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
-        
-        // Temporary files
-        File a = File.createTempFile("bnode_divergence", getInitialInputExtension());
-        File intermediateOutputDir = Files.createTempDirectory("bnode_divergence", new FileAttribute[0]).toFile();
-
-        try {
-            // Prepare the input data
-            // Two mentions of the same blank node in the same file
-            List<T> tuples = new ArrayList<>();
-            Node bnode = NodeFactory.createAnon();
-            Node pred = NodeFactory.createURI("http://example.org/predicate");
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
-            writeTuples(a, tuples);
-
-            // Set up fake job which will process the file as a single split
-            Configuration config = new Configuration(true);
-            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
-            Job job = Job.getInstance(config);
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()));
-            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            List<InputSplit> splits = inputFormat.getSplits(context);
-            Assert.assertEquals(1, splits.size());
-
-            for (InputSplit split : splits) {
-                // Initialize the input reading
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        createAttemptID(1, 1, 1));
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                // Copy the input to the output - each triple goes to a separate
-                // output file
-                // This is how we force multiple files to be produced
-                int taskID = 1;
-                while (reader.nextKeyValue()) {
-                    // Prepare the output writing
-                    OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
-                    TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                            createAttemptID(1, ++taskID, 1));
-                    RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
-
-                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
-                    writer.close(outputTaskContext);
-                }
-            }
-
-            // Promote outputs from temporary status
-            promoteInputs(intermediateOutputDir);
-
-            // Now we need to create a subsequent job that reads the
-            // intermediate outputs
-            // As described in JENA-820 at this point the blank nodes are
-            // consistent, however when we read them from different files they
-            // by default get treated as different nodes and so the blank nodes
-            // diverge which is incorrect and undesirable behaviour in
-            // multi-stage pipelines. However it is the default behaviour
-            // because when we start from external inputs we want them to be
-            // file scoped.
-            System.out.println(intermediateOutputDir.getAbsolutePath());
-            job = Job.getInstance(config);
-            inputFormat = createIntermediateInputFormat();
-            job.setInputFormatClass(inputFormat.getClass());
-            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
-
-            // Make sure JENA-820 flag is disabled
-            job.getConfiguration().setBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, false);
-            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            splits = inputFormat.getSplits(context);
-            Assert.assertEquals(2, splits.size());
-
-            // Expect to end up with a single blank node
-            Set<Node> nodes = new HashSet<Node>();
-            for (InputSplit split : splits) {
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        new TaskAttemptID());
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                while (reader.nextKeyValue()) {
-                    nodes.add(getSubject(reader.getCurrentValue().get()));
-                }
-            }
-            // Nodes should have diverged
-            Assert.assertEquals(2, nodes.size());
-
-        } finally {
-            a.delete();
-            deleteDirectory(intermediateOutputDir);
-        }
-    }
-
-    /**
-     * Test that starts with two blank nodes in two different files and checks
-     * that writing them to a single file does not conflate them
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void blank_node_identity_01() throws IOException, InterruptedException {
-        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
-        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
-        
-        // Temporary files
-        File a = File.createTempFile("bnode_identity", getInitialInputExtension());
-        File b = File.createTempFile("bnode_identity", getInitialInputExtension());
-        File intermediateOutputDir = Files.createTempDirectory("bnode_identity", new FileAttribute[0]).toFile();
-
-        try {
-            // Prepare the input data
-            // Different blank nodes in different files
-            List<T> tuples = new ArrayList<>();
-            Node bnode1 = NodeFactory.createAnon();
-            Node bnode2 = NodeFactory.createAnon();
-            Node pred = NodeFactory.createURI("http://example.org/predicate");
-
-            tuples.add(createTuple(bnode1, pred, NodeFactory.createLiteral("first")));
-            writeTuples(a, tuples);
-
-            tuples.clear();
-            tuples.add(createTuple(bnode2, pred, NodeFactory.createLiteral("second")));
-            writeTuples(b, tuples);
-
-            // Set up fake job which will process the two files
-            Configuration config = new Configuration(true);
-            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
-            Job job = Job.getInstance(config);
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()), new Path(b.getAbsolutePath()));
-            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            List<InputSplit> splits = inputFormat.getSplits(context);
-            Assert.assertEquals(2, splits.size());
-
-            // Prepare the output writing - putting all output to a single file
-            OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
-            TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(), createAttemptID(
-                    1, 2, 1));
-            RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
-
-            for (InputSplit split : splits) {
-                // Initialize the input reading
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        createAttemptID(1, 1, 1));
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                // Copy the input to the output - all triples go to a single
-                // output
-                while (reader.nextKeyValue()) {
-                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
-                }
-            }
-            writer.close(outputTaskContext);
-
-            // Promote outputs from temporary status
-            promoteInputs(intermediateOutputDir);
-
-            // Now we need to create a subsequent job that reads the
-            // intermediate outputs
-            // The Blank nodes should have been given separate identities so we
-            // should not be conflating them, this is the opposite problem to
-            // that described in JENA-820
-            System.out.println(intermediateOutputDir.getAbsolutePath());
-            job = Job.getInstance(config);
-            inputFormat = createIntermediateInputFormat();
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            splits = inputFormat.getSplits(context);
-            Assert.assertEquals(1, splits.size());
-
-            // Expect to end up with a single blank node
-            Set<Node> nodes = new HashSet<Node>();
-            for (InputSplit split : splits) {
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        new TaskAttemptID());
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                while (reader.nextKeyValue()) {
-                    nodes.add(getSubject(reader.getCurrentValue().get()));
-                }
-            }
-            // Nodes must not have converged
-            Assert.assertEquals(2, nodes.size());
-
-        } finally {
-            a.delete();
-            b.delete();
-            deleteDirectory(intermediateOutputDir);
-        }
-    }
-
-    /**
-     * Test that starts with two blank nodes in two different files and checks
-     * that writing them to a single file does not conflate them
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void blank_node_identity_02() throws IOException, InterruptedException {
-        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
-        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
-        
-        // Temporary files
-        File a = File.createTempFile("bnode_identity", getInitialInputExtension());
-        File b = File.createTempFile("bnode_identity", getInitialInputExtension());
-        File intermediateOutputDir = Files.createTempDirectory("bnode_identity", new FileAttribute[0]).toFile();
-
-        try {
-            // Prepare the input data
-            // Same blank node but in different files so must be treated as
-            // different blank nodes and not converge
-            List<T> tuples = new ArrayList<>();
-            Node bnode = NodeFactory.createAnon();
-            Node pred = NodeFactory.createURI("http://example.org/predicate");
-
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
-            writeTuples(a, tuples);
-
-            tuples.clear();
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
-            writeTuples(b, tuples);
-
-            // Set up fake job which will process the two files
-            Configuration config = new Configuration(true);
-            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
-            Job job = Job.getInstance(config);
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()), new Path(b.getAbsolutePath()));
-            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            List<InputSplit> splits = inputFormat.getSplits(context);
-            Assert.assertEquals(2, splits.size());
-
-            // Prepare the output writing - putting all output to a single file
-            OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
-            TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(), createAttemptID(
-                    1, 2, 1));
-            RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
-
-            for (InputSplit split : splits) {
-                // Initialize the input reading
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        createAttemptID(1, 1, 1));
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                // Copy the input to the output - all triples go to a single
-                // output
-                while (reader.nextKeyValue()) {
-                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
-                }
-            }
-            writer.close(outputTaskContext);
-
-            // Promote outputs from temporary status
-            promoteInputs(intermediateOutputDir);
-
-            // Now we need to create a subsequent job that reads the
-            // intermediate outputs
-            // The Blank nodes should have been given separate identities so we
-            // should not be conflating them, this is the opposite problem to
-            // that described in JENA-820
-            System.out.println(intermediateOutputDir.getAbsolutePath());
-            job = Job.getInstance(config);
-            inputFormat = createIntermediateInputFormat();
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            splits = inputFormat.getSplits(context);
-            Assert.assertEquals(1, splits.size());
-
-            // Expect to end up with a single blank node
-            Set<Node> nodes = new HashSet<Node>();
-            for (InputSplit split : splits) {
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        new TaskAttemptID());
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                while (reader.nextKeyValue()) {
-                    nodes.add(getSubject(reader.getCurrentValue().get()));
-                }
-            }
-            // Nodes must not diverge
-            Assert.assertEquals(2, nodes.size());
-
-        } finally {
-            a.delete();
-            b.delete();
-            deleteDirectory(intermediateOutputDir);
-        }
-    }
-
-    private TaskAttemptID createAttemptID(int jobID, int taskID, int id) {
-        return new TaskAttemptID("outputTest", jobID, TaskType.MAP, taskID, 1);
-    }
-
-    private void promoteInputs(File baseDir) throws IOException {
-        for (File f : baseDir.listFiles()) {
-            if (f.isDirectory()) {
-                promoteInputs(baseDir, f);
-            }
-        }
-    }
-
-    private void promoteInputs(File targetDir, File dir) throws IOException {
-        java.nio.file.Path target = Paths.get(targetDir.toURI());
-        for (File f : dir.listFiles()) {
-            if (f.isDirectory()) {
-                promoteInputs(targetDir, f);
-            } else {
-                LOGGER.debug("Moving {} to {}", f.getAbsolutePath(), target.resolve(f.getName()));
-                Files.move(Paths.get(f.toURI()), target.resolve(f.getName()), StandardCopyOption.REPLACE_EXISTING);
-            }
-        }
-
-        // Remove defunct sub-directory
-        dir.delete();
-    }
-
-    private void deleteDirectory(File dir) throws IOException {
-        for (File f : dir.listFiles()) {
-            if (f.isFile())
-                f.delete();
-            if (f.isDirectory())
-                deleteDirectory(f);
-        }
-        dir.delete();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
deleted file mode 100644
index bbd6742..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.bnodes;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.util.List;
-
-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.Node;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.graph.GraphFactory;
-
-/**
- *
- */
-public abstract class AbstractTripleBlankNodeTests extends AbstractBlankNodeTests<Triple, TripleWritable> {
-    
-    /**
-     * Gets the language to use
-     * 
-     * @return Language
-     */
-    protected abstract Lang getLanguage();
-
-    @Override
-    protected Triple createTuple(Node s, Node p, Node o) {
-        return new Triple(s, p, o);
-    }
-
-    @Override
-    protected void writeTuples(File f, List<Triple> tuples) throws FileNotFoundException {
-        Graph g = GraphFactory.createGraphMem();
-        for (Triple t : tuples) {
-            g.add(t);
-        }
-        RDFDataMgr.write(new FileOutputStream(f), g, getLanguage());
-    }
-
-    @Override
-    protected Node getSubject(Triple value) {
-        return value.getSubject();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
deleted file mode 100644
index f234127..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDTripleInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.jsonld.JsonLDTripleOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link RdfJsonInputFormat}
- */
-public class JsonLdTripleBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.JSONLD;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".jsonld";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new JsonLDTripleInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new JsonLDTripleOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new JsonLDTripleInputFormat();
-    }
-
-    @Override
-    protected boolean respectsParserProfile() {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
deleted file mode 100644
index 4c350c7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link NTriplesInputFormat}
- */
-public class NTriplesBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".nt";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new NTriplesInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new NTriplesOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new NTriplesInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
deleted file mode 100644
index 2be1e0e..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.rdfjson.RdfJsonOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link RdfJsonInputFormat}
- */
-public class RdfJsonBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.RDFJSON;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".rj";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new RdfJsonInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new RdfJsonOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new RdfJsonInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
deleted file mode 100644
index d6f32a2..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftTripleInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.thrift.ThriftTripleOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Tests blank node divergence when using the {@link RdfThriftInputFormat}
- */
-public class RdfThriftBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new ThriftTripleInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new ThriftTripleOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new ThriftTripleInputFormat();
-    }
-
-    @Override
-    protected boolean respectsParserProfile() {
-        return false;
-    }
-    
-    @Override
-    protected boolean preservesBlankNodeIdentity() {
-        return true;
-    }
-}


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

Posted by rv...@apache.org.
Further rebranding to Elephas


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

Branch: refs/heads/hadoop-rdf
Commit: 49c4cffe6ab67a810955db286633ced5798cfc3f
Parents: a6c0fef
Author: Rob Vesse <rv...@apache.org>
Authored: Mon Jan 5 15:05:55 2015 +0000
Committer: Rob Vesse <rv...@apache.org>
Committed: Mon Jan 5 15:05:55 2015 +0000

----------------------------------------------------------------------
 jena-elephas/LICENSE                            | 176 +++++
 jena-elephas/NOTICE                             |   5 +
 jena-elephas/jena-elephas-common/pom.xml        |  54 ++
 .../rdf/types/AbstractNodeTupleWritable.java    | 193 +++++
 .../rdf/types/CharacteristicSetWritable.java    | 298 ++++++++
 .../rdf/types/CharacteristicWritable.java       | 160 ++++
 .../hadoop/rdf/types/NodeTupleWritable.java     |  80 ++
 .../jena/hadoop/rdf/types/NodeWritable.java     | 188 +++++
 .../jena/hadoop/rdf/types/QuadWritable.java     | 136 ++++
 .../jena/hadoop/rdf/types/TripleWritable.java   | 138 ++++
 .../comparators/SimpleBinaryComparator.java     |  34 +
 .../rdf/types/converters/ThriftConverter.java   | 147 ++++
 .../rdf/io/types/CharacteristicTests.java       | 210 +++++
 .../jena/hadoop/rdf/io/types/RdfTypesTest.java  | 406 ++++++++++
 jena-elephas/jena-elephas-io/pom.xml            |  67 ++
 .../jena/hadoop/rdf/io/HadoopIOConstants.java   |  49 ++
 .../jena/hadoop/rdf/io/RdfIOConstants.java      |  81 ++
 .../io/input/AbstractNLineFileInputFormat.java  |  70 ++
 .../io/input/AbstractWholeFileInputFormat.java  |  42 +
 .../hadoop/rdf/io/input/QuadsInputFormat.java   |  46 ++
 .../hadoop/rdf/io/input/TriplesInputFormat.java |  42 +
 .../rdf/io/input/TriplesOrQuadsInputFormat.java |  47 ++
 .../io/input/jsonld/JsonLDQuadInputFormat.java  |  39 +
 .../input/jsonld/JsonLDTripleInputFormat.java   |  39 +
 .../input/nquads/BlockedNQuadsInputFormat.java  |  53 ++
 .../rdf/io/input/nquads/NQuadsInputFormat.java  |  46 ++
 .../nquads/WholeFileNQuadsInputFormat.java      |  51 ++
 .../ntriples/BlockedNTriplesInputFormat.java    |  53 ++
 .../io/input/ntriples/NTriplesInputFormat.java  |  46 ++
 .../ntriples/WholeFileNTriplesInputFormat.java  |  51 ++
 .../io/input/rdfjson/RdfJsonInputFormat.java    |  46 ++
 .../rdf/io/input/rdfxml/RdfXmlInputFormat.java  |  46 ++
 .../AbstractBlockBasedNodeTupleReader.java      | 344 +++++++++
 .../readers/AbstractBlockBasedQuadReader.java   |  51 ++
 .../readers/AbstractBlockBasedTripleReader.java |  51 ++
 .../AbstractLineBasedNodeTupleReader.java       | 265 +++++++
 .../readers/AbstractLineBasedQuadReader.java    |  50 ++
 .../readers/AbstractLineBasedTripleReader.java  |  51 ++
 .../rdf/io/input/readers/AbstractRdfReader.java | 108 +++
 .../AbstractWholeFileNodeTupleReader.java       | 328 ++++++++
 .../readers/AbstractWholeFileQuadReader.java    |  51 ++
 .../readers/AbstractWholeFileTripleReader.java  |  51 ++
 .../rdf/io/input/readers/QuadsReader.java       |  49 ++
 .../io/input/readers/TriplesOrQuadsReader.java  |  72 ++
 .../rdf/io/input/readers/TriplesReader.java     |  49 ++
 .../io/input/readers/TriplesToQuadsReader.java  | 102 +++
 .../input/readers/jsonld/JsonLDQuadReader.java  |  32 +
 .../readers/jsonld/JsonLDTripleReader.java      |  30 +
 .../readers/nquads/BlockedNQuadsReader.java     |  45 ++
 .../io/input/readers/nquads/NQuadsReader.java   |  49 ++
 .../readers/nquads/WholeFileNQuadsReader.java   |  42 +
 .../readers/ntriples/BlockedNTriplesReader.java |  45 ++
 .../input/readers/ntriples/NTriplesReader.java  |  48 ++
 .../ntriples/WholeFileNTriplesReader.java       |  42 +
 .../io/input/readers/rdfjson/RdfJsonReader.java |  37 +
 .../io/input/readers/rdfxml/RdfXmlReader.java   |  37 +
 .../input/readers/thrift/ThriftQuadReader.java  |  32 +
 .../readers/thrift/ThriftTripleReader.java      |  30 +
 .../rdf/io/input/readers/trig/TriGReader.java   |  37 +
 .../rdf/io/input/readers/trix/TriXReader.java   |  37 +
 .../io/input/readers/turtle/TurtleReader.java   |  37 +
 .../io/input/thrift/ThriftQuadInputFormat.java  |  39 +
 .../input/thrift/ThriftTripleInputFormat.java   |  39 +
 .../rdf/io/input/trig/TriGInputFormat.java      |  46 ++
 .../rdf/io/input/trix/TriXInputFormat.java      |  42 +
 .../rdf/io/input/turtle/TurtleInputFormat.java  |  46 ++
 .../rdf/io/input/util/BlockInputStream.java     |  94 +++
 .../hadoop/rdf/io/input/util/RdfIOUtils.java    | 101 +++
 .../rdf/io/input/util/TrackableInputStream.java |  38 +
 .../rdf/io/input/util/TrackedInputStream.java   | 124 +++
 .../io/input/util/TrackedPipedQuadsStream.java  |  55 ++
 .../io/input/util/TrackedPipedRDFStream.java    |  64 ++
 .../input/util/TrackedPipedTriplesStream.java   |  56 ++
 .../AbstractBatchedNodeTupleOutputFormat.java   |  55 ++
 .../rdf/io/output/AbstractNodeOutputFormat.java |  94 +++
 .../output/AbstractNodeTupleOutputFormat.java   | 109 +++
 .../AbstractStreamRdfNodeTupleOutputFormat.java |  73 ++
 .../hadoop/rdf/io/output/QuadsOutputFormat.java |  64 ++
 .../io/output/TriplesOrQuadsOutputFormat.java   |  74 ++
 .../rdf/io/output/TriplesOutputFormat.java      |  61 ++
 .../output/jsonld/JsonLDQuadOutputFormat.java   |  44 ++
 .../output/jsonld/JsonLDTripleOutputFormat.java |  44 ++
 .../io/output/nquads/NQuadsOutputFormat.java    |  52 ++
 .../ntriples/NTriplesNodeOutputFormat.java      |  45 ++
 .../output/ntriples/NTriplesOutputFormat.java   |  52 ++
 .../io/output/rdfjson/RdfJsonOutputFormat.java  |  52 ++
 .../io/output/rdfxml/RdfXmlOutputFormat.java    |  52 ++
 .../output/thrift/ThriftQuadOutputFormat.java   |  51 ++
 .../output/thrift/ThriftTripleOutputFormat.java |  52 ++
 .../io/output/trig/BatchedTriGOutputFormat.java |  54 ++
 .../rdf/io/output/trig/TriGOutputFormat.java    |  58 ++
 .../rdf/io/output/trix/TriXOutputFormat.java    |  57 ++
 .../turtle/BatchedTurtleOutputFormat.java       |  50 ++
 .../io/output/turtle/TurtleOutputFormat.java    |  56 ++
 .../writers/AbstractBatchedNodeTupleWriter.java | 113 +++
 .../writers/AbstractBatchedQuadWriter.java      |  80 ++
 .../writers/AbstractBatchedTripleWriter.java    |  68 ++
 .../AbstractLineBasedNodeTupleWriter.java       | 152 ++++
 .../writers/AbstractLineBasedQuadWriter.java    |  71 ++
 .../writers/AbstractLineBasedTripleWriter.java  |  68 ++
 .../io/output/writers/AbstractNodeWriter.java   | 192 +++++
 .../AbstractStreamRdfNodeTupleWriter.java       |  71 ++
 .../AbstractWholeFileNodeTupleWriter.java       |  96 +++
 .../writers/AbstractWholeFileQuadWriter.java    |  66 ++
 .../writers/AbstractWholeFileTripleWriter.java  |  65 ++
 .../io/output/writers/QuadsToTriplesWriter.java |  59 ++
 .../io/output/writers/StreamRdfQuadWriter.java  |  45 ++
 .../output/writers/StreamRdfTripleWriter.java   |  44 ++
 .../output/writers/jsonld/JsonLDQuadWriter.java |  38 +
 .../writers/jsonld/JsonLDTripleWriter.java      |  38 +
 .../io/output/writers/nquads/NQuadsWriter.java  |  57 ++
 .../writers/ntriples/NTriplesNodeWriter.java    |  59 ++
 .../output/writers/ntriples/NTriplesWriter.java |  58 ++
 .../output/writers/rdfjson/RdfJsonWriter.java   |  51 ++
 .../io/output/writers/rdfxml/RdfXmlWriter.java  |  51 ++
 .../output/writers/thrift/ThriftQuadWriter.java |  38 +
 .../writers/thrift/ThriftTripleWriter.java      |  38 +
 .../output/writers/trig/BatchedTriGWriter.java  |  52 ++
 .../writers/turtle/BatchedTurtleWriter.java     |  54 ++
 .../rdf/io/registry/HadoopRdfIORegistry.java    | 310 ++++++++
 .../hadoop/rdf/io/registry/ReaderFactory.java   |  83 ++
 .../hadoop/rdf/io/registry/WriterFactory.java   |  96 +++
 .../readers/AbstractQuadsOnlyReaderFactory.java |  83 ++
 .../registry/readers/AbstractReaderFactory.java |  80 ++
 .../AbstractTriplesOnlyReaderFactory.java       |  83 ++
 .../registry/readers/JsonLDReaderFactory.java   |  49 ++
 .../registry/readers/NQuadsReaderFactory.java   |  42 +
 .../registry/readers/NTriplesReaderFactory.java |  38 +
 .../registry/readers/RdfJsonReaderFactory.java  |  41 +
 .../registry/readers/RdfXmlReaderFactory.java   |  40 +
 .../registry/readers/ThriftReaderFactory.java   |  49 ++
 .../io/registry/readers/TriGReaderFactory.java  |  42 +
 .../io/registry/readers/TriXReaderFactory.java  |  41 +
 .../registry/readers/TurtleReaderFactory.java   |  40 +
 .../writers/AbstractQuadsOnlyWriterFactory.java |  86 +++
 .../AbstractTriplesOnlyWriterFactory.java       |  85 +++
 .../registry/writers/AbstractWriterFactory.java |  82 ++
 .../registry/writers/JsonLDWriterFactory.java   |  52 ++
 .../registry/writers/NQuadsWriterFactory.java   |  44 ++
 .../registry/writers/NTriplesWriterFactory.java |  44 ++
 .../registry/writers/RdfJsonWriterFactory.java  |  43 ++
 .../registry/writers/RdfXmlWriterFactory.java   |  44 ++
 .../registry/writers/ThriftWriterFactory.java   |  57 ++
 .../io/registry/writers/TriGWriterFactory.java  |  45 ++
 .../io/registry/writers/TriXWriterFactory.java  |  47 ++
 .../registry/writers/TurtleWriterFactory.java   |  45 ++
 ...he.jena.hadoop.rdf.io.registry.ReaderFactory |  10 +
 ...he.jena.hadoop.rdf.io.registry.WriterFactory |  10 +
 .../rdf/io/RdfTriplesInputTestMapper.java       |  47 ++
 .../AbstractBlockedQuadInputFormatTests.java    |  33 +
 .../AbstractBlockedTripleInputFormatTests.java  |  33 +
 .../AbstractNodeTupleInputFormatTests.java      | 612 +++++++++++++++
 .../io/input/AbstractQuadsInputFormatTests.java |  70 ++
 .../input/AbstractTriplesInputFormatTests.java  |  72 ++
 .../AbstractWholeFileQuadInputFormatTests.java  | 115 +++
 ...AbstractWholeFileTripleInputFormatTests.java | 108 +++
 .../io/input/bnodes/AbstractBlankNodeTests.java | 636 ++++++++++++++++
 .../bnodes/AbstractTripleBlankNodeTests.java    |  65 ++
 .../input/bnodes/JsonLdTripleBlankNodeTest.java |  63 ++
 .../io/input/bnodes/NTriplesBlankNodeTest.java  |  58 ++
 .../io/input/bnodes/RdfJsonBlankNodeTest.java   |  58 ++
 .../io/input/bnodes/RdfThriftBlankNodeTest.java |  68 ++
 .../io/input/bnodes/RdfXmlBlankNodeTest.java    |  62 ++
 .../io/input/bnodes/TurtleBlankNodeTest.java    |  58 ++
 ...ractCompressedNodeTupleInputFormatTests.java |  74 ++
 ...AbstractCompressedQuadsInputFormatTests.java |  71 ++
 ...stractCompressedTriplesInputFormatTests.java |  71 ++
 ...CompressedWholeFileQuadInputFormatTests.java | 150 ++++
 ...mpressedWholeFileTripleInputFormatTests.java | 144 ++++
 ...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 +
 ...bstractCompressedNQuadsInputFormatTests.java |  68 ++
 ...mpressedWholeFileNQuadsInputFormatTests.java |  75 ++
 .../nquads/BZipppedNQuadsInputTest.java         |  38 +
 .../BZipppedWholeFileNQuadsInputTest.java       |  37 +
 .../nquads/DeflatedNQuadsInputTest.java         |  37 +
 .../DeflatedWholeFileNQuadsInputTest.java       |  37 +
 .../nquads/GZippedNQuadsInputTest.java          |  38 +
 .../nquads/GZippedWholeFileNQuadsInputTest.java |  38 +
 ...mpressedBlockedNTriplesInputFormatTests.java |  53 ++
 ...tractCompressedNTriplesInputFormatTests.java |  68 ++
 ...ressedWholeFileNTriplesInputFormatTests.java |  75 ++
 .../ntriples/BZippedBlockedNTriplesInput.java   |  37 +
 .../ntriples/BZippedNTriplesInputTest.java      |  38 +
 .../BZippedWholeFileNTriplesInputTest.java      |  38 +
 .../ntriples/DeflatedBlockedNTriplesInput.java  |  37 +
 .../ntriples/DeflatedNTriplesInputTest.java     |  38 +
 .../DeflatedWholeFileNTriplesInputTest.java     |  38 +
 .../ntriples/GZippedBlockedNTriplesInput.java   |  37 +
 .../ntriples/GZippedNTriplesInputTest.java      |  41 +
 .../GZippedWholeFileNTriplesInputTest.java      |  38 +
 ...stractCompressedRdfJsonInputFormatTests.java |  74 ++
 .../rdfjson/BZippedRdfJsonInputTest.java        |  37 +
 .../rdfjson/DeflatedRdfJsonInputTest.java       |  37 +
 .../rdfjson/GZippedRdfJsonInputTest.java        |  37 +
 ...bstractCompressedRdfXmlInputFormatTests.java |  75 ++
 .../rdfxml/BZippedRdfXmlInputTest.java          |  37 +
 .../rdfxml/DeflatedRdfXmlInputTest.java         |  37 +
 .../rdfxml/GZippedRdfXmlInputTest.java          |  37 +
 ...actCompressedThriftQuadInputFormatTests.java |  72 ++
 ...tCompressedThriftTripleInputFormatTests.java |  72 ++
 .../thrift/BZippedThriftQuadInputTest.java      |  34 +
 .../thrift/BZippedThriftTripleInputTest.java    |  34 +
 .../thrift/DeflatedThriftQuadInputTest.java     |  34 +
 .../thrift/DeflatedThriftTripleInputTest.java   |  34 +
 .../thrift/GZippedThriftQuadInputTest.java      |  34 +
 .../thrift/GZippedThriftTripleInputTest.java    |  34 +
 .../AbstractCompressedTriGInputFormatTests.java |  72 ++
 .../compressed/trig/BZippedTriGInputTest.java   |  37 +
 .../compressed/trig/DeflatedTriGInputTest.java  |  37 +
 .../compressed/trig/GZippedTriGInputTest.java   |  37 +
 .../AbstractCompressedTriXInputFormatTests.java |  72 ++
 .../compressed/trix/BZippedTriXInputTest.java   |  35 +
 .../compressed/trix/DeflatedTriXInputTest.java  |  35 +
 .../compressed/trix/GZippedTriXInputTest.java   |  35 +
 ...bstractCompressedTurtleInputFormatTests.java |  75 ++
 .../turtle/BZippedTurtleInputTest.java          |  37 +
 .../turtle/DeflatedTurtleInputTest.java         |  37 +
 .../turtle/GZippedTurtleInputTest.java          |  37 +
 .../io/input/jsonld/JsonLDQuadInputTest.java    |  50 ++
 .../io/input/jsonld/JsonLDTripleInputTest.java  |  50 ++
 .../io/input/nquads/BlockedNQuadsInputTest.java |  51 ++
 .../rdf/io/input/nquads/NQuadsInputTest.java    |  44 ++
 .../input/nquads/WholeFileNQuadsInputTest.java  |  51 ++
 .../ntriples/BlockedNTriplesInputTest.java      |  50 ++
 .../io/input/ntriples/NTriplesInputTest.java    |  44 ++
 .../ntriples/WholeFileNTriplesInputTest.java    |  52 ++
 .../rdf/io/input/rdfjson/RdfJsonInputTest.java  |  51 ++
 .../rdf/io/input/rdfxml/RdfXmlInputTest.java    |  51 ++
 .../io/input/thrift/ThriftQuadInputTest.java    |  51 ++
 .../io/input/thrift/ThriftTripleInputTest.java  |  51 ++
 .../hadoop/rdf/io/input/trig/TriGInputTest.java |  50 ++
 .../hadoop/rdf/io/input/trix/TriXInputTest.java |  50 ++
 .../rdf/io/input/turtle/TurtleInputTest.java    |  50 ++
 .../util/AbstractTrackableInputStreamTests.java | 701 +++++++++++++++++
 .../rdf/io/input/util/BlockInputStreamTest.java | 240 ++++++
 .../io/input/util/TrackedInputStreamTest.java   |  39 +
 .../AbstractNodeTupleOutputFormatTests.java     | 255 +++++++
 .../output/AbstractQuadOutputFormatTests.java   |  51 ++
 .../output/AbstractTripleOutputFormatTests.java |  47 ++
 .../io/output/jsonld/JsonLdQuadOutputTest.java  |  47 ++
 .../output/jsonld/JsonLdTripleOutputTest.java   |  47 ++
 .../rdf/io/output/nquads/NQuadsOutputTest.java  |  51 ++
 .../io/output/ntriples/NTriplesOutputTest.java  |  51 ++
 .../io/output/rdfjson/RdfJsonOutputTest.java    |  51 ++
 .../rdf/io/output/rdfxml/RdfXmlOutputTest.java  |  51 ++
 .../io/output/thrift/ThriftQuadOutputTest.java  |  48 ++
 .../output/thrift/ThriftTripleOutputTest.java   |  48 ++
 .../io/output/trig/BatchedTriGOutputTest.java   |  92 +++
 .../io/output/trig/StreamedTriGOutputTest.java  |  92 +++
 .../output/trig/TriGBlankNodeOutputTests.java   | 120 +++
 .../rdf/io/output/trix/TriXOutputTest.java      |  47 ++
 .../output/turtle/BatchedTurtleOutputTest.java  |  92 +++
 .../output/turtle/StreamedTurtleOutputTest.java |  92 +++
 .../turtle/TurtleBlankNodeOutputTests.java      | 118 +++
 .../io/registry/TestHadoopRdfIORegistry.java    | 186 +++++
 jena-elephas/jena-elephas-mapreduce/pom.xml     |  87 +++
 .../jena/hadoop/rdf/mapreduce/KeyMapper.java    |  54 ++
 .../hadoop/rdf/mapreduce/KeyPlusNullMapper.java |  55 ++
 .../jena/hadoop/rdf/mapreduce/KeyReducer.java   |  39 +
 .../hadoop/rdf/mapreduce/NullPlusKeyMapper.java |  55 ++
 .../rdf/mapreduce/NullPlusKeyReducer.java       |  59 ++
 .../rdf/mapreduce/NullPlusValueMapper.java      |  55 ++
 .../rdf/mapreduce/NullPlusValueReducer.java     |  64 ++
 .../rdf/mapreduce/RdfMapReduceConstants.java    |  67 ++
 .../jena/hadoop/rdf/mapreduce/SwapMapper.java   |  55 ++
 .../jena/hadoop/rdf/mapreduce/SwapReducer.java  |  43 ++
 .../hadoop/rdf/mapreduce/TextCountReducer.java  |  49 ++
 .../jena/hadoop/rdf/mapreduce/ValueMapper.java  |  54 ++
 .../rdf/mapreduce/ValuePlusNullMapper.java      |  55 ++
 .../jena/hadoop/rdf/mapreduce/ValueReducer.java |  44 ++
 ...tractCharacteristicSetGeneratingReducer.java | 179 +++++
 .../CharacteristicSetReducer.java               |  68 ++
 .../QuadCharacteristicSetGeneratingReducer.java |  39 +
 ...ripleCharacteristicSetGeneratingReducer.java |  40 +
 .../count/AbstractNodeTupleNodeCountMapper.java |  66 ++
 .../rdf/mapreduce/count/NodeCountReducer.java   |  50 ++
 .../mapreduce/count/QuadNodeCountMapper.java    |  44 ++
 .../mapreduce/count/TripleNodeCountMapper.java  |  42 +
 .../datatypes/QuadDataTypeCountMapper.java      |  56 ++
 .../datatypes/TripleDataTypeCountMapper.java    |  56 ++
 .../AbstractNodeTupleNamespaceCountMapper.java  | 135 ++++
 .../namespaces/QuadNamespaceCountMapper.java    |  44 ++
 .../namespaces/TripleNamespaceCountMapper.java  |  44 ++
 .../count/positional/QuadObjectCountMapper.java |  42 +
 .../positional/QuadPredicateCountMapper.java    |  42 +
 .../positional/QuadSubjectCountMapper.java      |  41 +
 .../positional/TripleObjectCountMapper.java     |  41 +
 .../positional/TriplePredicateCountMapper.java  |  42 +
 .../positional/TripleSubjectCountMapper.java    |  41 +
 .../filter/AbstractNodeTupleFilterMapper.java   |  76 ++
 .../filter/AbstractQuadFilterMapper.java        |  35 +
 .../filter/AbstractTripleFilterMapper.java      |  35 +
 .../filter/GroundQuadFilterMapper.java          |  47 ++
 .../filter/GroundTripleFilterMapper.java        |  47 ++
 .../mapreduce/filter/ValidQuadFilterMapper.java |  48 ++
 .../filter/ValidTripleFilterMapper.java         |  47 ++
 .../AbstractQuadFilterByPositionMapper.java     | 171 +++++
 .../AbstractTripleFilterByPositionMapper.java   | 141 ++++
 .../positional/QuadFilterByGraphUriMapper.java  |  76 ++
 .../positional/QuadFilterByObjectUriMapper.java |  76 ++
 .../positional/QuadFilterByPredicateMapper.java |  76 ++
 .../QuadFilterBySubjectUriMapper.java           |  76 ++
 .../TripleFilterByObjectUriMapper.java          |  71 ++
 .../TripleFilterByPredicateUriMapper.java       |  71 ++
 .../TripleFilterBySubjectUriMapper.java         |  71 ++
 .../group/AbstractNodeTupleGroupingMapper.java  |  60 ++
 .../group/AbstractQuadGroupingMapper.java       |  50 ++
 .../group/AbstractTripleGroupingMapper.java     |  44 ++
 .../mapreduce/group/QuadGroupByGraphMapper.java |  39 +
 .../group/QuadGroupByObjectMapper.java          |  39 +
 .../group/QuadGroupByPredicateMapper.java       |  39 +
 .../group/QuadGroupBySubjectMapper.java         |  39 +
 .../group/TripleGroupByObjectMapper.java        |  41 +
 .../group/TripleGroupByPredicateMapper.java     |  41 +
 .../group/TripleGroupBySubjectMapper.java       |  41 +
 .../AbstractNodeTupleSplitToNodesMapper.java    |  60 ++
 .../AbstractNodeTupleSplitWithNodesMapper.java  |  60 ++
 .../mapreduce/split/QuadSplitToNodesMapper.java |  43 ++
 .../split/QuadSplitWithNodesMapper.java         |  43 ++
 .../split/TripleSplitToNodesMapper.java         |  41 +
 .../split/TripleSplitWithNodesMapper.java       |  41 +
 .../transform/AbstractTriplesToQuadsMapper.java |  60 ++
 .../transform/QuadsToTriplesMapper.java         |  46 ++
 .../TriplesToQuadsBySubjectMapper.java          |  40 +
 .../TriplesToQuadsConstantGraphMapper.java      |  75 ++
 .../rdf/mapreduce/AbstractMapReduceTests.java   |  69 ++
 .../rdf/mapreduce/AbstractMapperTests.java      |  69 ++
 .../rdf/mapreduce/TestDistinctTriples.java      | 129 ++++
 ...CharacteristicSetGeneratingReducerTests.java | 185 +++++
 .../CharacteristicSetReducerTest.java           | 192 +++++
 ...eCharacteristicSetGeneratingReducerTest.java |  59 ++
 .../AbstractNodeTupleNodeCountReducedTests.java | 149 ++++
 .../count/AbstractNodeTupleNodeCountTests.java  | 138 ++++
 .../count/QuadNodeCountMapReduceTest.java       |  67 ++
 .../count/QuadNodeCountMapperTest.java          |  59 ++
 .../count/TripleNodeCountMapReduceTest.java     |  66 ++
 .../count/TripleNodeCountMapperTest.java        |  58 ++
 .../filter/AbstractNodeTupleFilterTests.java    | 146 ++++
 .../filter/AbstractQuadValidityFilterTests.java |  86 +++
 .../AbstractTripleValidityFilterTests.java      |  73 ++
 .../TripleFilterByNoPredicateMapperTest.java    |  49 ++
 .../TripleFilterByPredicateMapperTest.java      |  80 ++
 ...leInvertedFilterByNoPredicateMapperTest.java |  54 ++
 ...ipleInvertedFilterByPredicateMapperTest.java |  87 +++
 .../filter/ValidQuadFilterMapperTest.java       |  40 +
 .../filter/ValidTripleFilterMapperTest.java     |  40 +
 .../group/AbstractNodeTupleGroupingTests.java   | 114 +++
 .../group/AbstractQuadGroupingTests.java        |  43 ++
 .../group/AbstractTripleGroupingTests.java      |  41 +
 .../group/QuadGroupByGraphMapperTest.java       |  46 ++
 .../group/QuadGroupByObjectMapperTest.java      |  46 ++
 .../group/QuadGroupByPredicateMapperTest.java   |  46 ++
 .../group/QuadGroupBySubjectMapperTest.java     |  46 ++
 .../group/TripleGroupByObjectMapperTest.java    |  46 ++
 .../group/TripleGroupByPredicateMapperTest.java |  46 ++
 .../group/TripleGroupBySubjectMapperTest.java   |  46 ++
 .../AbstractNodeTupleSplitToNodesTests.java     | 116 +++
 .../AbstractNodeTupleSplitWithNodesTests.java   | 116 +++
 .../split/AbstractQuadSplitToNodesTests.java    |  53 ++
 .../split/AbstractQuadSplitWithNodesTests.java  |  53 ++
 .../split/AbstractTripleSplitToNodesTests.java  |  52 ++
 .../AbstractTripleSplitWithNodesTests.java      |  52 ++
 .../split/QuadSplitToNodesMapperTest.java       |  41 +
 .../split/QuadSplitWithNodesMapperTest.java     |  41 +
 .../split/TripleSplitToNodesMapperTest.java     |  41 +
 .../split/TripleSplitWithNodesMapperTest.java   |  42 +
 .../transform/QuadsToTriplesMapperTest.java     | 113 +++
 .../TriplesToQuadsBySubjectMapperTest.java      | 113 +++
 .../TriplesToQuadsConstantGraphMapperTest.java  | 113 +++
 jena-elephas/jena-elephas-stats/hadoop-job.xml  |  46 ++
 jena-elephas/jena-elephas-stats/pom.xml         | 103 +++
 .../apache/jena/hadoop/rdf/stats/RdfStats.java  | 405 ++++++++++
 .../jena/hadoop/rdf/stats/jobs/JobFactory.java  | 757 +++++++++++++++++++
 jena-elephas/pom.xml                            |  97 +++
 jena-hadoop-rdf/LICENSE                         | 176 -----
 jena-hadoop-rdf/NOTICE                          |   5 -
 jena-hadoop-rdf/hadoop-rdf-mapreduce/pom.xml    |  87 ---
 .../jena/hadoop/rdf/mapreduce/KeyMapper.java    |  54 --
 .../hadoop/rdf/mapreduce/KeyPlusNullMapper.java |  55 --
 .../jena/hadoop/rdf/mapreduce/KeyReducer.java   |  39 -
 .../hadoop/rdf/mapreduce/NullPlusKeyMapper.java |  55 --
 .../rdf/mapreduce/NullPlusKeyReducer.java       |  59 --
 .../rdf/mapreduce/NullPlusValueMapper.java      |  55 --
 .../rdf/mapreduce/NullPlusValueReducer.java     |  64 --
 .../rdf/mapreduce/RdfMapReduceConstants.java    |  67 --
 .../jena/hadoop/rdf/mapreduce/SwapMapper.java   |  55 --
 .../jena/hadoop/rdf/mapreduce/SwapReducer.java  |  43 --
 .../hadoop/rdf/mapreduce/TextCountReducer.java  |  49 --
 .../jena/hadoop/rdf/mapreduce/ValueMapper.java  |  54 --
 .../rdf/mapreduce/ValuePlusNullMapper.java      |  55 --
 .../jena/hadoop/rdf/mapreduce/ValueReducer.java |  44 --
 ...tractCharacteristicSetGeneratingReducer.java | 179 -----
 .../CharacteristicSetReducer.java               |  68 --
 .../QuadCharacteristicSetGeneratingReducer.java |  39 -
 ...ripleCharacteristicSetGeneratingReducer.java |  40 -
 .../count/AbstractNodeTupleNodeCountMapper.java |  66 --
 .../rdf/mapreduce/count/NodeCountReducer.java   |  50 --
 .../mapreduce/count/QuadNodeCountMapper.java    |  44 --
 .../mapreduce/count/TripleNodeCountMapper.java  |  42 -
 .../datatypes/QuadDataTypeCountMapper.java      |  56 --
 .../datatypes/TripleDataTypeCountMapper.java    |  56 --
 .../AbstractNodeTupleNamespaceCountMapper.java  | 135 ----
 .../namespaces/QuadNamespaceCountMapper.java    |  44 --
 .../namespaces/TripleNamespaceCountMapper.java  |  44 --
 .../count/positional/QuadObjectCountMapper.java |  42 -
 .../positional/QuadPredicateCountMapper.java    |  42 -
 .../positional/QuadSubjectCountMapper.java      |  41 -
 .../positional/TripleObjectCountMapper.java     |  41 -
 .../positional/TriplePredicateCountMapper.java  |  42 -
 .../positional/TripleSubjectCountMapper.java    |  41 -
 .../filter/AbstractNodeTupleFilterMapper.java   |  76 --
 .../filter/AbstractQuadFilterMapper.java        |  35 -
 .../filter/AbstractTripleFilterMapper.java      |  35 -
 .../filter/GroundQuadFilterMapper.java          |  47 --
 .../filter/GroundTripleFilterMapper.java        |  47 --
 .../mapreduce/filter/ValidQuadFilterMapper.java |  48 --
 .../filter/ValidTripleFilterMapper.java         |  47 --
 .../AbstractQuadFilterByPositionMapper.java     | 171 -----
 .../AbstractTripleFilterByPositionMapper.java   | 141 ----
 .../positional/QuadFilterByGraphUriMapper.java  |  76 --
 .../positional/QuadFilterByObjectUriMapper.java |  76 --
 .../positional/QuadFilterByPredicateMapper.java |  76 --
 .../QuadFilterBySubjectUriMapper.java           |  76 --
 .../TripleFilterByObjectUriMapper.java          |  71 --
 .../TripleFilterByPredicateUriMapper.java       |  71 --
 .../TripleFilterBySubjectUriMapper.java         |  71 --
 .../group/AbstractNodeTupleGroupingMapper.java  |  60 --
 .../group/AbstractQuadGroupingMapper.java       |  50 --
 .../group/AbstractTripleGroupingMapper.java     |  44 --
 .../mapreduce/group/QuadGroupByGraphMapper.java |  39 -
 .../group/QuadGroupByObjectMapper.java          |  39 -
 .../group/QuadGroupByPredicateMapper.java       |  39 -
 .../group/QuadGroupBySubjectMapper.java         |  39 -
 .../group/TripleGroupByObjectMapper.java        |  41 -
 .../group/TripleGroupByPredicateMapper.java     |  41 -
 .../group/TripleGroupBySubjectMapper.java       |  41 -
 .../AbstractNodeTupleSplitToNodesMapper.java    |  60 --
 .../AbstractNodeTupleSplitWithNodesMapper.java  |  60 --
 .../mapreduce/split/QuadSplitToNodesMapper.java |  43 --
 .../split/QuadSplitWithNodesMapper.java         |  43 --
 .../split/TripleSplitToNodesMapper.java         |  41 -
 .../split/TripleSplitWithNodesMapper.java       |  41 -
 .../transform/AbstractTriplesToQuadsMapper.java |  60 --
 .../transform/QuadsToTriplesMapper.java         |  46 --
 .../TriplesToQuadsBySubjectMapper.java          |  40 -
 .../TriplesToQuadsConstantGraphMapper.java      |  75 --
 .../rdf/mapreduce/AbstractMapReduceTests.java   |  69 --
 .../rdf/mapreduce/AbstractMapperTests.java      |  69 --
 .../rdf/mapreduce/TestDistinctTriples.java      | 129 ----
 ...CharacteristicSetGeneratingReducerTests.java | 185 -----
 .../CharacteristicSetReducerTest.java           | 192 -----
 ...eCharacteristicSetGeneratingReducerTest.java |  59 --
 .../AbstractNodeTupleNodeCountReducedTests.java | 149 ----
 .../count/AbstractNodeTupleNodeCountTests.java  | 138 ----
 .../count/QuadNodeCountMapReduceTest.java       |  67 --
 .../count/QuadNodeCountMapperTest.java          |  59 --
 .../count/TripleNodeCountMapReduceTest.java     |  66 --
 .../count/TripleNodeCountMapperTest.java        |  58 --
 .../filter/AbstractNodeTupleFilterTests.java    | 146 ----
 .../filter/AbstractQuadValidityFilterTests.java |  86 ---
 .../AbstractTripleValidityFilterTests.java      |  73 --
 .../TripleFilterByNoPredicateMapperTest.java    |  49 --
 .../TripleFilterByPredicateMapperTest.java      |  80 --
 ...leInvertedFilterByNoPredicateMapperTest.java |  54 --
 ...ipleInvertedFilterByPredicateMapperTest.java |  87 ---
 .../filter/ValidQuadFilterMapperTest.java       |  40 -
 .../filter/ValidTripleFilterMapperTest.java     |  40 -
 .../group/AbstractNodeTupleGroupingTests.java   | 114 ---
 .../group/AbstractQuadGroupingTests.java        |  43 --
 .../group/AbstractTripleGroupingTests.java      |  41 -
 .../group/QuadGroupByGraphMapperTest.java       |  46 --
 .../group/QuadGroupByObjectMapperTest.java      |  46 --
 .../group/QuadGroupByPredicateMapperTest.java   |  46 --
 .../group/QuadGroupBySubjectMapperTest.java     |  46 --
 .../group/TripleGroupByObjectMapperTest.java    |  46 --
 .../group/TripleGroupByPredicateMapperTest.java |  46 --
 .../group/TripleGroupBySubjectMapperTest.java   |  46 --
 .../AbstractNodeTupleSplitToNodesTests.java     | 116 ---
 .../AbstractNodeTupleSplitWithNodesTests.java   | 116 ---
 .../split/AbstractQuadSplitToNodesTests.java    |  53 --
 .../split/AbstractQuadSplitWithNodesTests.java  |  53 --
 .../split/AbstractTripleSplitToNodesTests.java  |  52 --
 .../AbstractTripleSplitWithNodesTests.java      |  52 --
 .../split/QuadSplitToNodesMapperTest.java       |  41 -
 .../split/QuadSplitWithNodesMapperTest.java     |  41 -
 .../split/TripleSplitToNodesMapperTest.java     |  41 -
 .../split/TripleSplitWithNodesMapperTest.java   |  42 -
 .../transform/QuadsToTriplesMapperTest.java     | 113 ---
 .../TriplesToQuadsBySubjectMapperTest.java      | 113 ---
 .../TriplesToQuadsConstantGraphMapperTest.java  | 113 ---
 jena-hadoop-rdf/jena-elephas-common/pom.xml     |  54 --
 .../rdf/types/AbstractNodeTupleWritable.java    | 193 -----
 .../rdf/types/CharacteristicSetWritable.java    | 298 --------
 .../rdf/types/CharacteristicWritable.java       | 160 ----
 .../hadoop/rdf/types/NodeTupleWritable.java     |  80 --
 .../jena/hadoop/rdf/types/NodeWritable.java     | 188 -----
 .../jena/hadoop/rdf/types/QuadWritable.java     | 136 ----
 .../jena/hadoop/rdf/types/TripleWritable.java   | 138 ----
 .../comparators/SimpleBinaryComparator.java     |  34 -
 .../rdf/types/converters/ThriftConverter.java   | 147 ----
 .../rdf/io/types/CharacteristicTests.java       | 210 -----
 .../jena/hadoop/rdf/io/types/RdfTypesTest.java  | 406 ----------
 jena-hadoop-rdf/jena-elephas-io/pom.xml         |  67 --
 .../jena/hadoop/rdf/io/HadoopIOConstants.java   |  49 --
 .../jena/hadoop/rdf/io/RdfIOConstants.java      |  81 --
 .../io/input/AbstractNLineFileInputFormat.java  |  70 --
 .../io/input/AbstractWholeFileInputFormat.java  |  42 -
 .../hadoop/rdf/io/input/QuadsInputFormat.java   |  46 --
 .../hadoop/rdf/io/input/TriplesInputFormat.java |  42 -
 .../rdf/io/input/TriplesOrQuadsInputFormat.java |  47 --
 .../io/input/jsonld/JsonLDQuadInputFormat.java  |  39 -
 .../input/jsonld/JsonLDTripleInputFormat.java   |  39 -
 .../input/nquads/BlockedNQuadsInputFormat.java  |  53 --
 .../rdf/io/input/nquads/NQuadsInputFormat.java  |  46 --
 .../nquads/WholeFileNQuadsInputFormat.java      |  51 --
 .../ntriples/BlockedNTriplesInputFormat.java    |  53 --
 .../io/input/ntriples/NTriplesInputFormat.java  |  46 --
 .../ntriples/WholeFileNTriplesInputFormat.java  |  51 --
 .../io/input/rdfjson/RdfJsonInputFormat.java    |  46 --
 .../rdf/io/input/rdfxml/RdfXmlInputFormat.java  |  46 --
 .../AbstractBlockBasedNodeTupleReader.java      | 344 ---------
 .../readers/AbstractBlockBasedQuadReader.java   |  51 --
 .../readers/AbstractBlockBasedTripleReader.java |  51 --
 .../AbstractLineBasedNodeTupleReader.java       | 265 -------
 .../readers/AbstractLineBasedQuadReader.java    |  50 --
 .../readers/AbstractLineBasedTripleReader.java  |  51 --
 .../rdf/io/input/readers/AbstractRdfReader.java | 108 ---
 .../AbstractWholeFileNodeTupleReader.java       | 328 --------
 .../readers/AbstractWholeFileQuadReader.java    |  51 --
 .../readers/AbstractWholeFileTripleReader.java  |  51 --
 .../rdf/io/input/readers/QuadsReader.java       |  49 --
 .../io/input/readers/TriplesOrQuadsReader.java  |  72 --
 .../rdf/io/input/readers/TriplesReader.java     |  49 --
 .../io/input/readers/TriplesToQuadsReader.java  | 102 ---
 .../input/readers/jsonld/JsonLDQuadReader.java  |  32 -
 .../readers/jsonld/JsonLDTripleReader.java      |  30 -
 .../readers/nquads/BlockedNQuadsReader.java     |  45 --
 .../io/input/readers/nquads/NQuadsReader.java   |  49 --
 .../readers/nquads/WholeFileNQuadsReader.java   |  42 -
 .../readers/ntriples/BlockedNTriplesReader.java |  45 --
 .../input/readers/ntriples/NTriplesReader.java  |  48 --
 .../ntriples/WholeFileNTriplesReader.java       |  42 -
 .../io/input/readers/rdfjson/RdfJsonReader.java |  37 -
 .../io/input/readers/rdfxml/RdfXmlReader.java   |  37 -
 .../input/readers/thrift/ThriftQuadReader.java  |  32 -
 .../readers/thrift/ThriftTripleReader.java      |  30 -
 .../rdf/io/input/readers/trig/TriGReader.java   |  37 -
 .../rdf/io/input/readers/trix/TriXReader.java   |  37 -
 .../io/input/readers/turtle/TurtleReader.java   |  37 -
 .../io/input/thrift/ThriftQuadInputFormat.java  |  39 -
 .../input/thrift/ThriftTripleInputFormat.java   |  39 -
 .../rdf/io/input/trig/TriGInputFormat.java      |  46 --
 .../rdf/io/input/trix/TriXInputFormat.java      |  42 -
 .../rdf/io/input/turtle/TurtleInputFormat.java  |  46 --
 .../rdf/io/input/util/BlockInputStream.java     |  94 ---
 .../hadoop/rdf/io/input/util/RdfIOUtils.java    | 101 ---
 .../rdf/io/input/util/TrackableInputStream.java |  38 -
 .../rdf/io/input/util/TrackedInputStream.java   | 124 ---
 .../io/input/util/TrackedPipedQuadsStream.java  |  55 --
 .../io/input/util/TrackedPipedRDFStream.java    |  64 --
 .../input/util/TrackedPipedTriplesStream.java   |  56 --
 .../AbstractBatchedNodeTupleOutputFormat.java   |  55 --
 .../rdf/io/output/AbstractNodeOutputFormat.java |  94 ---
 .../output/AbstractNodeTupleOutputFormat.java   | 109 ---
 .../AbstractStreamRdfNodeTupleOutputFormat.java |  73 --
 .../hadoop/rdf/io/output/QuadsOutputFormat.java |  64 --
 .../io/output/TriplesOrQuadsOutputFormat.java   |  74 --
 .../rdf/io/output/TriplesOutputFormat.java      |  61 --
 .../output/jsonld/JsonLDQuadOutputFormat.java   |  44 --
 .../output/jsonld/JsonLDTripleOutputFormat.java |  44 --
 .../io/output/nquads/NQuadsOutputFormat.java    |  52 --
 .../ntriples/NTriplesNodeOutputFormat.java      |  45 --
 .../output/ntriples/NTriplesOutputFormat.java   |  52 --
 .../io/output/rdfjson/RdfJsonOutputFormat.java  |  52 --
 .../io/output/rdfxml/RdfXmlOutputFormat.java    |  52 --
 .../output/thrift/ThriftQuadOutputFormat.java   |  51 --
 .../output/thrift/ThriftTripleOutputFormat.java |  52 --
 .../io/output/trig/BatchedTriGOutputFormat.java |  54 --
 .../rdf/io/output/trig/TriGOutputFormat.java    |  58 --
 .../rdf/io/output/trix/TriXOutputFormat.java    |  57 --
 .../turtle/BatchedTurtleOutputFormat.java       |  50 --
 .../io/output/turtle/TurtleOutputFormat.java    |  56 --
 .../writers/AbstractBatchedNodeTupleWriter.java | 113 ---
 .../writers/AbstractBatchedQuadWriter.java      |  80 --
 .../writers/AbstractBatchedTripleWriter.java    |  68 --
 .../AbstractLineBasedNodeTupleWriter.java       | 152 ----
 .../writers/AbstractLineBasedQuadWriter.java    |  71 --
 .../writers/AbstractLineBasedTripleWriter.java  |  68 --
 .../io/output/writers/AbstractNodeWriter.java   | 192 -----
 .../AbstractStreamRdfNodeTupleWriter.java       |  71 --
 .../AbstractWholeFileNodeTupleWriter.java       |  96 ---
 .../writers/AbstractWholeFileQuadWriter.java    |  66 --
 .../writers/AbstractWholeFileTripleWriter.java  |  65 --
 .../io/output/writers/QuadsToTriplesWriter.java |  59 --
 .../io/output/writers/StreamRdfQuadWriter.java  |  45 --
 .../output/writers/StreamRdfTripleWriter.java   |  44 --
 .../output/writers/jsonld/JsonLDQuadWriter.java |  38 -
 .../writers/jsonld/JsonLDTripleWriter.java      |  38 -
 .../io/output/writers/nquads/NQuadsWriter.java  |  57 --
 .../writers/ntriples/NTriplesNodeWriter.java    |  59 --
 .../output/writers/ntriples/NTriplesWriter.java |  58 --
 .../output/writers/rdfjson/RdfJsonWriter.java   |  51 --
 .../io/output/writers/rdfxml/RdfXmlWriter.java  |  51 --
 .../output/writers/thrift/ThriftQuadWriter.java |  38 -
 .../writers/thrift/ThriftTripleWriter.java      |  38 -
 .../output/writers/trig/BatchedTriGWriter.java  |  52 --
 .../writers/turtle/BatchedTurtleWriter.java     |  54 --
 .../rdf/io/registry/HadoopRdfIORegistry.java    | 310 --------
 .../hadoop/rdf/io/registry/ReaderFactory.java   |  83 --
 .../hadoop/rdf/io/registry/WriterFactory.java   |  96 ---
 .../readers/AbstractQuadsOnlyReaderFactory.java |  83 --
 .../registry/readers/AbstractReaderFactory.java |  80 --
 .../AbstractTriplesOnlyReaderFactory.java       |  83 --
 .../registry/readers/JsonLDReaderFactory.java   |  49 --
 .../registry/readers/NQuadsReaderFactory.java   |  42 -
 .../registry/readers/NTriplesReaderFactory.java |  38 -
 .../registry/readers/RdfJsonReaderFactory.java  |  41 -
 .../registry/readers/RdfXmlReaderFactory.java   |  40 -
 .../registry/readers/ThriftReaderFactory.java   |  49 --
 .../io/registry/readers/TriGReaderFactory.java  |  42 -
 .../io/registry/readers/TriXReaderFactory.java  |  41 -
 .../registry/readers/TurtleReaderFactory.java   |  40 -
 .../writers/AbstractQuadsOnlyWriterFactory.java |  86 ---
 .../AbstractTriplesOnlyWriterFactory.java       |  85 ---
 .../registry/writers/AbstractWriterFactory.java |  82 --
 .../registry/writers/JsonLDWriterFactory.java   |  52 --
 .../registry/writers/NQuadsWriterFactory.java   |  44 --
 .../registry/writers/NTriplesWriterFactory.java |  44 --
 .../registry/writers/RdfJsonWriterFactory.java  |  43 --
 .../registry/writers/RdfXmlWriterFactory.java   |  44 --
 .../registry/writers/ThriftWriterFactory.java   |  57 --
 .../io/registry/writers/TriGWriterFactory.java  |  45 --
 .../io/registry/writers/TriXWriterFactory.java  |  47 --
 .../registry/writers/TurtleWriterFactory.java   |  45 --
 ...he.jena.hadoop.rdf.io.registry.ReaderFactory |  10 -
 ...he.jena.hadoop.rdf.io.registry.WriterFactory |  10 -
 .../rdf/io/RdfTriplesInputTestMapper.java       |  47 --
 .../AbstractBlockedQuadInputFormatTests.java    |  33 -
 .../AbstractBlockedTripleInputFormatTests.java  |  33 -
 .../AbstractNodeTupleInputFormatTests.java      | 612 ---------------
 .../io/input/AbstractQuadsInputFormatTests.java |  70 --
 .../input/AbstractTriplesInputFormatTests.java  |  72 --
 .../AbstractWholeFileQuadInputFormatTests.java  | 115 ---
 ...AbstractWholeFileTripleInputFormatTests.java | 108 ---
 .../io/input/bnodes/AbstractBlankNodeTests.java | 636 ----------------
 .../bnodes/AbstractTripleBlankNodeTests.java    |  65 --
 .../input/bnodes/JsonLdTripleBlankNodeTest.java |  63 --
 .../io/input/bnodes/NTriplesBlankNodeTest.java  |  58 --
 .../io/input/bnodes/RdfJsonBlankNodeTest.java   |  58 --
 .../io/input/bnodes/RdfThriftBlankNodeTest.java |  68 --
 .../io/input/bnodes/RdfXmlBlankNodeTest.java    |  62 --
 .../io/input/bnodes/TurtleBlankNodeTest.java    |  58 --
 ...ractCompressedNodeTupleInputFormatTests.java |  74 --
 ...AbstractCompressedQuadsInputFormatTests.java |  71 --
 ...stractCompressedTriplesInputFormatTests.java |  71 --
 ...CompressedWholeFileQuadInputFormatTests.java | 150 ----
 ...mpressedWholeFileTripleInputFormatTests.java | 144 ----
 ...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 -
 ...bstractCompressedNQuadsInputFormatTests.java |  68 --
 ...mpressedWholeFileNQuadsInputFormatTests.java |  75 --
 .../nquads/BZipppedNQuadsInputTest.java         |  38 -
 .../BZipppedWholeFileNQuadsInputTest.java       |  37 -
 .../nquads/DeflatedNQuadsInputTest.java         |  37 -
 .../DeflatedWholeFileNQuadsInputTest.java       |  37 -
 .../nquads/GZippedNQuadsInputTest.java          |  38 -
 .../nquads/GZippedWholeFileNQuadsInputTest.java |  38 -
 ...mpressedBlockedNTriplesInputFormatTests.java |  53 --
 ...tractCompressedNTriplesInputFormatTests.java |  68 --
 ...ressedWholeFileNTriplesInputFormatTests.java |  75 --
 .../ntriples/BZippedBlockedNTriplesInput.java   |  37 -
 .../ntriples/BZippedNTriplesInputTest.java      |  38 -
 .../BZippedWholeFileNTriplesInputTest.java      |  38 -
 .../ntriples/DeflatedBlockedNTriplesInput.java  |  37 -
 .../ntriples/DeflatedNTriplesInputTest.java     |  38 -
 .../DeflatedWholeFileNTriplesInputTest.java     |  38 -
 .../ntriples/GZippedBlockedNTriplesInput.java   |  37 -
 .../ntriples/GZippedNTriplesInputTest.java      |  41 -
 .../GZippedWholeFileNTriplesInputTest.java      |  38 -
 ...stractCompressedRdfJsonInputFormatTests.java |  74 --
 .../rdfjson/BZippedRdfJsonInputTest.java        |  37 -
 .../rdfjson/DeflatedRdfJsonInputTest.java       |  37 -
 .../rdfjson/GZippedRdfJsonInputTest.java        |  37 -
 ...bstractCompressedRdfXmlInputFormatTests.java |  75 --
 .../rdfxml/BZippedRdfXmlInputTest.java          |  37 -
 .../rdfxml/DeflatedRdfXmlInputTest.java         |  37 -
 .../rdfxml/GZippedRdfXmlInputTest.java          |  37 -
 ...actCompressedThriftQuadInputFormatTests.java |  72 --
 ...tCompressedThriftTripleInputFormatTests.java |  72 --
 .../thrift/BZippedThriftQuadInputTest.java      |  34 -
 .../thrift/BZippedThriftTripleInputTest.java    |  34 -
 .../thrift/DeflatedThriftQuadInputTest.java     |  34 -
 .../thrift/DeflatedThriftTripleInputTest.java   |  34 -
 .../thrift/GZippedThriftQuadInputTest.java      |  34 -
 .../thrift/GZippedThriftTripleInputTest.java    |  34 -
 .../AbstractCompressedTriGInputFormatTests.java |  72 --
 .../compressed/trig/BZippedTriGInputTest.java   |  37 -
 .../compressed/trig/DeflatedTriGInputTest.java  |  37 -
 .../compressed/trig/GZippedTriGInputTest.java   |  37 -
 .../AbstractCompressedTriXInputFormatTests.java |  72 --
 .../compressed/trix/BZippedTriXInputTest.java   |  35 -
 .../compressed/trix/DeflatedTriXInputTest.java  |  35 -
 .../compressed/trix/GZippedTriXInputTest.java   |  35 -
 ...bstractCompressedTurtleInputFormatTests.java |  75 --
 .../turtle/BZippedTurtleInputTest.java          |  37 -
 .../turtle/DeflatedTurtleInputTest.java         |  37 -
 .../turtle/GZippedTurtleInputTest.java          |  37 -
 .../io/input/jsonld/JsonLDQuadInputTest.java    |  50 --
 .../io/input/jsonld/JsonLDTripleInputTest.java  |  50 --
 .../io/input/nquads/BlockedNQuadsInputTest.java |  51 --
 .../rdf/io/input/nquads/NQuadsInputTest.java    |  44 --
 .../input/nquads/WholeFileNQuadsInputTest.java  |  51 --
 .../ntriples/BlockedNTriplesInputTest.java      |  50 --
 .../io/input/ntriples/NTriplesInputTest.java    |  44 --
 .../ntriples/WholeFileNTriplesInputTest.java    |  52 --
 .../rdf/io/input/rdfjson/RdfJsonInputTest.java  |  51 --
 .../rdf/io/input/rdfxml/RdfXmlInputTest.java    |  51 --
 .../io/input/thrift/ThriftQuadInputTest.java    |  51 --
 .../io/input/thrift/ThriftTripleInputTest.java  |  51 --
 .../hadoop/rdf/io/input/trig/TriGInputTest.java |  50 --
 .../hadoop/rdf/io/input/trix/TriXInputTest.java |  50 --
 .../rdf/io/input/turtle/TurtleInputTest.java    |  50 --
 .../util/AbstractTrackableInputStreamTests.java | 701 -----------------
 .../rdf/io/input/util/BlockInputStreamTest.java | 240 ------
 .../io/input/util/TrackedInputStreamTest.java   |  39 -
 .../AbstractNodeTupleOutputFormatTests.java     | 255 -------
 .../output/AbstractQuadOutputFormatTests.java   |  51 --
 .../output/AbstractTripleOutputFormatTests.java |  47 --
 .../io/output/jsonld/JsonLdQuadOutputTest.java  |  47 --
 .../output/jsonld/JsonLdTripleOutputTest.java   |  47 --
 .../rdf/io/output/nquads/NQuadsOutputTest.java  |  51 --
 .../io/output/ntriples/NTriplesOutputTest.java  |  51 --
 .../io/output/rdfjson/RdfJsonOutputTest.java    |  51 --
 .../rdf/io/output/rdfxml/RdfXmlOutputTest.java  |  51 --
 .../io/output/thrift/ThriftQuadOutputTest.java  |  48 --
 .../output/thrift/ThriftTripleOutputTest.java   |  48 --
 .../io/output/trig/BatchedTriGOutputTest.java   |  92 ---
 .../io/output/trig/StreamedTriGOutputTest.java  |  92 ---
 .../output/trig/TriGBlankNodeOutputTests.java   | 120 ---
 .../rdf/io/output/trix/TriXOutputTest.java      |  47 --
 .../output/turtle/BatchedTurtleOutputTest.java  |  92 ---
 .../output/turtle/StreamedTurtleOutputTest.java |  92 ---
 .../turtle/TurtleBlankNodeOutputTests.java      | 118 ---
 .../io/registry/TestHadoopRdfIORegistry.java    | 186 -----
 .../jena-elephas-stats/hadoop-job.xml           |  46 --
 jena-hadoop-rdf/jena-elephas-stats/pom.xml      | 103 ---
 .../apache/jena/hadoop/rdf/stats/RdfStats.java  | 405 ----------
 .../jena/hadoop/rdf/stats/jobs/JobFactory.java  | 757 -------------------
 jena-hadoop-rdf/pom.xml                         |  97 ---
 pom.xml                                         |   4 +-
 763 files changed, 28196 insertions(+), 28196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/LICENSE
----------------------------------------------------------------------
diff --git a/jena-elephas/LICENSE b/jena-elephas/LICENSE
new file mode 100644
index 0000000..68c771a
--- /dev/null
+++ b/jena-elephas/LICENSE
@@ -0,0 +1,176 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/NOTICE
----------------------------------------------------------------------
diff --git a/jena-elephas/NOTICE b/jena-elephas/NOTICE
new file mode 100644
index 0000000..f87b36f
--- /dev/null
+++ b/jena-elephas/NOTICE
@@ -0,0 +1,5 @@
+Apache Jena
+Copyright 2011-2014 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/pom.xml
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/pom.xml b/jena-elephas/jena-elephas-common/pom.xml
new file mode 100644
index 0000000..7dd68a0
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/pom.xml
@@ -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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<groupId>org.apache.jena</groupId>
+		<artifactId>jena-elephas</artifactId>
+		<version>0.9.0-SNAPSHOT</version>
+	</parent>
+	<artifactId>jena-elephas-common</artifactId>
+	<name>Apache Jena - Elephas - Common API</name>
+	<description>Common code for RDF on Hadoop such as writable types for RDF primitives</description>
+
+	<!-- Note that versions are managed by parent POMs -->
+	<dependencies>
+		<!-- Hadoop Dependencies -->
+		<!-- Note these will be provided on the Hadoop cluster hence the provided 
+			scope -->
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Jena dependencies -->
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-arq</artifactId>
+		</dependency>
+
+		<!-- Test Dependencies -->
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
new file mode 100644
index 0000000..f0acc09
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/AbstractNodeTupleWritable.java
@@ -0,0 +1,193 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.util.NodeUtils;
+
+/**
+ * A abstract general purpose writable where the actual class represented is
+ * composed of a number of {@link Node} instances
+ * <p>
+ * The binary encoding of this base implementation is just a variable integer
+ * indicating the number of nodes present followed by the binary encodings of
+ * the {@link NodeWritable} instances. Derived implementations may wish to
+ * override the {@link #readFields(DataInput)} and {@link #write(DataOutput)}
+ * methods in order to use more specialised encodings.
+ * </p>
+ * 
+ * @param <T>
+ *            Tuple type
+ */
+public abstract class AbstractNodeTupleWritable<T> implements WritableComparable<AbstractNodeTupleWritable<T>> {
+
+    private T tuple;
+
+    /**
+     * Creates a new empty instance
+     */
+    protected AbstractNodeTupleWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance with the given value
+     * 
+     * @param tuple
+     *            Tuple value
+     */
+    protected AbstractNodeTupleWritable(T tuple) {
+        this.tuple = tuple;
+    }
+
+    /**
+     * Gets the tuple
+     * 
+     * @return Tuple
+     */
+    public T get() {
+        return this.tuple;
+    }
+
+    /**
+     * Sets the tuple
+     * 
+     * @param tuple
+     *            Tuple
+     */
+    public void set(T tuple) {
+        this.tuple = tuple;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        // Determine how many nodes
+        int size = WritableUtils.readVInt(input);
+        Node[] ns = new Node[size];
+
+        NodeWritable nw = new NodeWritable();
+        for (int i = 0; i < ns.length; i++) {
+            nw.readFields(input);
+            ns[i] = nw.get();
+        }
+
+        // Load the tuple
+        this.tuple = this.createTuple(ns);
+    }
+
+    /**
+     * Creates the actual tuple type from an array of nodes
+     * 
+     * @param ns
+     *            Nodes
+     * @return Tuple
+     */
+    protected abstract T createTuple(Node[] ns);
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        // Determine how many nodes
+        Node[] ns = this.createNodes(this.tuple);
+        WritableUtils.writeVInt(output, ns.length);
+
+        // Write out nodes
+        NodeWritable nw = new NodeWritable();
+        for (int i = 0; i < ns.length; i++) {
+            nw.set(ns[i]);
+            nw.write(output);
+        }
+    }
+
+    /**
+     * Sets the tuple value
+     * <p>
+     * Intended only for internal use i.e. when a derived implementation
+     * overrides {@link #readFields(DataInput)} and needs to set the tuple value
+     * directly i.e. when a derived implementation is using a custom encoding
+     * scheme
+     * </p>
+     * 
+     * @param tuple
+     *            Tuple
+     */
+    protected final void setInternal(T tuple) {
+        this.tuple = tuple;
+    }
+
+    /**
+     * Converts the actual tuple type into an array of nodes
+     * 
+     * @param tuple
+     *            Tuples
+     * @return Nodes
+     */
+    protected abstract Node[] createNodes(T tuple);
+
+    /**
+     * Compares instances node by node
+     * <p>
+     * Derived implementations may wish to override this and substitute native
+     * tuple based comparisons
+     * </p>
+     * 
+     * @param other
+     *            Instance to compare with
+     */
+    @Override
+    public int compareTo(AbstractNodeTupleWritable<T> other) {
+        Node[] ns = this.createNodes(this.tuple);
+        Node[] otherNs = this.createNodes(other.tuple);
+
+        if (ns.length < otherNs.length) {
+            return -1;
+        } else if (ns.length > otherNs.length) {
+            return 1;
+        }
+        // Compare node by node
+        for (int i = 0; i < ns.length; i++) {
+            int c = NodeUtils.compareRDFTerms(ns[i], otherNs[i]);
+            if (c != 0)
+                return c;
+        }
+        return 0;
+    }
+
+    @Override
+    public String toString() {
+        return this.get().toString();
+    }
+
+    @Override
+    public int hashCode() {
+        return this.get().hashCode();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof AbstractNodeTupleWritable))
+            return false;
+        return this.compareTo((AbstractNodeTupleWritable<T>) other) == 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
new file mode 100644
index 0000000..f29b156
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
@@ -0,0 +1,298 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * Represents a characteristic set which is comprised of a count of nodes for
+ * which the characteristic is applicable and a set of characteristics which
+ * represents the number of usages of predicates with those nodes
+ * 
+ * 
+ * 
+ */
+public class CharacteristicSetWritable implements WritableComparable<CharacteristicSetWritable> {
+
+    private Map<NodeWritable, CharacteristicWritable> characteristics = new TreeMap<NodeWritable, CharacteristicWritable>();
+    private LongWritable count = new LongWritable();
+
+    /**
+     * Creates a new empty characteristic set with the default count of 1
+     */
+    public CharacteristicSetWritable() {
+        this(1);
+    }
+
+    /**
+     * Creates a new characteristic set with the default count of 1 and the
+     * given characteristics
+     * 
+     * @param characteristics
+     *            Characteristics
+     */
+    public CharacteristicSetWritable(CharacteristicWritable... characteristics) {
+        this(1, characteristics);
+    }
+
+    /**
+     * Creates an empty characteristic set with the given count
+     * 
+     * @param count
+     *            Count
+     */
+    public CharacteristicSetWritable(long count) {
+        this(count, new CharacteristicWritable[0]);
+    }
+
+    /**
+     * Creates a new characteristic set
+     * 
+     * @param count
+     *            Count
+     * @param characteristics
+     *            Characteristics
+     */
+    public CharacteristicSetWritable(long count, CharacteristicWritable... characteristics) {
+        this.count.set(count);
+        for (CharacteristicWritable characteristic : characteristics) {
+            this.characteristics.put(characteristic.getNode(), characteristic);
+        }
+    }
+
+    /**
+     * Creates a new instance and reads its data from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static CharacteristicSetWritable read(DataInput input) throws IOException {
+        CharacteristicSetWritable set = new CharacteristicSetWritable();
+        set.readFields(input);
+        return set;
+    }
+
+    /**
+     * Gets the count
+     * 
+     * @return Count
+     */
+    public LongWritable getCount() {
+        return this.count;
+    }
+
+    /**
+     * Gets the characteristics
+     * 
+     * @return Characteristics
+     */
+    public Iterator<CharacteristicWritable> getCharacteristics() {
+        return this.characteristics.values().iterator();
+    }
+
+    /**
+     * Gets the size of the characteristic set
+     * 
+     * @return Size
+     */
+    public int size() {
+        return this.characteristics.size();
+    }
+
+    /**
+     * Adds a characteristic to the set merging it into the appropriate existing
+     * characteristic if applicable
+     * 
+     * @param characteristic
+     *            Characteristics
+     */
+    public void add(CharacteristicWritable characteristic) {
+        if (this.characteristics.containsKey(characteristic.getNode())) {
+            this.characteristics.get(characteristic.getNode()).increment(characteristic.getCount().get());
+        } else {
+            this.characteristics.put(characteristic.getNode(), characteristic);
+        }
+    }
+
+    /**
+     * Adds some characteristics to the set merging them with the appropriate
+     * existing characteristics if applicable
+     * 
+     * @param characteristics
+     */
+    public void add(CharacteristicWritable... characteristics) {
+        for (CharacteristicWritable characteristic : characteristics) {
+            this.add(characteristic);
+        }
+    }
+
+    /**
+     * Adds the contents of the other characteristic set to this characteristic
+     * set
+     * 
+     * @param set
+     *            Characteristic set
+     */
+    public void add(CharacteristicSetWritable set) {
+        this.increment(set.getCount().get());
+        Iterator<CharacteristicWritable> iter = set.getCharacteristics();
+        while (iter.hasNext()) {
+            this.add(iter.next());
+        }
+    }
+
+    /**
+     * Gets whether the set contains a characteristic for the given predicate
+     * 
+     * @param uri
+     *            Predicate URI
+     * @return True if contained in the set, false otherwise
+     */
+    public boolean hasCharacteristic(String uri) {
+        return this.hasCharacteristic(NodeFactory.createURI(uri));
+    }
+
+    /**
+     * Gets whether the set contains a characteristic for the given predicate
+     * 
+     * @param n
+     *            Predicate
+     * @return True if contained in the set, false otherwise
+     */
+    public boolean hasCharacteristic(Node n) {
+        return this.hasCharacteristic(new NodeWritable(n));
+    }
+
+    /**
+     * Gets whether the set contains a characteristic for the given predicate
+     * 
+     * @param n
+     *            Predicate
+     * @return True if contained in the set, false otherwise
+     */
+    public boolean hasCharacteristic(NodeWritable n) {
+        return this.characteristics.containsKey(n);
+    }
+
+    /**
+     * Increments the count by the given increment
+     * 
+     * @param l
+     *            Increment
+     */
+    public void increment(long l) {
+        this.count.set(this.count.get() + l);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        // Write size, then count, then characteristics
+        WritableUtils.writeVInt(output, this.characteristics.size());
+        this.count.write(output);
+        for (CharacteristicWritable characteristic : this.characteristics.values()) {
+            characteristic.write(output);
+        }
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        // Read size, then count, then characteristics
+        int size = WritableUtils.readVInt(input);
+        this.count.readFields(input);
+        this.characteristics.clear();
+        for (int i = 0; i < size; i++) {
+            CharacteristicWritable cw = CharacteristicWritable.read(input);
+            this.characteristics.put(cw.getNode(), cw);
+        }
+    }
+
+    @Override
+    public int compareTo(CharacteristicSetWritable cs) {
+        int size = this.characteristics.size();
+        int otherSize = cs.characteristics.size();
+        if (size < otherSize) {
+            return -1;
+        } else if (size > otherSize) {
+            return 1;
+        } else {
+            // Compare characteristics in turn
+            Iterator<CharacteristicWritable> iter = this.getCharacteristics();
+            Iterator<CharacteristicWritable> otherIter = cs.getCharacteristics();
+
+            int compare = 0;
+            while (iter.hasNext()) {
+                CharacteristicWritable c = iter.next();
+                CharacteristicWritable otherC = otherIter.next();
+                compare = c.compareTo(otherC);
+                if (compare != 0)
+                    return compare;
+            }
+            return compare;
+        }
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof CharacteristicSetWritable))
+            return false;
+        return this.compareTo((CharacteristicSetWritable) other) == 0;
+    }
+
+    @Override
+    public int hashCode() {
+        // Build a hash code from characteristics
+        if (this.characteristics.size() == 0)
+            return 0;
+        Iterator<CharacteristicWritable> iter = this.getCharacteristics();
+        int hash = 17;
+        while (iter.hasNext()) {
+            hash = hash * 31 + iter.next().hashCode();
+        }
+        return hash;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder builder = new StringBuilder();
+        builder.append("{ ");
+        builder.append(this.count.get());
+        Iterator<CharacteristicWritable> iter = this.getCharacteristics();
+        while (iter.hasNext()) {
+            builder.append(" , ");
+            builder.append(iter.next().toString());
+        }
+        builder.append(" }");
+        return builder.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
new file mode 100644
index 0000000..90fc7db
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
@@ -0,0 +1,160 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.hp.hpl.jena.graph.Node;
+
+/**
+ * Represents a characteristic for a single node and contains the node and a
+ * count associated with that node
+ * <p>
+ * Note that characteristics are compared based upon only the nodes and not
+ * their counts
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class CharacteristicWritable implements WritableComparable<CharacteristicWritable> {
+
+    private NodeWritable node = new NodeWritable();
+    private LongWritable count = new LongWritable();
+
+    /**
+     * Creates an empty characteristic writable
+     */
+    public CharacteristicWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a characteristic writable with the given node and the default
+     * count of 1
+     * 
+     * @param n
+     *            Node
+     */
+    public CharacteristicWritable(Node n) {
+        this(n, 1);
+    }
+
+    /**
+     * Creates a characteristic writable with the given node and count
+     * 
+     * @param n
+     *            Node
+     * @param count
+     *            Count
+     */
+    public CharacteristicWritable(Node n, long count) {
+        this.node.set(n);
+        this.count.set(count);
+    }
+
+    /**
+     * Creates a new instance and reads in its data from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static CharacteristicWritable read(DataInput input) throws IOException {
+        CharacteristicWritable cw = new CharacteristicWritable();
+        cw.readFields(input);
+        return cw;
+    }
+
+    /**
+     * Gets the node
+     * 
+     * @return Node
+     */
+    public NodeWritable getNode() {
+        return this.node;
+    }
+
+    /**
+     * Gets the count
+     * 
+     * @return Count
+     */
+    public LongWritable getCount() {
+        return this.count;
+    }
+
+    /**
+     * Increments the count by 1
+     */
+    public void increment() {
+        this.increment(1);
+    }
+
+    /**
+     * Increments the count by the given value
+     * 
+     * @param l
+     *            Value to increment by
+     */
+    public void increment(long l) {
+        this.count.set(this.count.get() + l);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        this.node.write(output);
+        this.count.write(output);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.node.readFields(input);
+        this.count.readFields(input);
+    }
+
+    @Override
+    public int compareTo(CharacteristicWritable o) {
+        return this.node.compareTo(o.node);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof CharacteristicWritable))
+            return false;
+        return this.compareTo((CharacteristicWritable) other) == 0;
+    }
+
+    @Override
+    public int hashCode() {
+        return this.node.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return "(" + this.node.toString() + ", " + this.count.toString() + ")";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
new file mode 100644
index 0000000..e06aac4
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.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.types;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.jena.atlas.lib.Tuple;
+import com.hp.hpl.jena.graph.Node;
+
+/**
+ * A writable RDF tuple
+ * <p>
+ * Unlike the more specific {@link TripleWritable} and {@link QuadWritable} this
+ * class allows for arbitrary length tuples and does not restrict tuples to
+ * being of uniform size.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class NodeTupleWritable extends AbstractNodeTupleWritable<Tuple<Node>> {
+
+    /**
+     * Creates a new empty instance
+     */
+    public NodeTupleWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance with the given value
+     * 
+     * @param tuple
+     *            Tuple
+     */
+    public NodeTupleWritable(Tuple<Node> tuple) {
+        super(tuple);
+    }
+
+    /**
+     * Creates a new instance from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static NodeTupleWritable read(DataInput input) throws IOException {
+        NodeTupleWritable t = new NodeTupleWritable();
+        t.readFields(input);
+        return t;
+    }
+
+    @Override
+    protected Tuple<Node> createTuple(Node[] ns) {
+        return Tuple.create(ns);
+    }
+
+    @Override
+    protected Node[] createNodes(Tuple<Node> tuple) {
+        return tuple.tuple();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
new file mode 100644
index 0000000..cf00f8d
--- /dev/null
+++ b/jena-elephas/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
@@ -0,0 +1,188 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
+import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
+import org.apache.jena.riot.thrift.TRDF;
+import org.apache.jena.riot.thrift.ThriftConvert;
+import org.apache.jena.riot.thrift.wire.RDF_Term;
+import org.apache.thrift.TException;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.util.NodeUtils;
+
+/**
+ * A writable for {@link Node} instances
+ * <p>
+ * This uses <a
+ * href="http://afs.github.io/rdf-thrift/rdf-binary-thrift.html">RDF Thrift</a>
+ * for the binary encoding of terms. The in-memory storage for this type is both
+ * a {@link Node} and a {@link RDF_Term} with lazy conversion between the two
+ * forms as necessary.
+ * </p>
+ */
+public class NodeWritable implements WritableComparable<NodeWritable> {
+
+    static {
+        WritableComparator.define(NodeWritable.class, new SimpleBinaryComparator());
+    }
+
+    private Node node;
+    private RDF_Term term = new RDF_Term();
+
+    /**
+     * Creates an empty writable
+     */
+    public NodeWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static NodeWritable read(DataInput input) throws IOException {
+        NodeWritable nw = new NodeWritable();
+        nw.readFields(input);
+        return nw;
+    }
+
+    /**
+     * Creates a new writable with the given value
+     * 
+     * @param n
+     *            Node
+     */
+    public NodeWritable(Node n) {
+        this.set(n);
+    }
+
+    /**
+     * Gets the node
+     * 
+     * @return Node
+     */
+    public Node get() {
+        // We may not have yet loaded the node
+        if (this.node == null) {
+            // If term is set to undefined then node is supposed to be null
+            if (this.term.isSet() && !this.term.isSetUndefined()) {
+                this.node = ThriftConvert.convert(this.term);
+            }
+        }
+        return this.node;
+    }
+
+    /**
+     * Sets the node
+     * 
+     * @param n
+     *            Node
+     */
+    public void set(Node n) {
+        this.node = n;
+        // Clear the term for now
+        // We only convert the Node to a term as and when we want to write it
+        // out in order to not waste effort if the value is never written out
+        this.term.clear();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        // Clear previous value
+        this.node = null;
+        this.term.clear();
+
+        // Read in the new value
+        int termLength = input.readInt();
+        byte[] buffer = new byte[termLength];
+        input.readFully(buffer);
+        try {
+            ThriftConverter.fromBytes(buffer, this.term);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+
+        // Note that we don't convert it back into a Node at this time
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        // May not yet have prepared the Thrift term
+        if (!this.term.isSet()) {
+            if (this.node == null) {
+                this.term.setUndefined(TRDF.UNDEF);
+            } else {
+                ThriftConvert.toThrift(this.node, null, this.term, false);
+            }
+        }
+
+        // Write out the Thrift term
+        byte[] buffer;
+        try {
+            buffer = ThriftConverter.toBytes(this.term);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        output.writeInt(buffer.length);
+        output.write(buffer);
+    }
+
+    @Override
+    public int compareTo(NodeWritable other) {
+        // Use get() rather than accessing the field directly because the node
+        // field is lazily instantiated from the Thrift term
+        return NodeUtils.compareRDFTerms(this.get(), other.get());
+    }
+
+    @Override
+    public String toString() {
+        // Use get() rather than accessing the field directly because the node
+        // field is lazily instantiated from the Thrift term
+        Node n = this.get();
+        if (n == null)
+            return "";
+        return n.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        // Use get() rather than accessing the field directly because the node
+        // field is lazily instantiated from the Thrift term
+        Node n = this.get();
+        return n != null ? this.get().hashCode() : 0;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof NodeWritable))
+            return false;
+        return this.compareTo((NodeWritable) other) == 0;
+    }
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
deleted file mode 100644
index 6c1abe9..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-import java.util.Iterator;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.util.LineReader;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.system.ParserProfile;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract implementation of a record reader that reads records from line
- * based tuple formats. This only supports reading from file splits currently.
- * <p>
- * The keys produced are the position of the line in the file and the values
- * will be node tuples
- * </p>
- * 
- * 
- * 
- * @param <TValue>
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractLineBasedNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractLineBasedNodeTupleReader.class);
-    private CompressionCodecFactory compressionCodecs = null;
-    private long start, pos, end, estLength;
-    private int maxLineLength;
-    private LineReader in;
-    private LongWritable key = null;
-    private Text value = null;
-    private T tuple = null;
-    private ParserProfile profile = null;
-    private boolean ignoreBadTuples = true;
-
-    @Override
-    public final void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
-        LOG.debug("initialize({}, {})", genericSplit, context);
-
-        // Assuming file split
-        if (!(genericSplit instanceof FileSplit))
-            throw new IOException("This record reader only supports FileSplit inputs");
-        FileSplit split = (FileSplit) genericSplit;
-
-        // Configuration
-        profile = RdfIOUtils.createParserProfile(context, split.getPath());
-        Configuration config = context.getConfiguration();
-        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
-        if (this.ignoreBadTuples)
-            LOG.warn(
-                    "Configured to ignore bad tuples, parsing errors will be logged and the bad line skipped but no errors will be thrownConsider setting {} to false to disable this behaviour",
-                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
-
-        // Figure out what portion of the file to read
-        this.maxLineLength = config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE);
-        start = split.getStart();
-        end = start + split.getLength();
-        final Path file = split.getPath();
-        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
-        compressionCodecs = new CompressionCodecFactory(config);
-        final CompressionCodec codec = compressionCodecs.getCodec(file);
-
-        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { start, split.getLength(), totalLength }));
-
-        // Open the file and seek to the start of the split
-        FileSystem fs = file.getFileSystem(config);
-        FSDataInputStream fileIn = fs.open(file);
-        boolean skipFirstLine = false;
-        if (codec != null) {
-            // Compressed input
-            // For compressed input NLineInputFormat will have failed to find
-            // any line breaks and will give us a split from 0 -> (length - 1)
-            // Add 1 and verify we got complete split
-            if (totalLength > split.getLength() + 1)
-                throw new IOException("This record reader can only be used with compressed input where the split covers the whole file");
-            in = new LineReader(codec.createInputStream(fileIn), config);
-            estLength = end;
-            end = Long.MAX_VALUE;
-        } else {
-            // Uncompressed input
-            if (start != 0) {
-                skipFirstLine = true;
-                --start;
-                fileIn.seek(start);
-            }
-            in = new LineReader(fileIn, config);
-        }
-        // Skip first line and re-establish "start".
-        // This is to do with how line reader reads lines and how
-        // NLineInputFormat will provide the split information to use
-        if (skipFirstLine) {
-            start += in.readLine(new Text(), 0, (int) Math.min((long) Integer.MAX_VALUE, end - start));
-        }
-        this.pos = start;
-    }
-
-    /**
-     * Gets an iterator over the data on the current line
-     * 
-     * @param line
-     *            Line
-     * @param profile
-     *            Parser profile
-     * @return Iterator
-     */
-    protected abstract Iterator<TValue> getIterator(String line, ParserProfile profile);
-
-    /**
-     * Creates an instance of a writable tuple from the given tuple value
-     * 
-     * @param tuple
-     *            Tuple value
-     * @return Writable tuple
-     */
-    protected abstract T createInstance(TValue tuple);
-
-    @Override
-    public final boolean nextKeyValue() throws IOException, InterruptedException {
-        // Reuse key for efficiency
-        if (key == null) {
-            key = new LongWritable();
-        }
-
-        // Reset value which we use for reading lines
-        if (value == null) {
-            value = new Text();
-        }
-        tuple = null;
-
-        // Try to read the next valid line
-        int newSize = 0;
-        while (pos < end) {
-            // Read next line
-            newSize = in.readLine(value, maxLineLength, Math.max((int) Math.min(Integer.MAX_VALUE, end - pos), maxLineLength));
-
-            // Once we get an empty line we've reached the end of our input
-            if (newSize == 0) {
-                break;
-            }
-
-            // Update position, remember that where inputs are compressed we may
-            // be at a larger position then we expected because the length of
-            // the split is likely less than the length of the data once
-            // decompressed
-            key.set(pos);
-            pos += newSize;
-            if (pos > estLength)
-                estLength = pos + 1;
-
-            // Skip lines that exceed the line length limit that has been set
-            if (newSize >= maxLineLength) {
-                LOG.warn("Skipped oversized line of size {} at position {}", newSize, (pos - newSize));
-                continue;
-            }
-
-            // Attempt to read the tuple from current line
-            try {
-                Iterator<TValue> iter = this.getIterator(value.toString(), profile);
-                if (iter.hasNext()) {
-                    tuple = this.createInstance(iter.next());
-
-                    // If we reach here we've found a valid tuple so we can
-                    // break out of the loop
-                    break;
-                } else {
-                    // Empty line/Comment line
-                    LOG.debug("Valid line with no triple at position {}", (pos - newSize));
-                    continue;
-                }
-            } catch (Throwable e) {
-                // Failed to read the tuple on this line
-                LOG.error("Bad tuple at position " + (pos - newSize), e);
-                if (this.ignoreBadTuples)
-                    continue;
-                throw new IOException(String.format("Bad tuple at position %d", (pos - newSize)), e);
-            }
-        }
-        boolean result = this.tuple != null;
-
-        // End of input
-        if (newSize == 0) {
-            key = null;
-            value = null;
-            tuple = null;
-            result = false;
-            estLength = pos;
-        }
-        LOG.debug("nextKeyValue() --> {}", result);
-        return result;
-    }
-
-    @Override
-    public LongWritable getCurrentKey() throws IOException, InterruptedException {
-        LOG.debug("getCurrentKey() --> {}", key);
-        return key;
-    }
-
-    @Override
-    public T getCurrentValue() throws IOException, InterruptedException {
-        LOG.debug("getCurrentValue() --> {}", tuple);
-        return tuple;
-    }
-
-    @Override
-    public float getProgress() throws IOException, InterruptedException {
-        float progress = 0.0f;
-        if (start != end) {
-            if (end == Long.MAX_VALUE) {
-                if (estLength == 0)
-                    return 1.0f;
-                // Use estimated length
-                progress = Math.min(1.0f, (pos - start) / (float) (estLength - start));
-            } else {
-                // Use actual length
-                progress = Math.min(1.0f, (pos - start) / (float) (end - start));
-            }
-        }
-        LOG.debug("getProgress() --> {}", progress);
-        return progress;
-    }
-
-    @Override
-    public void close() throws IOException {
-        LOG.debug("close()");
-        if (in != null) {
-            in.close();
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
deleted file mode 100644
index ac93865..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.readers;
-
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.tokens.Tokenizer;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract reader for line based quad formats
- * 
- * 
- * 
- */
-public abstract class AbstractLineBasedQuadReader extends AbstractLineBasedNodeTupleReader<Quad, QuadWritable> {
-
-    @Override
-    protected Iterator<Quad> getIterator(String line, ParserProfile profile) {
-        Tokenizer tokenizer = getTokenizer(line);
-        return getQuadsIterator(tokenizer, profile);
-    }
-
-    @Override
-    protected QuadWritable createInstance(Quad q) {
-        return new QuadWritable(q);
-    }
-
-    protected abstract Tokenizer getTokenizer(String line);
-
-    protected abstract Iterator<Quad> getQuadsIterator(Tokenizer tokenizer, ParserProfile profile);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
deleted file mode 100644
index a0232f5..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.tokens.Tokenizer;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * An abstract record reader for line based triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractLineBasedTripleReader extends AbstractLineBasedNodeTupleReader<Triple, TripleWritable> {
-
-    @Override
-    protected Iterator<Triple> getIterator(String line, ParserProfile profile) {
-        Tokenizer tokenizer = getTokenizer(line);
-        return getTriplesIterator(tokenizer, profile);
-    }
-
-    @Override
-    protected TripleWritable createInstance(Triple t) {
-        return new TripleWritable(t);
-    }
-    
-    protected abstract Tokenizer getTokenizer(String line);
-
-    protected abstract Iterator<Triple> getTriplesIterator(Tokenizer tokenizer, ParserProfile profile);
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
deleted file mode 100644
index d0ffed8..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.Path;
-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.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract record reader for arbitrary RDF which provides support for
- * selecting the actual record reader to use based on detecting the RDF language
- * from the file name
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractRdfReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        RecordReader<LongWritable, T> {
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractRdfReader.class);
-
-    private RecordReader<LongWritable, T> reader;
-
-    @Override
-    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException,
-            InterruptedException {
-        LOG.debug("initialize({}, {})", genericSplit, context);
-
-        // Assuming file split
-        if (!(genericSplit instanceof FileSplit))
-            throw new IOException("This record reader only supports FileSplit inputs");
-
-        // Find RDF language
-        FileSplit split = (FileSplit) genericSplit;
-        Path path = split.getPath();
-        Lang lang = RDFLanguages.filenameToLang(path.getName());
-        if (lang == null)
-            throw new IOException("There is no registered RDF language for the input file " + path.toString());
-
-        // Select the record reader and initialize
-        this.reader = this.selectRecordReader(lang);
-        this.reader.initialize(split, context);
-    }
-
-    /**
-     * Selects the appropriate record reader to use for the given RDF language
-     * 
-     * @param lang
-     *            RDF language
-     * @return Record reader
-     * @throws IOException
-     *             Should be thrown if no record reader can be selected
-     */
-    protected abstract RecordReader<LongWritable, T> selectRecordReader(Lang lang) throws IOException;
-
-    @Override
-    public final boolean nextKeyValue() throws IOException, InterruptedException {
-        return this.reader.nextKeyValue();
-    }
-
-    @Override
-    public final LongWritable getCurrentKey() throws IOException, InterruptedException {
-        return this.reader.getCurrentKey();
-    }
-
-    @Override
-    public final T getCurrentValue() throws IOException, InterruptedException {
-        return this.reader.getCurrentValue();
-    }
-
-    @Override
-    public final float getProgress() throws IOException, InterruptedException {
-        return this.reader.getProgress();
-    }
-
-    @Override
-    public final void close() throws IOException {
-        this.reader.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
deleted file mode 100644
index c2da3f7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFDataMgr;
-import org.apache.jena.riot.ReaderRIOT;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-import org.apache.jena.riot.lang.PipedRDFStream;
-import org.apache.jena.riot.system.ParserProfile;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract implementation for a record reader that reads records from whole
- * files i.e. the whole file must be kept together to allow tuples to be
- * successfully read. This only supports reading from file splits currently.
- * <p>
- * The keys produced are the approximate position in the file at which a tuple
- * was found and the values will be node tuples. Positions are approximate
- * because they are recorded after the point at which the most recent tuple was
- * parsed from the input thus they reflect the approximate position in the
- * stream immediately after which the triple was found.
- * </p>
- * <p>
- * You should also be aware that with whole file formats syntax compressions in
- * the format may mean that there are multiple triples produced with the same
- * position and thus key.
- * </p>
- * 
- * 
- * 
- * @param <TValue>
- *            Value type
- * @param <T>
- *            Tuple type
- */
-public abstract class AbstractWholeFileNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractLineBasedNodeTupleReader.class);
-    private CompressionCodec compressionCodecs;
-    private TrackedInputStream input;
-    private LongWritable key;
-    private long length;
-    private T tuple;
-    private TrackedPipedRDFStream<TValue> stream;
-    private PipedRDFIterator<TValue> iter;
-    private Thread parserThread;
-    private boolean finished = false;
-    private boolean ignoreBadTuples = true;
-    private boolean parserFinished = false;
-    private Throwable parserError = null;
-
-    @Override
-    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
-        LOG.debug("initialize({}, {})", genericSplit, context);
-
-        // Assuming file split
-        if (!(genericSplit instanceof FileSplit))
-            throw new IOException("This record reader only supports FileSplit inputs");
-        FileSplit split = (FileSplit) genericSplit;
-
-        // Configuration
-        Configuration config = context.getConfiguration();
-        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
-        if (this.ignoreBadTuples)
-            LOG.warn(
-                    "Configured to ignore bad tuples, parsing errors will be logged and further parsing aborted but no user visible errors will be thrown.  Consider setting {} to false to disable this behaviour",
-                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
-
-        // Figure out what portion of the file to read
-        if (split.getStart() > 0)
-            throw new IOException("This record reader requires a file split which covers the entire file");
-        final Path file = split.getPath();
-        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
-        CompressionCodecFactory factory = new CompressionCodecFactory(config);
-        this.compressionCodecs = factory.getCodec(file);
-
-        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { split.getStart(), split.getLength(),
-                totalLength }));
-
-        if (totalLength > split.getLength())
-            throw new IOException("This record reader requires a file split which covers the entire file");
-
-        // Open the file and prepare the input stream
-        FileSystem fs = file.getFileSystem(config);
-        FSDataInputStream fileIn = fs.open(file);
-        this.length = split.getLength();
-        if (this.compressionCodecs != null) {
-            // Compressed input
-            input = new TrackedInputStream(this.compressionCodecs.createInputStream(fileIn));
-        } else {
-            // Uncompressed input
-            input = new TrackedInputStream(fileIn);
-        }
-
-        // Set up background thread for parser
-        iter = this.getPipedIterator();
-        this.stream = this.getPipedStream(iter, this.input);
-        ParserProfile profile = RdfIOUtils.createParserProfile(context, file);
-        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage(), profile);
-        this.parserThread = new Thread(parserRunnable);
-        this.parserThread.setDaemon(true);
-        this.parserThread.start();
-    }
-
-    /**
-     * Gets the RDF iterator to use
-     * 
-     * @return Iterator
-     */
-    protected abstract PipedRDFIterator<TValue> getPipedIterator();
-
-    /**
-     * Gets the RDF stream to parse to
-     * 
-     * @param iterator
-     *            Iterator
-     * @return RDF stream
-     */
-    protected abstract TrackedPipedRDFStream<TValue> getPipedStream(PipedRDFIterator<TValue> iterator, TrackableInputStream input);
-
-    /**
-     * Gets the RDF language to use for parsing
-     * 
-     * @return
-     */
-    protected abstract Lang getRdfLanguage();
-
-    /**
-     * Creates the runnable upon which the parsing will run
-     * 
-     * @param input
-     *            Input
-     * @param stream
-     *            Stream
-     * @param lang
-     *            Language to use for parsing
-     * @return Parser runnable
-     */
-    private Runnable createRunnable(@SuppressWarnings("rawtypes") final AbstractWholeFileNodeTupleReader reader, final InputStream input,
-            final PipedRDFStream<TValue> stream, final Lang lang, final ParserProfile profile) {
-        return new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    ReaderRIOT riotReader = RDFDataMgr.createReader(lang);
-                    riotReader.setParserProfile(profile);
-                    riotReader.read(input, null, lang.getContentType(), stream, null);
-                    reader.setParserFinished(null);
-                } catch (Throwable e) {
-                    reader.setParserFinished(e);
-                }
-            }
-        };
-    }
-
-    /**
-     * Sets the parser thread finished state
-     * 
-     * @param e
-     *            Error (if any)
-     */
-    private void setParserFinished(Throwable e) {
-        synchronized (this.parserThread) {
-            this.parserError = e;
-            this.parserFinished = true;
-        }
-    }
-
-    /**
-     * Waits for the parser thread to have reported as finished
-     * 
-     * @throws InterruptedException
-     */
-    private void waitForParserFinished() throws InterruptedException {
-        do {
-            synchronized (this.parserThread) {
-                if (this.parserFinished)
-                    return;
-            }
-            Thread.sleep(50);
-        } while (true);
-    }
-
-    /**
-     * Creates an instance of a writable tuple from the given tuple value
-     * 
-     * @param tuple
-     *            Tuple value
-     * @return Writable tuple
-     */
-    protected abstract T createInstance(TValue tuple);
-
-    @Override
-    public boolean nextKeyValue() throws IOException, InterruptedException {
-        // Reuse key for efficiency
-        if (key == null) {
-            key = new LongWritable();
-        }
-
-        if (this.finished)
-            return false;
-
-        try {
-            if (this.iter.hasNext()) {
-                Long l = this.stream.getPosition();
-                if (l != null) {
-                    this.key.set(l);
-                    // For compressed input the actual length from which we
-                    // calculate progress is likely less than the actual
-                    // uncompressed length so we may need to increment the
-                    // length as we go along
-                    // We always add 1 more than the current length because we
-                    // don't want to report 100% progress until we really have
-                    // finished
-                    if (this.compressionCodecs != null && l > this.length)
-                        this.length = l + 1;
-                }
-                this.tuple = this.createInstance(this.iter.next());
-                return true;
-            } else {
-                // Need to ensure that the parser thread has finished in order
-                // to determine whether we finished without error
-                this.waitForParserFinished();
-                if (this.parserError != null) {
-                    LOG.error("Error parsing whole file, aborting further parsing", this.parserError);
-                    if (!this.ignoreBadTuples)
-                        throw new IOException("Error parsing whole file at position " + this.input.getBytesRead() + ", aborting further parsing",
-                                this.parserError);
-
-                }
-
-                this.key = null;
-                this.tuple = null;
-                this.finished = true;
-                // This is necessary so that when compressed input is used we
-                // report 100% progress once we've reached the genuine end of
-                // the stream
-                if (this.compressionCodecs != null)
-                    this.length--;
-                return false;
-            }
-        } catch (Throwable e) {
-            // Failed to read the tuple on this line
-            LOG.error("Error parsing whole file, aborting further parsing", e);
-            if (!this.ignoreBadTuples) {
-                this.iter.close();
-                throw new IOException("Error parsing whole file at position " + this.input.getBytesRead() + ", aborting further parsing", e);
-            }
-            this.key = null;
-            this.tuple = null;
-            this.finished = true;
-            return false;
-        }
-    }
-
-    @Override
-    public LongWritable getCurrentKey() throws IOException, InterruptedException {
-        return this.key;
-    }
-
-    @Override
-    public T getCurrentValue() throws IOException, InterruptedException {
-        return this.tuple;
-    }
-
-    @Override
-    public float getProgress() throws IOException, InterruptedException {
-        float progress = 0.0f;
-        if (this.key == null) {
-            // We've either not started or we've finished
-            progress = (this.finished ? 1.0f : 0.0f);
-        } else if (this.key.get() == Long.MIN_VALUE) {
-            // We don't have a position so we've either in-progress or finished
-            progress = (this.finished ? 1.0f : 0.5f);
-        } else {
-            // We're some way through the file
-            progress = this.key.get() / (float) this.length;
-        }
-        LOG.debug("getProgress() --> {}", progress);
-        return progress;
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.iter.close();
-        this.input.close();
-        this.finished = true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
deleted file mode 100644
index e525bea..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedQuadsStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract record reader for whole file triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractWholeFileQuadReader extends AbstractWholeFileNodeTupleReader<Quad, QuadWritable> {
-
-    @Override
-    protected PipedRDFIterator<Quad> getPipedIterator() {
-        return new PipedRDFIterator<Quad>();
-    }
-
-    @Override
-    protected TrackedPipedRDFStream<Quad> getPipedStream(PipedRDFIterator<Quad> iterator, TrackableInputStream input) {
-        return new TrackedPipedQuadsStream(iterator, input);
-    }
-
-    @Override
-    protected QuadWritable createInstance(Quad tuple) {
-        return new QuadWritable(tuple);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
deleted file mode 100644
index 8710b99..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedTriplesStream;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * An abstract record reader for whole file triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractWholeFileTripleReader extends AbstractWholeFileNodeTupleReader<Triple, TripleWritable> {
-
-    @Override
-    protected PipedRDFIterator<Triple> getPipedIterator() {
-        return new PipedRDFIterator<Triple>();
-    }
-
-    @Override
-    protected TrackedPipedRDFStream<Triple> getPipedStream(PipedRDFIterator<Triple> iterator, TrackableInputStream input) {
-        return new TrackedPipedTriplesStream(iterator, input);
-    }
-
-    @Override
-    protected TripleWritable createInstance(Triple tuple) {
-        return new TripleWritable(tuple);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
deleted file mode 100644
index 26b0a8b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-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;
-
-/**
- * A record reader that reads triples from any RDF quads format
- */
-public class QuadsReader extends AbstractRdfReader<Quad, QuadWritable> {
-
-    @Override
-    protected RecordReader<LongWritable, QuadWritable> selectRecordReader(Lang lang) throws IOException {
-        if (!RDFLanguages.isQuads(lang))
-            throw new IOException(
-                    lang.getLabel()
-                            + " is not a RDF quads format, perhaps you wanted TriplesInputFormat or TriplesOrQuadsInputFormat instead?");
-
-        // This will throw an appropriate error if the language does not support
-        // triples
-        return HadoopRdfIORegistry.createQuadReader(lang);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
deleted file mode 100644
index b071f67..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-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.graph.Node;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A record reader that reads RDF from any triples/quads format. Triples are
- * converted into quads in the default graph. This behaviour can be changed by
- * deriving from this class and overriding the {@link #getGraphNode()} method
- * 
- * 
- * 
- */
-@SuppressWarnings("javadoc")
-public class TriplesOrQuadsReader extends AbstractRdfReader<Quad, QuadWritable> {
-
-    @Override
-    protected RecordReader<LongWritable, QuadWritable> selectRecordReader(Lang lang) throws IOException {
-        if (!RDFLanguages.isQuads(lang) && !RDFLanguages.isTriples(lang))
-            throw new IOException(lang.getLabel() + " is not a RDF triples/quads format");
-
-        if (HadoopRdfIORegistry.hasQuadReader(lang)) {
-            // Supports quads directly
-            return HadoopRdfIORegistry.createQuadReader(lang);
-        } else {
-            // Try to create a triples reader and wrap upwards into quads
-            // This will throw an error if a triple reader is not available
-            return new TriplesToQuadsReader(HadoopRdfIORegistry.createTripleReader(lang));
-        }
-    }
-
-    /**
-     * Gets the graph node which represents the graph into which triples will be
-     * indicated to belong to when they are converting into quads.
-     * <p>
-     * Defaults to {@link Quad#defaultGraphNodeGenerated} which represents the
-     * default graph
-     * </p>
-     * 
-     * @return Graph node
-     */
-    protected Node getGraphNode() {
-        return Quad.defaultGraphNodeGenerated;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
deleted file mode 100644
index 96c4a65..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-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;
-
-/**
- * A record reader that reads triples from any RDF triples format
- */
-public class TriplesReader extends AbstractRdfReader<Triple, TripleWritable> {
-
-    @Override
-    protected RecordReader<LongWritable, TripleWritable> selectRecordReader(Lang lang) throws IOException {
-        if (!RDFLanguages.isTriples(lang))
-            throw new IOException(
-                    lang.getLabel()
-                            + " is not a RDF triples format, perhaps you wanted QuadsInputFormat or TriplesOrQuadsInputFormat instead?");
-
-        // This will throw an appropriate error if the language does not support
-        // triples
-        return HadoopRdfIORegistry.createTripleReader(lang);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
deleted file mode 100644
index a388f0e..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.readers;
-
-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.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A record reader that converts triples into quads by wrapping a
- * {@code RecordReader<LongWritable, TripleWritable>} implementation
- * 
- * 
- * 
- */
-public class TriplesToQuadsReader extends RecordReader<LongWritable, QuadWritable> {
-
-    private final RecordReader<LongWritable, TripleWritable> reader;
-    private Node graph;
-
-    /**
-     * Creates a new reader
-     * 
-     * @param reader
-     *            Triple reader
-     */
-    public TriplesToQuadsReader(RecordReader<LongWritable, TripleWritable> reader) {
-        this(reader, Quad.defaultGraphNodeGenerated);
-    }
-
-    /**
-     * Creates a new reader
-     * 
-     * @param reader
-     *            Triple reader
-     * @param graphNode
-     *            Graph node
-     */
-    public TriplesToQuadsReader(RecordReader<LongWritable, TripleWritable> reader, Node graphNode) {
-        if (reader == null)
-            throw new NullPointerException("reader cannot be null");
-        if (graphNode == null)
-            throw new NullPointerException("Graph node cannot be null");
-        this.reader = reader;
-        this.graph = graphNode;
-    }
-
-    @Override
-    public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
-        this.reader.initialize(split, context);
-    }
-
-    @Override
-    public final boolean nextKeyValue() throws IOException, InterruptedException {
-        return this.reader.nextKeyValue();
-    }
-
-    @Override
-    public final LongWritable getCurrentKey() throws IOException, InterruptedException {
-        return this.reader.getCurrentKey();
-    }
-
-    @Override
-    public final QuadWritable getCurrentValue() throws IOException, InterruptedException {
-        TripleWritable t = this.reader.getCurrentValue();
-        return new QuadWritable(new Quad(this.graph, t.get()));
-    }
-
-    @Override
-    public final float getProgress() throws IOException, InterruptedException {
-        return this.reader.getProgress();
-    }
-
-    @Override
-    public final void close() throws IOException {
-        this.reader.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
deleted file mode 100644
index 1b3f467..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.readers.jsonld;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
deleted file mode 100644
index 7cdea9e..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.readers.jsonld;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
deleted file mode 100644
index cef8ef1..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.readers.nquads;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractBlockBasedQuadReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record record for NQuads
- * <p>
- * This is a hybrid of the {@link NQuadsReader} and the
- * {@link WholeFileNQuadsReader} in that it does not process individual lines
- * rather it processes the inputs in blocks of lines parsing the whole block
- * rather than individual lines. This provides a compromise between the higher
- * parser setup of creating more parsers and the benefit of being able to split
- * input files over multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class BlockedNQuadsReader extends AbstractBlockBasedQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java
deleted file mode 100644
index e00e318..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.readers.nquads;
-
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractLineBasedQuadReader;
-import org.apache.jena.riot.lang.LangNQuads;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.tokens.Tokenizer;
-import org.apache.jena.riot.tokens.TokenizerFactory;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A record reader for NQuads
- * 
- * 
- * 
- */
-public class NQuadsReader extends AbstractLineBasedQuadReader {
-
-    @Override
-    protected Tokenizer getTokenizer(String line) {
-        return TokenizerFactory.makeTokenizerString(line);
-    }
-
-    @Override
-    protected Iterator<Quad> getQuadsIterator(Tokenizer tokenizer, ParserProfile profile) {
-        return new LangNQuads(tokenizer, profile, null);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
deleted file mode 100644
index 96e6f80..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.readers.nquads;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record record for NQuads
- * <p>
- * Unlike the {@link NQuadsReader} this processes files as a whole rather than
- * individual lines. This has the advantage of less parser setup overhead but
- * the disadvantage that the input cannot be split between multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class WholeFileNQuadsReader extends AbstractWholeFileQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java
deleted file mode 100644
index 7268d5a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.readers.ntriples;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractBlockBasedTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record record for NTriples
- * <p>
- * This is a hybrid of the {@link NTriplesReader} and the
- * {@link WholeFileNTriplesReader} in that it does not process individual lines
- * rather it processes the inputs in blocks of lines parsing the whole block
- * rather than individual lines. This provides a compromise between the higher
- * parser setup of creating more parsers and the benefit of being able to split
- * input files over multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class BlockedNTriplesReader extends AbstractBlockBasedTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
deleted file mode 100644
index bfc8503..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.readers.ntriples;
-
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractLineBasedTripleReader;
-import org.apache.jena.riot.lang.LangNTriples;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.tokens.Tokenizer;
-import org.apache.jena.riot.tokens.TokenizerFactory;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A record reader for NTriples
- * 
- * 
- * 
- */
-public class NTriplesReader extends AbstractLineBasedTripleReader {
-
-    @Override
-    protected Iterator<Triple> getTriplesIterator(Tokenizer tokenizer, ParserProfile profile) {
-        return new LangNTriples(tokenizer, profile, null);
-    }
-
-    @Override
-    protected Tokenizer getTokenizer(String line) {
-        return TokenizerFactory.makeTokenizerString(line);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
deleted file mode 100644
index c200d93..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.readers.ntriples;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record record for NTriples
- * <p>
- * Unlike the {@link NTriplesReader} this processes files as a whole rather than
- * individual lines. This has the advantage of less parser setup overhead but
- * the disadvantage that the input cannot be split between multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class WholeFileNTriplesReader extends AbstractWholeFileTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
deleted file mode 100644
index 009024b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.rdfjson;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for RDF/JSON files
- * 
- * 
- * 
- */
-public class RdfJsonReader extends AbstractWholeFileTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFJSON;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
deleted file mode 100644
index 9c374c6..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.rdfxml;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for RDF/XML files
- * 
- * 
- * 
- */
-public class RdfXmlReader extends AbstractWholeFileTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFXML;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
deleted file mode 100644
index 084b1ec..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.readers.thrift;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-public class ThriftQuadReader extends AbstractWholeFileQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
deleted file mode 100644
index 713bfa7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.readers.thrift;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-public class ThriftTripleReader extends AbstractWholeFileTripleReader {
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
deleted file mode 100644
index b1b0c3c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.trig;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for TriG files
- * 
- * 
- * 
- */
-public class TriGReader extends AbstractWholeFileQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIG;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
deleted file mode 100644
index 6873c64..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.trix;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for TriX files
- * 
- * 
- * 
- */
-public class TriXReader extends AbstractWholeFileQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIX;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
deleted file mode 100644
index b3fb377..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.turtle;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for Turtle files
- * 
- * 
- * 
- */
-public class TurtleReader extends AbstractWholeFileTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TURTLE;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
deleted file mode 100644
index f75542a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.thrift;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftQuadReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-public class ThriftQuadInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new ThriftQuadReader();
-    }
-
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
new file mode 100644
index 0000000..fd886a3
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+
+/**
+ * Tests for TriG output
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class BatchedTriGOutputTest extends AbstractQuadOutputFormatTests {
+
+    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+    static long $bs2 = 1000;
+    static long $bs3 = 100;
+    static long $bs4 = 1;
+
+    /**
+     * @return Test parameters
+     */
+    @Parameters
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
+    }
+
+    private final long batchSize;
+
+    /**
+     * Creates new tests
+     * 
+     * @param batchSize
+     *            Batch size
+     */
+    public BatchedTriGOutputTest(long batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trig";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIG;
+    }
+
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
+        return config;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
+        return new BatchedTriGOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
new file mode 100644
index 0000000..9b2b669
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+
+/**
+ * Tests for Turtle output
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class StreamedTriGOutputTest extends AbstractQuadOutputFormatTests {
+
+    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+    static long $bs2 = 1000;
+    static long $bs3 = 100;
+    static long $bs4 = 1;
+
+    /**
+     * @return Test parameters
+     */
+    @Parameters
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
+    }
+
+    private final long batchSize;
+
+    /**
+     * Creates new tests
+     * 
+     * @param batchSize
+     *            Batch size
+     */
+    public StreamedTriGOutputTest(long batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trig";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIG;
+    }
+    
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
+        return config;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
+        return new TriGOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
new file mode 100644
index 0000000..c9b3a26
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
@@ -0,0 +1,120 @@
+/*
+ * 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.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.RDFDataMgr;
+import org.junit.Assert;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ResIterator;
+import com.hp.hpl.jena.rdf.model.Resource;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Tests for TriG output with blank nodes
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class TriGBlankNodeOutputTests extends StreamedTriGOutputTest {
+
+	static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+	static long $bs2 = 1000;
+	static long $bs3 = 100;
+	static long $bs4 = 1;
+
+	/**
+	 * @return Test parameters
+	 */
+	@Parameters
+	public static Collection<Object[]> data() {
+		return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 },
+				{ $bs4 } });
+	}
+
+	/**
+	 * Creates new tests
+	 * 
+	 * @param batchSize
+	 *            Batch size
+	 */
+	public TriGBlankNodeOutputTests(long batchSize) {
+		super(batchSize);
+	}
+
+	@Override
+	protected Iterator<QuadWritable> generateTuples(int num) {
+		List<QuadWritable> qs = new ArrayList<QuadWritable>();
+		Node subject = NodeFactory.createAnon();
+		for (int i = 0; i < num; i++) {
+			Quad t = new Quad(
+					NodeFactory.createURI("http://example.org/graphs/" + i),
+					subject,
+					NodeFactory.createURI("http://example.org/predicate"),
+					NodeFactory.createLiteral(Integer.toString(i),
+							XSDDatatype.XSDinteger));
+			qs.add(new QuadWritable(t));
+		}
+		return qs.iterator();
+	}
+
+	@Override
+	protected void checkTuples(File f, long expected) {
+		super.checkTuples(f, expected);
+
+		Model m = RDFDataMgr.loadModel("file://" + f.getAbsolutePath(),
+				this.getRdfLanguage());
+		ResIterator iter = m.listSubjects();
+		Set<Node> subjects = new HashSet<Node>();
+		while (iter.hasNext()) {
+			Resource res = iter.next();
+			Assert.assertTrue(res.isAnon());
+			subjects.add(res.asNode());
+		}
+		// Should only be one subject unless the data was empty in which case
+		// there will be zero subjects
+		Assert.assertEquals(expected == 0 ? 0 : 1, subjects.size());
+	}
+
+	@Override
+	protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
+		return new TriGOutputFormat<NullWritable>();
+	}
+
+}

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

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
new file mode 100644
index 0000000..a6c4d70
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+
+/**
+ * Tests for Turtle output
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class BatchedTurtleOutputTest extends AbstractTripleOutputFormatTests {
+
+    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+    static long $bs2 = 1000;
+    static long $bs3 = 100;
+    static long $bs4 = 1;
+
+    /**
+     * @return Test parameters
+     */
+    @Parameters
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
+    }
+
+    private final long batchSize;
+
+    /**
+     * Creates new tests
+     * 
+     * @param batchSize
+     *            Batch size
+     */
+    public BatchedTurtleOutputTest(long batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".ttl";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TURTLE;
+    }
+    
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
+        return config;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new BatchedTurtleOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
new file mode 100644
index 0000000..d8843d3
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+
+/**
+ * Tests for Turtle output
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class StreamedTurtleOutputTest extends AbstractTripleOutputFormatTests {
+
+    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+    static long $bs2 = 1000;
+    static long $bs3 = 100;
+    static long $bs4 = 1;
+
+    /**
+     * @return Test parameters
+     */
+    @Parameters
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
+    }
+
+    private final long batchSize;
+
+    /**
+     * Creates new tests
+     * 
+     * @param batchSize
+     *            Batch size
+     */
+    public StreamedTurtleOutputTest(long batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".ttl";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TURTLE;
+    }
+    
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
+        return config;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new TurtleOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
new file mode 100644
index 0000000..8dcae4e
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
@@ -0,0 +1,118 @@
+/*
+ * 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.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.RDFDataMgr;
+import org.junit.Assert;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ResIterator;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * Tests for Turtle output with blank nodes
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class TurtleBlankNodeOutputTests extends StreamedTurtleOutputTest {
+
+	static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+	static long $bs2 = 1000;
+	static long $bs3 = 100;
+	static long $bs4 = 1;
+
+	/**
+	 * @return Test parameters
+	 */
+	@Parameters
+	public static Collection<Object[]> data() {
+		return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 },
+				{ $bs4 } });
+	}
+
+	/**
+	 * Creates new tests
+	 * 
+	 * @param batchSize
+	 *            Batch size
+	 */
+	public TurtleBlankNodeOutputTests(long batchSize) {
+		super(batchSize);
+	}
+
+	@Override
+	protected Iterator<TripleWritable> generateTuples(int num) {
+		List<TripleWritable> ts = new ArrayList<TripleWritable>();
+		Node subject = NodeFactory.createAnon();
+		for (int i = 0; i < num; i++) {
+			Triple t = new Triple(subject,
+					NodeFactory.createURI("http://example.org/predicate"),
+					NodeFactory.createLiteral(Integer.toString(i),
+							XSDDatatype.XSDinteger));
+			ts.add(new TripleWritable(t));
+		}
+		return ts.iterator();
+	}
+
+	@Override
+	protected void checkTuples(File f, long expected) {
+		super.checkTuples(f, expected);
+
+		Model m = RDFDataMgr.loadModel("file://" + f.getAbsolutePath(),
+				this.getRdfLanguage());
+		ResIterator iter = m.listSubjects();
+		Set<Node> subjects = new HashSet<Node>();
+		while (iter.hasNext()) {
+			Resource res = iter.next();
+			Assert.assertTrue(res.isAnon());
+			subjects.add(res.asNode());
+		}
+		// Should only be one subject unless the data was empty in which case
+		// there will be zero subjects
+		Assert.assertEquals(expected == 0 ? 0 : 1, subjects.size());
+	}
+
+	@Override
+	protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+		return new TurtleOutputFormat<NullWritable>();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
new file mode 100644
index 0000000..2eae232
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
@@ -0,0 +1,186 @@
+/*
+ * 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.registry;
+
+import java.io.IOException;
+import java.io.StringWriter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the {@link HadoopRdfIORegistry}
+ */
+public class TestHadoopRdfIORegistry {
+
+    private void testLang(Lang lang, boolean triples, boolean quads, boolean writesSupported) {
+        Assert.assertEquals(triples, HadoopRdfIORegistry.hasTriplesReader(lang));
+        Assert.assertEquals(quads, HadoopRdfIORegistry.hasQuadReader(lang));
+
+        // Some formats may be asymmetric
+        if (writesSupported) {
+            Assert.assertEquals(triples, HadoopRdfIORegistry.hasTriplesWriter(lang));
+            Assert.assertEquals(quads, HadoopRdfIORegistry.hasQuadWriter(lang));
+        } else {
+            Assert.assertFalse(HadoopRdfIORegistry.hasTriplesWriter(lang));
+            Assert.assertFalse(HadoopRdfIORegistry.hasQuadWriter(lang));
+        }
+
+        if (triples) {
+            // Check that triples are supported
+            RecordReader<LongWritable, TripleWritable> tripleReader;
+            try {
+                tripleReader = HadoopRdfIORegistry.createTripleReader(lang);
+                Assert.assertNotNull(tripleReader);
+            } catch (IOException e) {
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " can read triples but fails to produce a triple reader when asked: " + e.getMessage());
+            }
+
+            if (writesSupported) {
+                RecordWriter<NullWritable, TripleWritable> tripleWriter;
+                try {
+                    tripleWriter = HadoopRdfIORegistry.createTripleWriter(lang, new StringWriter(), new Configuration(
+                            false));
+                    Assert.assertNotNull(tripleWriter);
+                } catch (IOException e) {
+                    Assert.fail("Registry indicates that " + lang.getName()
+                            + " can write triples but fails to produce a triple writer when asked: " + e.getMessage());
+                }
+            }
+        } else {
+            // Check that triples are not supported
+            try {
+                HadoopRdfIORegistry.createTripleReader(lang);
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " cannot read triples but produced a triple reader when asked (error was expected)");
+            } catch (IOException e) {
+                // This is expected
+            }
+            try {
+                HadoopRdfIORegistry.createTripleWriter(lang, new StringWriter(), new Configuration(false));
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " cannot write triples but produced a triple write when asked (error was expected)");
+            } catch (IOException e) {
+                // This is expected
+            }
+        }
+
+        if (quads) {
+            // Check that quads are supported
+            RecordReader<LongWritable, QuadWritable> quadReader;
+            try {
+                quadReader = HadoopRdfIORegistry.createQuadReader(lang);
+                Assert.assertNotNull(quadReader);
+            } catch (IOException e) {
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " can read quads but fails to produce a quad reader when asked: " + e.getMessage());
+            }
+
+            if (writesSupported) {
+                RecordWriter<NullWritable, QuadWritable> quadWriter;
+                try {
+                    quadWriter = HadoopRdfIORegistry.createQuadWriter(lang, new StringWriter(),
+                            new Configuration(false));
+                    Assert.assertNotNull(quadWriter);
+                } catch (IOException e) {
+                    Assert.fail("Registry indicates that " + lang.getName()
+                            + " can write quads but fails to produce a triple writer when asked: " + e.getMessage());
+                }
+            }
+        } else {
+            try {
+                HadoopRdfIORegistry.createQuadReader(lang);
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " cannot read quads but produced a quad reader when asked (error was expected)");
+            } catch (IOException e) {
+                // This is expected
+            }
+            try {
+                HadoopRdfIORegistry.createQuadWriter(lang, new StringWriter(), new Configuration(false));
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " cannot write quads but produced a quad writer when asked (error was expected)");
+            } catch (IOException e) {
+                // This is expected
+            }
+        }
+    }
+
+    @Test
+    public void json_ld_registered() {
+        testLang(Lang.JSONLD, true, true, true);
+    }
+
+    @Test
+    public void nquads_registered() {
+        testLang(Lang.NQUADS, false, true, true);
+        testLang(Lang.NQ, false, true, true);
+    }
+
+    @Test
+    public void ntriples_registered() {
+        testLang(Lang.NTRIPLES, true, false, true);
+        testLang(Lang.NT, true, false, true);
+    }
+
+    @Test
+    public void rdf_json_registered() {
+        testLang(Lang.RDFJSON, true, false, true);
+    }
+
+    @Test
+    public void rdf_xml_registered() {
+        testLang(Lang.RDFXML, true, false, true);
+    }
+
+    @Test
+    public void rdf_thrift_registered() {
+        testLang(RDFLanguages.THRIFT, true, true, true);
+    }
+
+    @Test
+    public void trig_registered() {
+        testLang(Lang.TRIG, false, true, true);
+    }
+
+    @Test
+    public void trix_registered() {
+        testLang(Lang.TRIX, false, true, true);
+    }
+
+    @Test
+    public void turtle_registered() {
+        testLang(Lang.TURTLE, true, false, true);
+        testLang(Lang.TTL, true, false, true);
+        testLang(Lang.N3, true, false, true);
+    }
+
+    @Test
+    public void unregistered() {
+        testLang(Lang.RDFNULL, false, false, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/pom.xml b/jena-elephas/jena-elephas-mapreduce/pom.xml
new file mode 100644
index 0000000..aed59be
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/pom.xml
@@ -0,0 +1,87 @@
+<!--
+ 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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<groupId>org.apache.jena</groupId>
+		<artifactId>jena-elephas</artifactId>
+		<version>0.9.0-SNAPSHOT</version>
+	</parent>
+	<artifactId>jena-elephas-mapreduce</artifactId>
+	<name>Apache Jena - Elephas - Map/Reduce</name>
+	<description>Contains some basic Map/Reduce implementations for working with RDF on Hadoop</description>
+
+	<dependencies>
+		<!-- Internal Project Dependencies -->
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-hadoop-rdf-common</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<!-- Hadoop Dependencies -->
+		<!-- Note these will be provided on the Hadoop cluster hence the provided 
+			scope -->
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Jena dependencies -->
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-arq</artifactId>
+		</dependency>
+
+		<!-- Test Dependencies -->
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.mrunit</groupId>
+			<artifactId>mrunit</artifactId>
+			<scope>test</scope>
+			<classifier>hadoop2</classifier>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<!-- JAR plugin to ensure tests jar is built -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyMapper.java
new file mode 100644
index 0000000..306a697
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyMapper.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.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A mapper which discards the value replacing it with the key
+ * 
+ *
+ * @param <TKey> Key type
+ * @param <TValue> Value type
+ */
+public class KeyMapper<TKey, TValue> extends Mapper<TKey, TValue, TKey, TKey> {
+    private static final Logger LOG = LoggerFactory.getLogger(KeyMapper.class);
+
+    private boolean tracing = false;
+    
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOG.isTraceEnabled();
+    }
+
+    @Override
+    protected void map(TKey key, TValue value, Context context) throws IOException,
+            InterruptedException {
+        if (this.tracing) {
+            LOG.trace("Key = {}", key);
+        }
+        context.write(key, key);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyPlusNullMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyPlusNullMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyPlusNullMapper.java
new file mode 100644
index 0000000..a6e9a6a
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyPlusNullMapper.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A mapper which discards the value replacing it with a null
+ * 
+ *
+ * @param <TKey> Key type
+ * @param <TValue> Value type
+ */
+public class KeyPlusNullMapper<TKey, TValue> extends Mapper<TKey, TValue, TKey, NullWritable> {
+    private static final Logger LOG = LoggerFactory.getLogger(KeyPlusNullMapper.class);
+
+    private boolean tracing = false;
+    
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOG.isTraceEnabled();
+    }
+
+    @Override
+    protected void map(TKey key, TValue value, Context context) throws IOException,
+            InterruptedException {
+        if (this.tracing) {
+            LOG.trace("Key = {}", key);
+        }
+        context.write(key, NullWritable.get());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyReducer.java
new file mode 100644
index 0000000..7805f16
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/KeyReducer.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Reducer;
+
+/**
+ * A reducer that outputs a single pair consists of the key as both fields ignoring the values
+ * @author rvesse
+ *
+ * @param <TKey> Key
+ * @param <TValue> Value
+ */
+public class KeyReducer<TKey, TValue> extends Reducer<TKey, TValue, TKey, TKey> {
+
+    @Override
+    protected void reduce(TKey key, Iterable<TValue> values, Context context)
+            throws IOException, InterruptedException {
+        context.write(key, key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyMapper.java
new file mode 100644
index 0000000..7a48c1d
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyMapper.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A mapper which discards the value, moves the key to the value position and uses a null key
+ * 
+ *
+ * @param <TKey> Key type
+ * @param <TValue> Value type
+ */
+public class NullPlusKeyMapper<TKey, TValue> extends Mapper<TKey, TValue, NullWritable, TKey> {
+    private static final Logger LOG = LoggerFactory.getLogger(NullPlusKeyMapper.class);
+
+    private boolean tracing = false;
+    
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOG.isTraceEnabled();
+    }
+
+    @Override
+    protected void map(TKey key, TValue value, Context context) throws IOException,
+            InterruptedException {
+        if (this.tracing) {
+            LOG.trace("Key = {}", key);
+        }
+        context.write(NullWritable.get(), key);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyReducer.java
new file mode 100644
index 0000000..dfc6ec1
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusKeyReducer.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A reducer that outputs a single pair consists of a null as the key and the key as the value
+ * @author rvesse
+ *
+ * @param <TKey> Key
+ * @param <TValue> Value
+ */
+public class NullPlusKeyReducer<TKey, TValue> extends Reducer<TKey, TValue, NullWritable, TKey> {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(NullPlusKeyReducer.class);
+    private boolean tracing = false;
+    
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOGGER.isTraceEnabled();
+    }
+
+    @Override
+    protected void reduce(TKey key, Iterable<TValue> values, Context context)
+            throws IOException, InterruptedException {
+        if (this.tracing) {
+            LOGGER.trace("Input Key = {}", key);
+            Iterator<TValue> iter = values.iterator();
+            while (iter.hasNext()) {
+                LOGGER.trace("Input Value = {}", iter.next());
+            }
+        }
+        context.write(NullWritable.get(), key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueMapper.java
new file mode 100644
index 0000000..a5ac199
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueMapper.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A mapper which discards the key replacing it with a null leaving the value as is
+ * 
+ *
+ * @param <TKey> Key type
+ * @param <TValue> Value type
+ */
+public class NullPlusValueMapper<TKey, TValue> extends Mapper<TKey, TValue, NullWritable, TValue> {
+    private static final Logger LOG = LoggerFactory.getLogger(NullPlusValueMapper.class);
+
+    private boolean tracing = false;
+    
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOG.isTraceEnabled();
+    }
+
+    @Override
+    protected void map(TKey key, TValue value, Context context) throws IOException,
+            InterruptedException {
+        if (this.tracing) {
+            LOG.trace("Value = {}", value);
+        }
+        context.write(NullWritable.get(), value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueReducer.java
new file mode 100644
index 0000000..c6b270f
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/NullPlusValueReducer.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.mapreduce;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A reducer that outputs a pair for each value consisting of a null key and the
+ * value
+ * 
+ * @author rvesse
+ * 
+ * @param <TKey>
+ *            Key
+ * @param <TValue>
+ *            Value
+ */
+public class NullPlusValueReducer<TKey, TValue> extends Reducer<TKey, TValue, NullWritable, TValue> {
+    private static final Logger LOGGER = LoggerFactory.getLogger(NullPlusValueReducer.class);
+    private boolean tracing = false;
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOGGER.isTraceEnabled();
+    }
+
+    @Override
+    protected void reduce(TKey key, Iterable<TValue> values, Context context) throws IOException, InterruptedException {
+        if (this.tracing) {
+            LOGGER.trace("Input Key = {}", key);
+        }
+        Iterator<TValue> iter = values.iterator();
+        while (iter.hasNext()) {
+            TValue value = iter.next();
+            if (tracing) {
+                LOGGER.trace("Input Value = {}", value);
+            }
+            context.write(NullWritable.get(), value);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/RdfMapReduceConstants.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/RdfMapReduceConstants.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/RdfMapReduceConstants.java
new file mode 100644
index 0000000..6a8cf18
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/RdfMapReduceConstants.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+/**
+ * RDF Map/Reduce related constants
+ * 
+ * 
+ * 
+ */
+public class RdfMapReduceConstants {
+
+    /**
+     * Private constructor prevents instantiation
+     */
+    private RdfMapReduceConstants() {
+
+    }
+
+    /**
+     * Configuration key used to set whether the behaviour of the filter mappers
+     * is inverted. When enabled the filter mappers will invert their selection
+     * i.e. tuples that would normally be accepted will be rejected and vice
+     * versa.
+     */
+    public static final String FILTER_INVERT = "rdf.mapreduce.filter.invert";
+
+    /**
+     * Configuration key used to set a command separated list of predicate URIs
+     * to filter upon
+     */
+    public static final String FILTER_PREDICATE_URIS = "rdf.mapreduce.filter.predicate.uris";
+
+    /**
+     * Configuration key used to set a command separated list of subject URIs to
+     * filter upon
+     */
+    public static final String FILTER_SUBJECT_URIS = "rdf.mapreduce.filter.subject.uris";
+
+    /**
+     * Configuration key used to set a command separated list of object URIs to
+     * filter upon
+     */
+    public static final String FILTER_OBJECT_URIS = "rdf.mapreduce.filter.object.uris";
+
+    /**
+     * Configuration key used to set a command separated list of graph URIs to
+     * filter upon
+     */
+    public static final String FILTER_GRAPH_URIS = "rdf.mapreduce.filter.graph.uris";
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapMapper.java
new file mode 100644
index 0000000..ef518a9
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapMapper.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A mapper which swaps the key and value around
+ * 
+ *
+ * @param <TKey> Key type
+ * @param <TValue> Value type
+ */
+public class SwapMapper<TKey, TValue> extends Mapper<TKey, TValue, TValue, TKey> {
+    private static final Logger LOG = LoggerFactory.getLogger(SwapMapper.class);
+
+    private boolean tracing = false;
+    
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOG.isTraceEnabled();
+    }
+
+    @Override
+    protected void map(TKey key, TValue value, Context context) throws IOException,
+            InterruptedException {
+        if (this.tracing) {
+            LOG.trace("Key = {}", key);
+            LOG.trace("Value = {}", value);
+        }
+        context.write(value, key);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapReducer.java
new file mode 100644
index 0000000..e7e42a0
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/SwapReducer.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.mapreduce.Reducer;
+
+/**
+ * A reducer that swaps the key and value
+ * @author rvesse
+ *
+ * @param <TKey> Key
+ * @param <TValue> Value
+ */
+public class SwapReducer<TKey, TValue> extends Reducer<TKey, TValue, TValue, TKey> {
+
+    @Override
+    protected void reduce(TKey key, Iterable<TValue> values, Context context)
+            throws IOException, InterruptedException {
+        Iterator<TValue> iter = values.iterator();
+        while (iter.hasNext()) {
+            context.write(iter.next(), key);
+        }
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueMapper.java
new file mode 100644
index 0000000..23ae5f0
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueMapper.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.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A mapper which discards the key replacing it with the value
+ * 
+ *
+ * @param <TKey> Key type
+ * @param <TValue> Value type
+ */
+public class ValueMapper<TKey, TValue> extends Mapper<TKey, TValue, TValue, TValue> {
+    private static final Logger LOG = LoggerFactory.getLogger(ValueMapper.class);
+
+    private boolean tracing = false;
+    
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOG.isTraceEnabled();
+    }
+
+    @Override
+    protected void map(TKey key, TValue value, Context context) throws IOException,
+            InterruptedException {
+        if (this.tracing) {
+            LOG.trace("Value = {}", value);
+        }
+        context.write(value, value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValuePlusNullMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValuePlusNullMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValuePlusNullMapper.java
new file mode 100644
index 0000000..094fb2d
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValuePlusNullMapper.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A mapper which discards the key replacing it with the value and nulls out the value
+ * 
+ *
+ * @param <TKey> Key type
+ * @param <TValue> Value type
+ */
+public class ValuePlusNullMapper<TKey, TValue> extends Mapper<TKey, TValue, TValue, NullWritable> {
+    private static final Logger LOG = LoggerFactory.getLogger(ValuePlusNullMapper.class);
+
+    private boolean tracing = false;
+    
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOG.isTraceEnabled();
+    }
+
+    @Override
+    protected void map(TKey key, TValue value, Context context) throws IOException,
+            InterruptedException {
+        if (this.tracing) {
+            LOG.trace("Value = {}", value);
+        }
+        context.write(value, NullWritable.get());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueReducer.java
new file mode 100644
index 0000000..7d25799
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/ValueReducer.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.mapreduce.Reducer;
+
+/**
+ * A reducer that outputs a pair for each value consisting of the value as both the key and value
+ * @author rvesse
+ *
+ * @param <TKey> Key
+ * @param <TValue> Value
+ */
+public class ValueReducer<TKey, TValue> extends Reducer<TKey, TValue, TValue, TValue> {
+
+    @Override
+    protected void reduce(TKey key, Iterable<TValue> values, Context context)
+            throws IOException, InterruptedException {
+        Iterator<TValue> iter = values.iterator();
+        while (iter.hasNext()) {
+            TValue value = iter.next();
+            context.write(value, value);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducer.java
new file mode 100644
index 0000000..daf61d4
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducer.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
+import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Abstract reducer which takes in tuples grouped by some node and generating
+ * initial characteristic sets.
+ * <p>
+ * This produces the characteristic sets as both the key and value so that in a
+ * subsequent job the characteristic steps may be further combined together to
+ * total up the usage counts appropriately.
+ * </p>
+ * <p>
+ * It is important to note that the output from this mapper can be very large
+ * and since it typically needs to be written to HDFS before being processed by
+ * further jobs it is strongly recommended that you use appropriate output
+ * compression
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractCharacteristicSetGeneratingReducer<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Reducer<NodeWritable, T, CharacteristicSetWritable, NullWritable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractCharacteristicSetGeneratingReducer.class);
+
+    private boolean tracing = false;
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOG.isTraceEnabled();
+    }
+
+    @Override
+    protected void reduce(NodeWritable key, Iterable<T> values, Context context) throws IOException, InterruptedException {
+        Map<NodeWritable, CharacteristicWritable> characteristics = new TreeMap<NodeWritable, CharacteristicWritable>();
+
+        // Firstly need to find individual characteristics
+        Iterator<T> iter = values.iterator();
+        while (iter.hasNext()) {
+            T tuple = iter.next();
+            NodeWritable predicate = this.getPredicate(tuple);
+
+            if (characteristics.containsKey(predicate)) {
+                characteristics.get(predicate).increment();
+            } else {
+                characteristics.put(predicate, new CharacteristicWritable(predicate.get()));
+            }
+        }
+
+        // Then we need to produce all the possible characteristic sets based on
+        // this information
+        List<CharacteristicWritable> cs = new ArrayList<CharacteristicWritable>(characteristics.values());
+        if (cs.size() == 0)
+            return;
+        for (int i = 1; i <= cs.size(); i++) {
+            this.outputSets(cs, i, context);
+        }
+    }
+
+    /**
+     * Output all sets of a given size
+     * 
+     * @param cs
+     *            Characteristics
+     * @param perSet
+     *            Set size
+     * @param context
+     *            Context to output sets to
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected void outputSets(List<CharacteristicWritable> cs, int perSet, Context context) throws IOException,
+            InterruptedException {
+        if (perSet == 1) {
+            for (CharacteristicWritable c : cs) {
+                CharacteristicSetWritable set = new CharacteristicSetWritable(c);
+                context.write(set, NullWritable.get());
+                if (this.tracing) {
+                    LOG.trace("Key = {}", set);
+                }
+            }
+        } else if (perSet == cs.size()) {
+            CharacteristicSetWritable set = new CharacteristicSetWritable();
+            for (CharacteristicWritable c : cs) {
+                set.add(c);
+            }
+            context.write(set, NullWritable.get());
+            if (this.tracing) {
+                LOG.trace("Key = {}", set);
+            }
+        } else {
+            CharacteristicWritable[] members = new CharacteristicWritable[perSet];
+            this.combinations(cs, perSet, 0, members, context);
+        }
+    }
+
+    /**
+     * Calculate all available combinations of N elements from the given
+     * characteristics
+     * 
+     * @param cs
+     *            Characteristics
+     * @param len
+     *            Desired number of elements
+     * @param startPosition
+     *            Start position
+     * @param result
+     *            Result array to fill
+     * @param context
+     *            Context to write completed combinations to
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected final void combinations(List<CharacteristicWritable> cs, int len, int startPosition,
+            CharacteristicWritable[] result, Context context) throws IOException, InterruptedException {
+        if (len == 0) {
+            CharacteristicSetWritable set = new CharacteristicSetWritable(result);
+            context.write(set, NullWritable.get());
+            if (this.tracing) {
+                LOG.trace("Key = {}", set);
+            }
+            return;
+        }
+        for (int i = startPosition; i <= cs.size() - len; i++) {
+            result[result.length - len] = cs.get(i);
+            combinations(cs, len - 1, i + 1, result, context);
+        }
+    }
+
+    /**
+     * Gets the predicate for the tuple
+     * 
+     * @param tuple
+     *            Tuple
+     * @return
+     */
+    protected abstract NodeWritable getPredicate(T tuple);
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducer.java
new file mode 100644
index 0000000..e70698a
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducer.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.mapreduce.characteristics;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Reducer which takes in characteristic sets and sums up all their usage counts
+ * 
+ * 
+ */
+public class CharacteristicSetReducer extends
+        Reducer<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CharacteristicSetReducer.class);
+    private boolean tracing = false;
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.tracing = LOG.isTraceEnabled();
+    }
+
+    @Override
+    protected void reduce(CharacteristicSetWritable key, Iterable<CharacteristicSetWritable> values, Context context)
+            throws IOException, InterruptedException {
+        Iterator<CharacteristicSetWritable> iter = values.iterator();
+        CharacteristicSetWritable output = new CharacteristicSetWritable(0);
+
+        if (this.tracing) {
+            LOG.trace("Key = {}", key);
+        }
+
+        while (iter.hasNext()) {
+            CharacteristicSetWritable set = iter.next();
+            if (this.tracing) {
+                LOG.trace("Value = {}", set);
+            }
+            output.add(set);
+        }
+
+        context.write(output, NullWritable.get());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/QuadCharacteristicSetGeneratingReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/QuadCharacteristicSetGeneratingReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/QuadCharacteristicSetGeneratingReducer.java
new file mode 100644
index 0000000..d11cd56
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/QuadCharacteristicSetGeneratingReducer.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A reducer which converts quads grouped by some node into characteristic sets
+ * 
+ * 
+ * 
+ */
+public class QuadCharacteristicSetGeneratingReducer extends AbstractCharacteristicSetGeneratingReducer<Quad, QuadWritable> {
+
+    @Override
+    protected NodeWritable getPredicate(QuadWritable tuple) {
+        return new NodeWritable(tuple.get().getPredicate());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducer.java
new file mode 100644
index 0000000..6515c91
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducer.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A reducer which converts triples grouped by some node into characteristic
+ * sets
+ * 
+ * 
+ * 
+ */
+public class TripleCharacteristicSetGeneratingReducer extends AbstractCharacteristicSetGeneratingReducer<Triple, TripleWritable> {
+
+    @Override
+    protected NodeWritable getPredicate(TripleWritable tuple) {
+        return new NodeWritable(tuple.get().getPredicate());
+    }
+
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
new file mode 100644
index 0000000..f234127
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDTripleInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.jsonld.JsonLDTripleOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link RdfJsonInputFormat}
+ */
+public class JsonLdTripleBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.JSONLD;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".jsonld";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new JsonLDTripleInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new JsonLDTripleOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new JsonLDTripleInputFormat();
+    }
+
+    @Override
+    protected boolean respectsParserProfile() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
new file mode 100644
index 0000000..4c350c7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.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.input.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link NTriplesInputFormat}
+ */
+public class NTriplesBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".nt";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new NTriplesInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new NTriplesOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new NTriplesInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
new file mode 100644
index 0000000..2be1e0e
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.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.input.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.rdfjson.RdfJsonOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link RdfJsonInputFormat}
+ */
+public class RdfJsonBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.RDFJSON;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".rj";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new RdfJsonInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new RdfJsonOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new RdfJsonInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
new file mode 100644
index 0000000..d6f32a2
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.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.input.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftTripleInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.thrift.ThriftTripleOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+/**
+ * Tests blank node divergence when using the {@link RdfThriftInputFormat}
+ */
+public class RdfThriftBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".trdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new ThriftTripleInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new ThriftTripleOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new ThriftTripleInputFormat();
+    }
+
+    @Override
+    protected boolean respectsParserProfile() {
+        return false;
+    }
+    
+    @Override
+    protected boolean preservesBlankNodeIdentity() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
new file mode 100644
index 0000000..da70007
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.rdfxml.RdfXmlInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.rdfxml.RdfXmlOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link RdfXmlInputFormat}
+ */
+public class RdfXmlBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.RDFXML;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".rdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new RdfXmlInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new RdfXmlOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new RdfXmlInputFormat();
+    }
+
+    @Override
+    protected boolean respectsParserProfile() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
new file mode 100644
index 0000000..146c836
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.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.input.bnodes;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.input.turtle.TurtleInputFormat;
+import org.apache.jena.hadoop.rdf.io.output.turtle.TurtleOutputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * Tests blank node divergence when using the {@link TurtleInputFormat}
+ */
+public class TurtleBlankNodeTest extends AbstractTripleBlankNodeTests {
+
+    @Override
+    protected Lang getLanguage() {
+        return Lang.TURTLE;
+    }
+
+    @Override
+    protected String getInitialInputExtension() {
+        return ".ttl";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
+        return new TurtleInputFormat();
+    }
+
+    @Override
+    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
+        return new TurtleOutputFormat<>();
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
+        return new TurtleInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
new file mode 100644
index 0000000..1f18a95
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.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;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+
+
+/**
+ * 
+ * 
+ * @param <TValue>
+ * @param <T>
+ */
+public abstract class AbstractCompressedNodeTupleInputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        AbstractNodeTupleInputFormatTests<TValue, T> {
+    
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
+        return config;
+    }
+
+    @Override
+    protected OutputStream getOutputStream(File f) throws IOException {
+        CompressionCodec codec = this.getCompressionCodec();
+        if (codec instanceof Configurable) {
+            ((Configurable) codec).setConf(this.prepareConfiguration());
+        }
+        FileOutputStream fileOutput = new FileOutputStream(f, false);
+        return codec.createOutputStream(fileOutput);
+    }
+
+    /**
+     * Gets the compression codec to use
+     * 
+     * @return Compression codec
+     */
+    protected abstract CompressionCodec getCompressionCodec();
+
+    /**
+     * Indicates whether inputs can be split, defaults to false for compressed
+     * input tests
+     */
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
new file mode 100644
index 0000000..312aae7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.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.input.compressed;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for Quad input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedQuadsInputFormatTests extends
+        AbstractCompressedNodeTupleInputFormatTests<Quad, QuadWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected void generateTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateBadTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write("<http://broken\n".getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
+        boolean bad = false;
+        for (int i = 0; i < num; i++, bad = !bad) {
+            if (bad) {
+                output.write("<http://broken\n".getBytes(utf8));
+            } else {
+                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
+            }
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
new file mode 100644
index 0000000..f0f0caf
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.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.input.compressed;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for Triple input formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedTriplesInputFormatTests extends
+        AbstractCompressedNodeTupleInputFormatTests<Triple, TripleWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected void generateTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateBadTuples(OutputStream output, int num) throws IOException {
+        for (int i = 0; i < num; i++) {
+            output.write("<http://broken\n".getBytes(utf8));
+        }
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
+        boolean bad = false;
+        for (int i = 0; i < num; i++, bad = !bad) {
+            if (bad) {
+                output.write("<http://broken\n".getBytes(utf8));
+            } else {
+                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
+            }
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
new file mode 100644
index 0000000..be2b1d7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
@@ -0,0 +1,150 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
+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.query.Dataset;
+import com.hp.hpl.jena.query.DatasetFactory;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Property;
+import com.hp.hpl.jena.rdf.model.Resource;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for compressed whole file quad formats
+ * 
+ * 
+ */
+public abstract class AbstractCompressedWholeFileQuadInputFormatTests extends
+        AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
+    
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
+        return config;
+    }
+
+    @Override
+    protected OutputStream getOutputStream(File f) throws IOException {
+        CompressionCodec codec = this.getCompressionCodec();
+        if (codec instanceof Configurable) {
+            ((Configurable) codec).setConf(this.prepareConfiguration());
+        }
+        FileOutputStream fileOutput = new FileOutputStream(f, false);
+        return codec.createOutputStream(fileOutput);
+    }
+
+    /**
+     * Gets the compression codec to use
+     * 
+     * @return Compression codec
+     */
+    protected abstract CompressionCodec getCompressionCodec();
+
+    /**
+     * Indicates whether inputs can be split, defaults to false for compressed
+     * input tests
+     */
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+
+    private void writeTuples(Dataset ds, OutputStream output) {
+        RDFDataMgr.write(output, ds, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
+    }
+
+    /**
+     * Gets the RDF language to write out generated tuples in
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+    private void writeGoodTuples(OutputStream output, int num) throws IOException {
+        Dataset ds = DatasetFactory.createMem();
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num; i++) {
+            if (i % 100 == 0) {
+                ds.addNamedModel("http://example.org/graphs/" + (i / 100), m);
+                m = ModelFactory.createDefaultModel();
+            }
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        if (!m.isEmpty()) {
+            ds.addNamedModel("http://example.org/graphs/extra", m);
+        }
+        this.writeTuples(ds, output);
+    }
+
+    @Override
+    protected final void generateTuples(OutputStream output, int num) throws IOException {
+        this.writeGoodTuples(output, num);
+        output.close();
+    }
+
+    @Override
+    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
+        // Write good data
+        this.writeGoodTuples(output, num / 2);
+
+        // Write junk data
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num / 2; i++) {
+            output.write(junk);
+        }
+
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
new file mode 100644
index 0000000..56dd8ca
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
@@ -0,0 +1,144 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
+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.Triple;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Property;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * Abstract tests for compressed whole file triple formats
+ * 
+ * 
+ */
+public abstract class AbstractCompressedWholeFileTripleInputFormatTests extends
+        AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
+
+    private static final Charset utf8 = Charset.forName("utf-8");
+
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
+        return config;
+    }
+
+    @Override
+    protected OutputStream getOutputStream(File f) throws IOException {
+        CompressionCodec codec = this.getCompressionCodec();
+        if (codec instanceof Configurable) {
+            ((Configurable) codec).setConf(this.prepareConfiguration());
+        }
+        FileOutputStream fileOutput = new FileOutputStream(f, false);
+        return codec.createOutputStream(fileOutput);
+    }
+
+    /**
+     * Gets the compression codec to use
+     * 
+     * @return Compression codec
+     */
+    protected abstract CompressionCodec getCompressionCodec();
+
+    /**
+     * Indicates whether inputs can be split, defaults to false for compressed
+     * input tests
+     */
+    @Override
+    protected boolean canSplitInputs() {
+        return false;
+    }
+
+    private void writeTuples(Model m, OutputStream output) {
+        RDFDataMgr.write(output, m, this.getRdfLanguage());
+    }
+
+    /**
+     * Gets the RDF language to write out generated tuples in
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+    @Override
+    protected final void generateTuples(OutputStream output, int num) throws IOException {
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num; i++) {
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        this.writeTuples(m, output);
+        output.close();
+    }
+
+    @Override
+    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
+        // Write good data
+        Model m = ModelFactory.createDefaultModel();
+        Resource currSubj = m.createResource("http://example.org/subjects/0");
+        Property predicate = m.createProperty("http://example.org/predicate");
+        for (int i = 0; i < num / 2; i++) {
+            if (i % 10 == 0) {
+                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
+            }
+            m.add(currSubj, predicate, m.createTypedLiteral(i));
+        }
+        this.writeTuples(m, output);
+
+        // Write junk data
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num / 2; i++) {
+            output.write(junk);
+        }
+
+        output.flush();
+        output.close();
+    }
+
+    @Override
+    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
+        byte[] junk = "junk data\n".getBytes(utf8);
+        for (int i = 0; i < num; i++) {
+            output.write(junk);
+        }
+        output.flush();
+        output.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
new file mode 100644
index 0000000..d118f29
--- /dev/null
+++ b/jena-elephas/jena-elephas-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDQuadInputFormat;
+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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
new file mode 100644
index 0000000..acb9e08
--- /dev/null
+++ b/jena-elephas/jena-elephas-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDTripleInputFormat;
+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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java b/jena-elephas/jena-elephas-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-elephas/jena-elephas-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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java b/jena-elephas/jena-elephas-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-elephas/jena-elephas-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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java b/jena-elephas/jena-elephas-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-elephas/jena-elephas-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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java b/jena-elephas/jena-elephas-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-elephas/jena-elephas-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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java b/jena-elephas/jena-elephas-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-elephas/jena-elephas-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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java b/jena-elephas/jena-elephas-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-elephas/jena-elephas-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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
new file mode 100644
index 0000000..2b8447b
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.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.input.compressed.nquads;
+
+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.compressed.AbstractCompressedQuadsInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.nquads.NQuadsInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Abstract compressed NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedNQuadsInputFormatTests extends AbstractCompressedQuadsInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedNQuadsInputFormatTests(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 InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new NQuadsInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
new file mode 100644
index 0000000..326258a
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
@@ -0,0 +1,75 @@
+/*
+ * 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.nquads;
+
+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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.nquads.WholeFileNQuadsInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed whole file NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedWholeFileNQuadsInputFormatTests extends
+        AbstractCompressedWholeFileQuadInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedWholeFileNQuadsInputFormatTests(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.NQUADS;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new WholeFileNQuadsInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
new file mode 100644
index 0000000..62dc9ce
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+
+/**
+ * Tests for BZipped NQuads input
+ * 
+ * 
+ * 
+ */
+public class BZipppedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZipppedNQuadsInputTest() {
+        super(".nq.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
new file mode 100644
index 0000000..e3d670c
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped NQuads input
+ * 
+ * 
+ * 
+ */
+public class BZipppedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZipppedWholeFileNQuadsInputTest() {
+        super(".nq.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
new file mode 100644
index 0000000..5fec23e
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for deflated NQuads input
+ * 
+ * 
+ * 
+ */
+public class DeflatedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedNQuadsInputTest() {
+        super(".nq.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
new file mode 100644
index 0000000..a31472d
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for deflated NQuads input
+ * 
+ * 
+ * 
+ */
+public class DeflatedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedWholeFileNQuadsInputTest() {
+        super(".nq.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
new file mode 100644
index 0000000..2d40dec
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped NQuads input
+ * 
+ * 
+ * 
+ */
+public class GZippedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedNQuadsInputTest() {
+        super(".nq.gz", new GzipCodec());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
new file mode 100644
index 0000000..0f73eb7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.nquads;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped NQuads input
+ * 
+ * 
+ * 
+ */
+public class GZippedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedWholeFileNQuadsInputTest() {
+        super(".nq.gz", new GzipCodec());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
new file mode 100644
index 0000000..611d862
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
@@ -0,0 +1,53 @@
+/*
+ * 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.ntriples;
+
+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.ntriples.BlockedNTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Abstract compressed blocked NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedBlockedNTriplesInputFormatTests extends
+        AbstractCompressedWholeFileNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedBlockedNTriplesInputFormatTests(String ext, CompressionCodec codec) {
+        super(ext, codec);
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new BlockedNTriplesInputFormat();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
new file mode 100644
index 0000000..4d9aeba
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.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.input.compressed.ntriples;
+
+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.compressed.AbstractCompressedTriplesInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Abstract compressed NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedNTriplesInputFormatTests extends AbstractCompressedTriplesInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedNTriplesInputFormatTests(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 InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new NTriplesInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
new file mode 100644
index 0000000..7d4b510
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
@@ -0,0 +1,75 @@
+/*
+ * 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.ntriples;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.ntriples.WholeFileNTriplesInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed whole file NTriples input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedWholeFileNTriplesInputFormatTests extends
+        AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedWholeFileNTriplesInputFormatTests(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.NTRIPLES;
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new WholeFileNTriplesInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
new file mode 100644
index 0000000..de45f17
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
@@ -0,0 +1,37 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped blocked NTriples input
+ * 
+ * 
+ * 
+ */
+public class BZippedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedBlockedNTriplesInput() {
+        super(".nt.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
new file mode 100644
index 0000000..fb1ab8b
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.ntriples;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+
+/**
+ * Tests for BZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class BZippedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedNTriplesInputTest() {
+        super(".nt.bz2", new BZip2Codec());
+    }
+}


[10/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
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/input/jsonld/JsonLDQuadInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
new file mode 100644
index 0000000..2464946
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.jsonld;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
new file mode 100644
index 0000000..0e08a4b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.jsonld;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
new file mode 100644
index 0000000..6829c4d
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
@@ -0,0 +1,53 @@
+/*
+ * 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.nquads;
+
+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.AbstractNLineFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.nquads.BlockedNQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * NTriples input format where files are processed as blocks of lines rather
+ * than in a line based manner as with the {@link NQuadsInputFormat} or as
+ * whole files with the {@link WholeFileNQuadsInputFormat}
+ * <p>
+ * This provides a compromise between the higher parser setup of creating more
+ * parsers and the benefit of being able to split input files over multiple
+ * mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class BlockedNQuadsInputFormat extends AbstractNLineFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new BlockedNQuadsReader();
+    }
+
+}

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/input/nquads/NQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
new file mode 100644
index 0000000..802fbea
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.nquads;
+
+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.AbstractNLineFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.nquads.NQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * NQuads input format
+ * 
+ * 
+ * 
+ */
+public class NQuadsInputFormat extends AbstractNLineFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit arg0, TaskAttemptContext arg1)
+            throws IOException, InterruptedException {
+        return new NQuadsReader();
+    }
+
+}

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/input/nquads/WholeFileNQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java
new file mode 100644
index 0000000..128d079
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.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.input.nquads;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.nquads.WholeFileNQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * NQuads input format where files are processed as complete files rather than
+ * in a line based manner as with the {@link NQuadsInputFormat}
+ * <p>
+ * This has the advantage of less parser setup overhead but the disadvantage
+ * that the input cannot be split over multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class WholeFileNQuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new WholeFileNQuadsReader();
+    }
+
+}

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/input/ntriples/BlockedNTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
new file mode 100644
index 0000000..292167b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
@@ -0,0 +1,53 @@
+/*
+ * 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.ntriples;
+
+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.AbstractNLineFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.BlockedNTriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * NTriples input format where files are processed as blocks of lines rather
+ * than in a line based manner as with the {@link NTriplesInputFormat} or as
+ * whole files with the {@link WholeFileNTriplesInputFormat}
+ * <p>
+ * This provides a compromise between the higher parser setup of creating more
+ * parsers and the benefit of being able to split input files over multiple
+ * mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class BlockedNTriplesInputFormat extends AbstractNLineFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new BlockedNTriplesReader();
+    }
+
+}

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/input/ntriples/NTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
new file mode 100644
index 0000000..1694c87
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.ntriples;
+
+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.AbstractNLineFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.NTriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * NTriples input format
+ * 
+ * 
+ * 
+ */
+public class NTriplesInputFormat extends AbstractNLineFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit inputSplit, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new NTriplesReader();
+    }
+
+}

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/input/ntriples/WholeFileNTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java
new file mode 100644
index 0000000..31c1252
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.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.input.ntriples;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.WholeFileNTriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * NTriples input format where files are processed as complete files rather than
+ * in a line based manner as with the {@link NTriplesInputFormat}
+ * <p>
+ * This has the advantage of less parser setup overhead but the disadvantage
+ * that the input cannot be split over multiple mappers.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class WholeFileNTriplesInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new WholeFileNTriplesReader();
+    }
+
+}

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/input/rdfjson/RdfJsonInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
new file mode 100644
index 0000000..e5a7940
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.rdfjson;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.rdfjson.RdfJsonReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * RDF/JSON input format
+ * 
+ * 
+ * 
+ */
+public class RdfJsonInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new RdfJsonReader();
+    }
+
+}

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/input/rdfxml/RdfXmlInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
new file mode 100644
index 0000000..4deb925
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.rdfxml;
+
+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.AbstractWholeFileInputFormat;
+import org.apache.jena.hadoop.rdf.io.input.readers.rdfxml.RdfXmlReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * RDF/XML input format
+ * 
+ * 
+ * 
+ */
+public class RdfXmlInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new RdfXmlReader();
+    }
+
+}

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/input/readers/AbstractBlockBasedNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
new file mode 100644
index 0000000..56d031e
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
@@ -0,0 +1,344 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.util.BlockInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.ReaderRIOT;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+import org.apache.jena.riot.lang.PipedRDFStream;
+import org.apache.jena.riot.system.ParserProfile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract implementation for a record reader that reads records from blocks
+ * of files, this is a hybrid between {@link AbstractLineBasedNodeTupleReader}
+ * and {@link AbstractWholeFileNodeTupleReader} in that it can only be used by
+ * formats which can be split by lines but reduces the overhead by parsing the
+ * split as a whole rather than as individual lines.
+ * <p>
+ * The keys produced are the approximate position in the file at which a tuple
+ * was found and the values will be node tuples. Positions are approximate
+ * because they are recorded after the point at which the most recent tuple was
+ * parsed from the input thus they reflect the approximate position in the
+ * stream immediately after which the triple was found.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Value type
+ * @param <T>
+ *            Tuple type
+ */
+public abstract class AbstractBlockBasedNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractBlockBasedNodeTupleReader.class);
+    private CompressionCodec compressionCodecs;
+    private TrackableInputStream input;
+    private LongWritable key;
+    private long start, length;
+    private T tuple;
+    private TrackedPipedRDFStream<TValue> stream;
+    private PipedRDFIterator<TValue> iter;
+    private Thread parserThread;
+    private boolean finished = false;
+    private boolean ignoreBadTuples = true;
+    private boolean parserFinished = false;
+    private Throwable parserError = null;
+
+    @Override
+    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
+        LOG.debug("initialize({}, {})", genericSplit, context);
+
+        // Assuming file split
+        if (!(genericSplit instanceof FileSplit))
+            throw new IOException("This record reader only supports FileSplit inputs");
+        FileSplit split = (FileSplit) genericSplit;
+
+        // Configuration
+        Configuration config = context.getConfiguration();
+        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
+        if (this.ignoreBadTuples)
+            LOG.warn(
+                    "Configured to ignore bad tuples, parsing errors will be logged and further parsing aborted but no user visible errors will be thrown.  Consider setting {} to false to disable this behaviour",
+                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
+
+        // Figure out what portion of the file to read
+        start = split.getStart();
+        long end = start + split.getLength();
+        final Path file = split.getPath();
+        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
+        boolean readToEnd = end == totalLength;
+        CompressionCodecFactory factory = new CompressionCodecFactory(config);
+        this.compressionCodecs = factory.getCodec(file);
+
+        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { start, split.getLength(), totalLength }));
+
+        // Open the file and prepare the input stream
+        FileSystem fs = file.getFileSystem(config);
+        FSDataInputStream fileIn = fs.open(file);
+        this.length = split.getLength();
+        if (start > 0)
+            fileIn.seek(start);
+
+        if (this.compressionCodecs != null) {
+            // Compressed input
+            // For compressed input NLineInputFormat will have failed to find
+            // any line breaks and will give us a split from 0 -> (length - 1)
+            // Add 1 and re-verify readToEnd so we can abort correctly if ever
+            // given a partial split of a compressed file
+            end++;
+            readToEnd = end == totalLength;
+            if (start > 0 || !readToEnd)
+                throw new IOException("This record reader can only be used with compressed input where the split is a whole file");
+            input = new TrackedInputStream(this.compressionCodecs.createInputStream(fileIn));
+        } else {
+            // Uncompressed input
+
+            if (readToEnd) {
+                input = new TrackedInputStream(fileIn);
+            } else {
+                // Need to limit the portion of the file we are reading
+                input = new BlockInputStream(fileIn, split.getLength());
+            }
+        }
+
+        // Set up background thread for parser
+        iter = this.getPipedIterator();
+        this.stream = this.getPipedStream(iter, this.input);
+        ParserProfile profile = RdfIOUtils.createParserProfile(context, file);
+        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage(), profile);
+        this.parserThread = new Thread(parserRunnable);
+        this.parserThread.setDaemon(true);
+        this.parserThread.start();
+    }
+
+    /**
+     * Gets the RDF iterator to use
+     * 
+     * @return Iterator
+     */
+    protected abstract PipedRDFIterator<TValue> getPipedIterator();
+
+    /**
+     * Gets the RDF stream to parse to
+     * 
+     * @param iterator
+     *            Iterator
+     * @return RDF stream
+     */
+    protected abstract TrackedPipedRDFStream<TValue> getPipedStream(PipedRDFIterator<TValue> iterator, TrackableInputStream input);
+
+    /**
+     * Gets the RDF language to use for parsing
+     * 
+     * @return
+     */
+    protected abstract Lang getRdfLanguage();
+
+    /**
+     * Creates the runnable upon which the parsing will run
+     * 
+     * @param input
+     *            Input
+     * @param stream
+     *            Stream
+     * @param lang
+     *            Language to use for parsing
+     * @return Parser runnable
+     */
+    private Runnable createRunnable(@SuppressWarnings("rawtypes") final AbstractBlockBasedNodeTupleReader reader, final InputStream input,
+            final PipedRDFStream<TValue> stream, final Lang lang, final ParserProfile profile) {
+        return new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    ReaderRIOT riotReader = RDFDataMgr.createReader(lang);
+                    riotReader.setParserProfile(profile);
+                    riotReader.read(input, null, lang.getContentType(), stream, null);
+                    //RDFDataMgr.parse(stream, input, null, lang);
+                    reader.setParserFinished(null);
+                } catch (Throwable e) {
+                    reader.setParserFinished(e);
+                }
+            }
+        };
+    }
+
+    /**
+     * Sets the parser thread finished state
+     * 
+     * @param e
+     *            Error (if any)
+     */
+    private void setParserFinished(Throwable e) {
+        synchronized (this.parserThread) {
+            this.parserError = e;
+            this.parserFinished = true;
+        }
+    }
+
+    /**
+     * Waits for the parser thread to have reported as finished
+     * 
+     * @throws InterruptedException
+     */
+    private void waitForParserFinished() throws InterruptedException {
+        do {
+            synchronized (this.parserThread) {
+                if (this.parserFinished)
+                    return;
+            }
+            Thread.sleep(50);
+        } while (true);
+    }
+
+    /**
+     * Creates an instance of a writable tuple from the given tuple value
+     * 
+     * @param tuple
+     *            Tuple value
+     * @return Writable tuple
+     */
+    protected abstract T createInstance(TValue tuple);
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+        // Reuse key for efficiency
+        if (key == null) {
+            key = new LongWritable();
+        }
+
+        if (this.finished)
+            return false;
+
+        try {
+            if (this.iter.hasNext()) {
+                // Position will be relative to the start for the split we're
+                // processing
+                Long l = this.start + this.stream.getPosition();
+                if (l != null) {
+                    this.key.set(l);
+                    // For compressed input the actual length from which we
+                    // calculate progress is likely less than the actual
+                    // uncompressed length so we need to increment the
+                    // length as we go along
+                    // We always add 1 more than the current length because we
+                    // don't want to report 100% progress until we really have
+                    // finished
+                    if (this.compressionCodecs != null && l > this.length)
+                        this.length = l + 1;
+                }
+                this.tuple = this.createInstance(this.iter.next());
+                return true;
+            } else {
+                // Need to ensure that the parser thread has finished in order
+                // to determine whether we finished without error
+                this.waitForParserFinished();
+                if (this.parserError != null) {
+                    LOG.error("Error parsing block, aborting further parsing", this.parserError);
+                    if (!this.ignoreBadTuples)
+                        throw new IOException("Error parsing block at position " + (this.start + this.input.getBytesRead()) + ", aborting further parsing",
+                                this.parserError);
+                }
+
+                this.key = null;
+                this.tuple = null;
+                this.finished = true;
+                // This is necessary so that when compressed input is used we
+                // report 100% progress once we've reached the genuine end of
+                // the stream
+                if (this.compressionCodecs != null)
+                    this.length--;
+                return false;
+            }
+        } catch (IOException e) {
+            throw e;
+        } catch (Throwable e) {
+            // Failed to read the tuple on this line
+            LOG.error("Error parsing block, aborting further parsing", e);
+            if (!this.ignoreBadTuples) {
+                this.iter.close();
+                throw new IOException("Error parsing block at position " + (this.start + this.input.getBytesRead()) + ", aborting further parsing", e);
+            }
+            this.key = null;
+            this.tuple = null;
+            this.finished = true;
+            return false;
+        }
+    }
+
+    @Override
+    public LongWritable getCurrentKey() throws IOException, InterruptedException {
+        return this.key;
+    }
+
+    @Override
+    public T getCurrentValue() throws IOException, InterruptedException {
+        return this.tuple;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+        float progress = 0.0f;
+        if (this.key == null) {
+            // We've either not started or we've finished
+            progress = (this.finished ? 1.0f : 0.0f);
+        } else if (this.key.get() == Long.MIN_VALUE) {
+            // We don't have a position so we've either in-progress or finished
+            progress = (this.finished ? 1.0f : 0.5f);
+        } else {
+            // We're some way through the file
+            progress = (this.key.get() - this.start) / (float) this.length;
+        }
+        LOG.debug("getProgress() --> {}", progress);
+        return progress;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.iter.close();
+        this.input.close();
+        this.finished = true;
+    }
+
+}

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/input/readers/AbstractBlockBasedQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
new file mode 100644
index 0000000..2279444
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.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.input.readers;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedQuadsStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract record reader for whole file triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractBlockBasedQuadReader extends AbstractBlockBasedNodeTupleReader<Quad, QuadWritable> {
+
+    @Override
+    protected PipedRDFIterator<Quad> getPipedIterator() {
+        return new PipedRDFIterator<Quad>();
+    }
+
+    @Override
+    protected TrackedPipedRDFStream<Quad> getPipedStream(PipedRDFIterator<Quad> iterator, TrackableInputStream input) {
+        return new TrackedPipedQuadsStream(iterator, input);
+    }
+
+    @Override
+    protected QuadWritable createInstance(Quad tuple) {
+        return new QuadWritable(tuple);
+    }
+}

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/input/readers/AbstractBlockBasedTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
new file mode 100644
index 0000000..2afd329
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.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.input.readers;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedTriplesStream;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An abstract record reader for whole file triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractBlockBasedTripleReader extends AbstractBlockBasedNodeTupleReader<Triple, TripleWritable> {
+
+    @Override
+    protected PipedRDFIterator<Triple> getPipedIterator() {
+        return new PipedRDFIterator<Triple>();
+    }
+
+    @Override
+    protected TrackedPipedRDFStream<Triple> getPipedStream(PipedRDFIterator<Triple> iterator, TrackableInputStream input) {
+        return new TrackedPipedTriplesStream(iterator, input);
+    }
+
+    @Override
+    protected TripleWritable createInstance(Triple tuple) {
+        return new TripleWritable(tuple);
+    }
+}

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/input/readers/AbstractLineBasedNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
new file mode 100644
index 0000000..6c1abe9
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
@@ -0,0 +1,265 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.util.LineReader;
+import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.system.ParserProfile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract implementation of a record reader that reads records from line
+ * based tuple formats. This only supports reading from file splits currently.
+ * <p>
+ * The keys produced are the position of the line in the file and the values
+ * will be node tuples
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TValue>
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractLineBasedNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractLineBasedNodeTupleReader.class);
+    private CompressionCodecFactory compressionCodecs = null;
+    private long start, pos, end, estLength;
+    private int maxLineLength;
+    private LineReader in;
+    private LongWritable key = null;
+    private Text value = null;
+    private T tuple = null;
+    private ParserProfile profile = null;
+    private boolean ignoreBadTuples = true;
+
+    @Override
+    public final void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
+        LOG.debug("initialize({}, {})", genericSplit, context);
+
+        // Assuming file split
+        if (!(genericSplit instanceof FileSplit))
+            throw new IOException("This record reader only supports FileSplit inputs");
+        FileSplit split = (FileSplit) genericSplit;
+
+        // Configuration
+        profile = RdfIOUtils.createParserProfile(context, split.getPath());
+        Configuration config = context.getConfiguration();
+        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
+        if (this.ignoreBadTuples)
+            LOG.warn(
+                    "Configured to ignore bad tuples, parsing errors will be logged and the bad line skipped but no errors will be thrownConsider setting {} to false to disable this behaviour",
+                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
+
+        // Figure out what portion of the file to read
+        this.maxLineLength = config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE);
+        start = split.getStart();
+        end = start + split.getLength();
+        final Path file = split.getPath();
+        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
+        compressionCodecs = new CompressionCodecFactory(config);
+        final CompressionCodec codec = compressionCodecs.getCodec(file);
+
+        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { start, split.getLength(), totalLength }));
+
+        // Open the file and seek to the start of the split
+        FileSystem fs = file.getFileSystem(config);
+        FSDataInputStream fileIn = fs.open(file);
+        boolean skipFirstLine = false;
+        if (codec != null) {
+            // Compressed input
+            // For compressed input NLineInputFormat will have failed to find
+            // any line breaks and will give us a split from 0 -> (length - 1)
+            // Add 1 and verify we got complete split
+            if (totalLength > split.getLength() + 1)
+                throw new IOException("This record reader can only be used with compressed input where the split covers the whole file");
+            in = new LineReader(codec.createInputStream(fileIn), config);
+            estLength = end;
+            end = Long.MAX_VALUE;
+        } else {
+            // Uncompressed input
+            if (start != 0) {
+                skipFirstLine = true;
+                --start;
+                fileIn.seek(start);
+            }
+            in = new LineReader(fileIn, config);
+        }
+        // Skip first line and re-establish "start".
+        // This is to do with how line reader reads lines and how
+        // NLineInputFormat will provide the split information to use
+        if (skipFirstLine) {
+            start += in.readLine(new Text(), 0, (int) Math.min((long) Integer.MAX_VALUE, end - start));
+        }
+        this.pos = start;
+    }
+
+    /**
+     * Gets an iterator over the data on the current line
+     * 
+     * @param line
+     *            Line
+     * @param profile
+     *            Parser profile
+     * @return Iterator
+     */
+    protected abstract Iterator<TValue> getIterator(String line, ParserProfile profile);
+
+    /**
+     * Creates an instance of a writable tuple from the given tuple value
+     * 
+     * @param tuple
+     *            Tuple value
+     * @return Writable tuple
+     */
+    protected abstract T createInstance(TValue tuple);
+
+    @Override
+    public final boolean nextKeyValue() throws IOException, InterruptedException {
+        // Reuse key for efficiency
+        if (key == null) {
+            key = new LongWritable();
+        }
+
+        // Reset value which we use for reading lines
+        if (value == null) {
+            value = new Text();
+        }
+        tuple = null;
+
+        // Try to read the next valid line
+        int newSize = 0;
+        while (pos < end) {
+            // Read next line
+            newSize = in.readLine(value, maxLineLength, Math.max((int) Math.min(Integer.MAX_VALUE, end - pos), maxLineLength));
+
+            // Once we get an empty line we've reached the end of our input
+            if (newSize == 0) {
+                break;
+            }
+
+            // Update position, remember that where inputs are compressed we may
+            // be at a larger position then we expected because the length of
+            // the split is likely less than the length of the data once
+            // decompressed
+            key.set(pos);
+            pos += newSize;
+            if (pos > estLength)
+                estLength = pos + 1;
+
+            // Skip lines that exceed the line length limit that has been set
+            if (newSize >= maxLineLength) {
+                LOG.warn("Skipped oversized line of size {} at position {}", newSize, (pos - newSize));
+                continue;
+            }
+
+            // Attempt to read the tuple from current line
+            try {
+                Iterator<TValue> iter = this.getIterator(value.toString(), profile);
+                if (iter.hasNext()) {
+                    tuple = this.createInstance(iter.next());
+
+                    // If we reach here we've found a valid tuple so we can
+                    // break out of the loop
+                    break;
+                } else {
+                    // Empty line/Comment line
+                    LOG.debug("Valid line with no triple at position {}", (pos - newSize));
+                    continue;
+                }
+            } catch (Throwable e) {
+                // Failed to read the tuple on this line
+                LOG.error("Bad tuple at position " + (pos - newSize), e);
+                if (this.ignoreBadTuples)
+                    continue;
+                throw new IOException(String.format("Bad tuple at position %d", (pos - newSize)), e);
+            }
+        }
+        boolean result = this.tuple != null;
+
+        // End of input
+        if (newSize == 0) {
+            key = null;
+            value = null;
+            tuple = null;
+            result = false;
+            estLength = pos;
+        }
+        LOG.debug("nextKeyValue() --> {}", result);
+        return result;
+    }
+
+    @Override
+    public LongWritable getCurrentKey() throws IOException, InterruptedException {
+        LOG.debug("getCurrentKey() --> {}", key);
+        return key;
+    }
+
+    @Override
+    public T getCurrentValue() throws IOException, InterruptedException {
+        LOG.debug("getCurrentValue() --> {}", tuple);
+        return tuple;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+        float progress = 0.0f;
+        if (start != end) {
+            if (end == Long.MAX_VALUE) {
+                if (estLength == 0)
+                    return 1.0f;
+                // Use estimated length
+                progress = Math.min(1.0f, (pos - start) / (float) (estLength - start));
+            } else {
+                // Use actual length
+                progress = Math.min(1.0f, (pos - start) / (float) (end - start));
+            }
+        }
+        LOG.debug("getProgress() --> {}", progress);
+        return progress;
+    }
+
+    @Override
+    public void close() throws IOException {
+        LOG.debug("close()");
+        if (in != null) {
+            in.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/input/readers/AbstractLineBasedQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
new file mode 100644
index 0000000..ac93865
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.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.input.readers;
+
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.tokens.Tokenizer;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract reader for line based quad formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractLineBasedQuadReader extends AbstractLineBasedNodeTupleReader<Quad, QuadWritable> {
+
+    @Override
+    protected Iterator<Quad> getIterator(String line, ParserProfile profile) {
+        Tokenizer tokenizer = getTokenizer(line);
+        return getQuadsIterator(tokenizer, profile);
+    }
+
+    @Override
+    protected QuadWritable createInstance(Quad q) {
+        return new QuadWritable(q);
+    }
+
+    protected abstract Tokenizer getTokenizer(String line);
+
+    protected abstract Iterator<Quad> getQuadsIterator(Tokenizer tokenizer, ParserProfile profile);
+}

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/input/readers/AbstractLineBasedTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
new file mode 100644
index 0000000..a0232f5
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.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.input.readers;
+
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.tokens.Tokenizer;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An abstract record reader for line based triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractLineBasedTripleReader extends AbstractLineBasedNodeTupleReader<Triple, TripleWritable> {
+
+    @Override
+    protected Iterator<Triple> getIterator(String line, ParserProfile profile) {
+        Tokenizer tokenizer = getTokenizer(line);
+        return getTriplesIterator(tokenizer, profile);
+    }
+
+    @Override
+    protected TripleWritable createInstance(Triple t) {
+        return new TripleWritable(t);
+    }
+    
+    protected abstract Tokenizer getTokenizer(String line);
+
+    protected abstract Iterator<Triple> getTriplesIterator(Tokenizer tokenizer, ParserProfile profile);
+
+}

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/input/readers/AbstractRdfReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
new file mode 100644
index 0000000..d0ffed8
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
@@ -0,0 +1,108 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+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.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract record reader for arbitrary RDF which provides support for
+ * selecting the actual record reader to use based on detecting the RDF language
+ * from the file name
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractRdfReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        RecordReader<LongWritable, T> {
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractRdfReader.class);
+
+    private RecordReader<LongWritable, T> reader;
+
+    @Override
+    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException,
+            InterruptedException {
+        LOG.debug("initialize({}, {})", genericSplit, context);
+
+        // Assuming file split
+        if (!(genericSplit instanceof FileSplit))
+            throw new IOException("This record reader only supports FileSplit inputs");
+
+        // Find RDF language
+        FileSplit split = (FileSplit) genericSplit;
+        Path path = split.getPath();
+        Lang lang = RDFLanguages.filenameToLang(path.getName());
+        if (lang == null)
+            throw new IOException("There is no registered RDF language for the input file " + path.toString());
+
+        // Select the record reader and initialize
+        this.reader = this.selectRecordReader(lang);
+        this.reader.initialize(split, context);
+    }
+
+    /**
+     * Selects the appropriate record reader to use for the given RDF language
+     * 
+     * @param lang
+     *            RDF language
+     * @return Record reader
+     * @throws IOException
+     *             Should be thrown if no record reader can be selected
+     */
+    protected abstract RecordReader<LongWritable, T> selectRecordReader(Lang lang) throws IOException;
+
+    @Override
+    public final boolean nextKeyValue() throws IOException, InterruptedException {
+        return this.reader.nextKeyValue();
+    }
+
+    @Override
+    public final LongWritable getCurrentKey() throws IOException, InterruptedException {
+        return this.reader.getCurrentKey();
+    }
+
+    @Override
+    public final T getCurrentValue() throws IOException, InterruptedException {
+        return this.reader.getCurrentValue();
+    }
+
+    @Override
+    public final float getProgress() throws IOException, InterruptedException {
+        return this.reader.getProgress();
+    }
+
+    @Override
+    public final void close() throws IOException {
+        this.reader.close();
+    }
+
+}

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/input/readers/AbstractWholeFileNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
new file mode 100644
index 0000000..c2da3f7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
@@ -0,0 +1,328 @@
+/*
+ * 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.readers;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.ReaderRIOT;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+import org.apache.jena.riot.lang.PipedRDFStream;
+import org.apache.jena.riot.system.ParserProfile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract implementation for a record reader that reads records from whole
+ * files i.e. the whole file must be kept together to allow tuples to be
+ * successfully read. This only supports reading from file splits currently.
+ * <p>
+ * The keys produced are the approximate position in the file at which a tuple
+ * was found and the values will be node tuples. Positions are approximate
+ * because they are recorded after the point at which the most recent tuple was
+ * parsed from the input thus they reflect the approximate position in the
+ * stream immediately after which the triple was found.
+ * </p>
+ * <p>
+ * You should also be aware that with whole file formats syntax compressions in
+ * the format may mean that there are multiple triples produced with the same
+ * position and thus key.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TValue>
+ *            Value type
+ * @param <T>
+ *            Tuple type
+ */
+public abstract class AbstractWholeFileNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractLineBasedNodeTupleReader.class);
+    private CompressionCodec compressionCodecs;
+    private TrackedInputStream input;
+    private LongWritable key;
+    private long length;
+    private T tuple;
+    private TrackedPipedRDFStream<TValue> stream;
+    private PipedRDFIterator<TValue> iter;
+    private Thread parserThread;
+    private boolean finished = false;
+    private boolean ignoreBadTuples = true;
+    private boolean parserFinished = false;
+    private Throwable parserError = null;
+
+    @Override
+    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
+        LOG.debug("initialize({}, {})", genericSplit, context);
+
+        // Assuming file split
+        if (!(genericSplit instanceof FileSplit))
+            throw new IOException("This record reader only supports FileSplit inputs");
+        FileSplit split = (FileSplit) genericSplit;
+
+        // Configuration
+        Configuration config = context.getConfiguration();
+        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
+        if (this.ignoreBadTuples)
+            LOG.warn(
+                    "Configured to ignore bad tuples, parsing errors will be logged and further parsing aborted but no user visible errors will be thrown.  Consider setting {} to false to disable this behaviour",
+                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
+
+        // Figure out what portion of the file to read
+        if (split.getStart() > 0)
+            throw new IOException("This record reader requires a file split which covers the entire file");
+        final Path file = split.getPath();
+        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
+        CompressionCodecFactory factory = new CompressionCodecFactory(config);
+        this.compressionCodecs = factory.getCodec(file);
+
+        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { split.getStart(), split.getLength(),
+                totalLength }));
+
+        if (totalLength > split.getLength())
+            throw new IOException("This record reader requires a file split which covers the entire file");
+
+        // Open the file and prepare the input stream
+        FileSystem fs = file.getFileSystem(config);
+        FSDataInputStream fileIn = fs.open(file);
+        this.length = split.getLength();
+        if (this.compressionCodecs != null) {
+            // Compressed input
+            input = new TrackedInputStream(this.compressionCodecs.createInputStream(fileIn));
+        } else {
+            // Uncompressed input
+            input = new TrackedInputStream(fileIn);
+        }
+
+        // Set up background thread for parser
+        iter = this.getPipedIterator();
+        this.stream = this.getPipedStream(iter, this.input);
+        ParserProfile profile = RdfIOUtils.createParserProfile(context, file);
+        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage(), profile);
+        this.parserThread = new Thread(parserRunnable);
+        this.parserThread.setDaemon(true);
+        this.parserThread.start();
+    }
+
+    /**
+     * Gets the RDF iterator to use
+     * 
+     * @return Iterator
+     */
+    protected abstract PipedRDFIterator<TValue> getPipedIterator();
+
+    /**
+     * Gets the RDF stream to parse to
+     * 
+     * @param iterator
+     *            Iterator
+     * @return RDF stream
+     */
+    protected abstract TrackedPipedRDFStream<TValue> getPipedStream(PipedRDFIterator<TValue> iterator, TrackableInputStream input);
+
+    /**
+     * Gets the RDF language to use for parsing
+     * 
+     * @return
+     */
+    protected abstract Lang getRdfLanguage();
+
+    /**
+     * Creates the runnable upon which the parsing will run
+     * 
+     * @param input
+     *            Input
+     * @param stream
+     *            Stream
+     * @param lang
+     *            Language to use for parsing
+     * @return Parser runnable
+     */
+    private Runnable createRunnable(@SuppressWarnings("rawtypes") final AbstractWholeFileNodeTupleReader reader, final InputStream input,
+            final PipedRDFStream<TValue> stream, final Lang lang, final ParserProfile profile) {
+        return new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    ReaderRIOT riotReader = RDFDataMgr.createReader(lang);
+                    riotReader.setParserProfile(profile);
+                    riotReader.read(input, null, lang.getContentType(), stream, null);
+                    reader.setParserFinished(null);
+                } catch (Throwable e) {
+                    reader.setParserFinished(e);
+                }
+            }
+        };
+    }
+
+    /**
+     * Sets the parser thread finished state
+     * 
+     * @param e
+     *            Error (if any)
+     */
+    private void setParserFinished(Throwable e) {
+        synchronized (this.parserThread) {
+            this.parserError = e;
+            this.parserFinished = true;
+        }
+    }
+
+    /**
+     * Waits for the parser thread to have reported as finished
+     * 
+     * @throws InterruptedException
+     */
+    private void waitForParserFinished() throws InterruptedException {
+        do {
+            synchronized (this.parserThread) {
+                if (this.parserFinished)
+                    return;
+            }
+            Thread.sleep(50);
+        } while (true);
+    }
+
+    /**
+     * Creates an instance of a writable tuple from the given tuple value
+     * 
+     * @param tuple
+     *            Tuple value
+     * @return Writable tuple
+     */
+    protected abstract T createInstance(TValue tuple);
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+        // Reuse key for efficiency
+        if (key == null) {
+            key = new LongWritable();
+        }
+
+        if (this.finished)
+            return false;
+
+        try {
+            if (this.iter.hasNext()) {
+                Long l = this.stream.getPosition();
+                if (l != null) {
+                    this.key.set(l);
+                    // For compressed input the actual length from which we
+                    // calculate progress is likely less than the actual
+                    // uncompressed length so we may need to increment the
+                    // length as we go along
+                    // We always add 1 more than the current length because we
+                    // don't want to report 100% progress until we really have
+                    // finished
+                    if (this.compressionCodecs != null && l > this.length)
+                        this.length = l + 1;
+                }
+                this.tuple = this.createInstance(this.iter.next());
+                return true;
+            } else {
+                // Need to ensure that the parser thread has finished in order
+                // to determine whether we finished without error
+                this.waitForParserFinished();
+                if (this.parserError != null) {
+                    LOG.error("Error parsing whole file, aborting further parsing", this.parserError);
+                    if (!this.ignoreBadTuples)
+                        throw new IOException("Error parsing whole file at position " + this.input.getBytesRead() + ", aborting further parsing",
+                                this.parserError);
+
+                }
+
+                this.key = null;
+                this.tuple = null;
+                this.finished = true;
+                // This is necessary so that when compressed input is used we
+                // report 100% progress once we've reached the genuine end of
+                // the stream
+                if (this.compressionCodecs != null)
+                    this.length--;
+                return false;
+            }
+        } catch (Throwable e) {
+            // Failed to read the tuple on this line
+            LOG.error("Error parsing whole file, aborting further parsing", e);
+            if (!this.ignoreBadTuples) {
+                this.iter.close();
+                throw new IOException("Error parsing whole file at position " + this.input.getBytesRead() + ", aborting further parsing", e);
+            }
+            this.key = null;
+            this.tuple = null;
+            this.finished = true;
+            return false;
+        }
+    }
+
+    @Override
+    public LongWritable getCurrentKey() throws IOException, InterruptedException {
+        return this.key;
+    }
+
+    @Override
+    public T getCurrentValue() throws IOException, InterruptedException {
+        return this.tuple;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+        float progress = 0.0f;
+        if (this.key == null) {
+            // We've either not started or we've finished
+            progress = (this.finished ? 1.0f : 0.0f);
+        } else if (this.key.get() == Long.MIN_VALUE) {
+            // We don't have a position so we've either in-progress or finished
+            progress = (this.finished ? 1.0f : 0.5f);
+        } else {
+            // We're some way through the file
+            progress = this.key.get() / (float) this.length;
+        }
+        LOG.debug("getProgress() --> {}", progress);
+        return progress;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.iter.close();
+        this.input.close();
+        this.finished = true;
+    }
+
+}

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/input/readers/AbstractWholeFileQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
new file mode 100644
index 0000000..e525bea
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.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.input.readers;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedQuadsStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract record reader for whole file triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractWholeFileQuadReader extends AbstractWholeFileNodeTupleReader<Quad, QuadWritable> {
+
+    @Override
+    protected PipedRDFIterator<Quad> getPipedIterator() {
+        return new PipedRDFIterator<Quad>();
+    }
+
+    @Override
+    protected TrackedPipedRDFStream<Quad> getPipedStream(PipedRDFIterator<Quad> iterator, TrackableInputStream input) {
+        return new TrackedPipedQuadsStream(iterator, input);
+    }
+
+    @Override
+    protected QuadWritable createInstance(Quad tuple) {
+        return new QuadWritable(tuple);
+    }
+}

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/input/readers/AbstractWholeFileTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
new file mode 100644
index 0000000..8710b99
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.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.input.readers;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedTriplesStream;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.lang.PipedRDFIterator;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An abstract record reader for whole file triple formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractWholeFileTripleReader extends AbstractWholeFileNodeTupleReader<Triple, TripleWritable> {
+
+    @Override
+    protected PipedRDFIterator<Triple> getPipedIterator() {
+        return new PipedRDFIterator<Triple>();
+    }
+
+    @Override
+    protected TrackedPipedRDFStream<Triple> getPipedStream(PipedRDFIterator<Triple> iterator, TrackableInputStream input) {
+        return new TrackedPipedTriplesStream(iterator, input);
+    }
+
+    @Override
+    protected TripleWritable createInstance(Triple tuple) {
+        return new TripleWritable(tuple);
+    }
+}

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/input/readers/QuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
new file mode 100644
index 0000000..26b0a8b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+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;
+
+/**
+ * A record reader that reads triples from any RDF quads format
+ */
+public class QuadsReader extends AbstractRdfReader<Quad, QuadWritable> {
+
+    @Override
+    protected RecordReader<LongWritable, QuadWritable> selectRecordReader(Lang lang) throws IOException {
+        if (!RDFLanguages.isQuads(lang))
+            throw new IOException(
+                    lang.getLabel()
+                            + " is not a RDF quads format, perhaps you wanted TriplesInputFormat or TriplesOrQuadsInputFormat instead?");
+
+        // This will throw an appropriate error if the language does not support
+        // triples
+        return HadoopRdfIORegistry.createQuadReader(lang);
+    }
+
+}


[11/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
new file mode 100644
index 0000000..f29b156
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicSetWritable.java
@@ -0,0 +1,298 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * Represents a characteristic set which is comprised of a count of nodes for
+ * which the characteristic is applicable and a set of characteristics which
+ * represents the number of usages of predicates with those nodes
+ * 
+ * 
+ * 
+ */
+public class CharacteristicSetWritable implements WritableComparable<CharacteristicSetWritable> {
+
+    private Map<NodeWritable, CharacteristicWritable> characteristics = new TreeMap<NodeWritable, CharacteristicWritable>();
+    private LongWritable count = new LongWritable();
+
+    /**
+     * Creates a new empty characteristic set with the default count of 1
+     */
+    public CharacteristicSetWritable() {
+        this(1);
+    }
+
+    /**
+     * Creates a new characteristic set with the default count of 1 and the
+     * given characteristics
+     * 
+     * @param characteristics
+     *            Characteristics
+     */
+    public CharacteristicSetWritable(CharacteristicWritable... characteristics) {
+        this(1, characteristics);
+    }
+
+    /**
+     * Creates an empty characteristic set with the given count
+     * 
+     * @param count
+     *            Count
+     */
+    public CharacteristicSetWritable(long count) {
+        this(count, new CharacteristicWritable[0]);
+    }
+
+    /**
+     * Creates a new characteristic set
+     * 
+     * @param count
+     *            Count
+     * @param characteristics
+     *            Characteristics
+     */
+    public CharacteristicSetWritable(long count, CharacteristicWritable... characteristics) {
+        this.count.set(count);
+        for (CharacteristicWritable characteristic : characteristics) {
+            this.characteristics.put(characteristic.getNode(), characteristic);
+        }
+    }
+
+    /**
+     * Creates a new instance and reads its data from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static CharacteristicSetWritable read(DataInput input) throws IOException {
+        CharacteristicSetWritable set = new CharacteristicSetWritable();
+        set.readFields(input);
+        return set;
+    }
+
+    /**
+     * Gets the count
+     * 
+     * @return Count
+     */
+    public LongWritable getCount() {
+        return this.count;
+    }
+
+    /**
+     * Gets the characteristics
+     * 
+     * @return Characteristics
+     */
+    public Iterator<CharacteristicWritable> getCharacteristics() {
+        return this.characteristics.values().iterator();
+    }
+
+    /**
+     * Gets the size of the characteristic set
+     * 
+     * @return Size
+     */
+    public int size() {
+        return this.characteristics.size();
+    }
+
+    /**
+     * Adds a characteristic to the set merging it into the appropriate existing
+     * characteristic if applicable
+     * 
+     * @param characteristic
+     *            Characteristics
+     */
+    public void add(CharacteristicWritable characteristic) {
+        if (this.characteristics.containsKey(characteristic.getNode())) {
+            this.characteristics.get(characteristic.getNode()).increment(characteristic.getCount().get());
+        } else {
+            this.characteristics.put(characteristic.getNode(), characteristic);
+        }
+    }
+
+    /**
+     * Adds some characteristics to the set merging them with the appropriate
+     * existing characteristics if applicable
+     * 
+     * @param characteristics
+     */
+    public void add(CharacteristicWritable... characteristics) {
+        for (CharacteristicWritable characteristic : characteristics) {
+            this.add(characteristic);
+        }
+    }
+
+    /**
+     * Adds the contents of the other characteristic set to this characteristic
+     * set
+     * 
+     * @param set
+     *            Characteristic set
+     */
+    public void add(CharacteristicSetWritable set) {
+        this.increment(set.getCount().get());
+        Iterator<CharacteristicWritable> iter = set.getCharacteristics();
+        while (iter.hasNext()) {
+            this.add(iter.next());
+        }
+    }
+
+    /**
+     * Gets whether the set contains a characteristic for the given predicate
+     * 
+     * @param uri
+     *            Predicate URI
+     * @return True if contained in the set, false otherwise
+     */
+    public boolean hasCharacteristic(String uri) {
+        return this.hasCharacteristic(NodeFactory.createURI(uri));
+    }
+
+    /**
+     * Gets whether the set contains a characteristic for the given predicate
+     * 
+     * @param n
+     *            Predicate
+     * @return True if contained in the set, false otherwise
+     */
+    public boolean hasCharacteristic(Node n) {
+        return this.hasCharacteristic(new NodeWritable(n));
+    }
+
+    /**
+     * Gets whether the set contains a characteristic for the given predicate
+     * 
+     * @param n
+     *            Predicate
+     * @return True if contained in the set, false otherwise
+     */
+    public boolean hasCharacteristic(NodeWritable n) {
+        return this.characteristics.containsKey(n);
+    }
+
+    /**
+     * Increments the count by the given increment
+     * 
+     * @param l
+     *            Increment
+     */
+    public void increment(long l) {
+        this.count.set(this.count.get() + l);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        // Write size, then count, then characteristics
+        WritableUtils.writeVInt(output, this.characteristics.size());
+        this.count.write(output);
+        for (CharacteristicWritable characteristic : this.characteristics.values()) {
+            characteristic.write(output);
+        }
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        // Read size, then count, then characteristics
+        int size = WritableUtils.readVInt(input);
+        this.count.readFields(input);
+        this.characteristics.clear();
+        for (int i = 0; i < size; i++) {
+            CharacteristicWritable cw = CharacteristicWritable.read(input);
+            this.characteristics.put(cw.getNode(), cw);
+        }
+    }
+
+    @Override
+    public int compareTo(CharacteristicSetWritable cs) {
+        int size = this.characteristics.size();
+        int otherSize = cs.characteristics.size();
+        if (size < otherSize) {
+            return -1;
+        } else if (size > otherSize) {
+            return 1;
+        } else {
+            // Compare characteristics in turn
+            Iterator<CharacteristicWritable> iter = this.getCharacteristics();
+            Iterator<CharacteristicWritable> otherIter = cs.getCharacteristics();
+
+            int compare = 0;
+            while (iter.hasNext()) {
+                CharacteristicWritable c = iter.next();
+                CharacteristicWritable otherC = otherIter.next();
+                compare = c.compareTo(otherC);
+                if (compare != 0)
+                    return compare;
+            }
+            return compare;
+        }
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof CharacteristicSetWritable))
+            return false;
+        return this.compareTo((CharacteristicSetWritable) other) == 0;
+    }
+
+    @Override
+    public int hashCode() {
+        // Build a hash code from characteristics
+        if (this.characteristics.size() == 0)
+            return 0;
+        Iterator<CharacteristicWritable> iter = this.getCharacteristics();
+        int hash = 17;
+        while (iter.hasNext()) {
+            hash = hash * 31 + iter.next().hashCode();
+        }
+        return hash;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder builder = new StringBuilder();
+        builder.append("{ ");
+        builder.append(this.count.get());
+        Iterator<CharacteristicWritable> iter = this.getCharacteristics();
+        while (iter.hasNext()) {
+            builder.append(" , ");
+            builder.append(iter.next().toString());
+        }
+        builder.append(" }");
+        return builder.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
new file mode 100644
index 0000000..90fc7db
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/CharacteristicWritable.java
@@ -0,0 +1,160 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.hp.hpl.jena.graph.Node;
+
+/**
+ * Represents a characteristic for a single node and contains the node and a
+ * count associated with that node
+ * <p>
+ * Note that characteristics are compared based upon only the nodes and not
+ * their counts
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class CharacteristicWritable implements WritableComparable<CharacteristicWritable> {
+
+    private NodeWritable node = new NodeWritable();
+    private LongWritable count = new LongWritable();
+
+    /**
+     * Creates an empty characteristic writable
+     */
+    public CharacteristicWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a characteristic writable with the given node and the default
+     * count of 1
+     * 
+     * @param n
+     *            Node
+     */
+    public CharacteristicWritable(Node n) {
+        this(n, 1);
+    }
+
+    /**
+     * Creates a characteristic writable with the given node and count
+     * 
+     * @param n
+     *            Node
+     * @param count
+     *            Count
+     */
+    public CharacteristicWritable(Node n, long count) {
+        this.node.set(n);
+        this.count.set(count);
+    }
+
+    /**
+     * Creates a new instance and reads in its data from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static CharacteristicWritable read(DataInput input) throws IOException {
+        CharacteristicWritable cw = new CharacteristicWritable();
+        cw.readFields(input);
+        return cw;
+    }
+
+    /**
+     * Gets the node
+     * 
+     * @return Node
+     */
+    public NodeWritable getNode() {
+        return this.node;
+    }
+
+    /**
+     * Gets the count
+     * 
+     * @return Count
+     */
+    public LongWritable getCount() {
+        return this.count;
+    }
+
+    /**
+     * Increments the count by 1
+     */
+    public void increment() {
+        this.increment(1);
+    }
+
+    /**
+     * Increments the count by the given value
+     * 
+     * @param l
+     *            Value to increment by
+     */
+    public void increment(long l) {
+        this.count.set(this.count.get() + l);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        this.node.write(output);
+        this.count.write(output);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.node.readFields(input);
+        this.count.readFields(input);
+    }
+
+    @Override
+    public int compareTo(CharacteristicWritable o) {
+        return this.node.compareTo(o.node);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof CharacteristicWritable))
+            return false;
+        return this.compareTo((CharacteristicWritable) other) == 0;
+    }
+
+    @Override
+    public int hashCode() {
+        return this.node.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return "(" + this.node.toString() + ", " + this.count.toString() + ")";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.java
new file mode 100644
index 0000000..e06aac4
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeTupleWritable.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.types;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.jena.atlas.lib.Tuple;
+import com.hp.hpl.jena.graph.Node;
+
+/**
+ * A writable RDF tuple
+ * <p>
+ * Unlike the more specific {@link TripleWritable} and {@link QuadWritable} this
+ * class allows for arbitrary length tuples and does not restrict tuples to
+ * being of uniform size.
+ * </p>
+ * 
+ * 
+ * 
+ */
+public class NodeTupleWritable extends AbstractNodeTupleWritable<Tuple<Node>> {
+
+    /**
+     * Creates a new empty instance
+     */
+    public NodeTupleWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance with the given value
+     * 
+     * @param tuple
+     *            Tuple
+     */
+    public NodeTupleWritable(Tuple<Node> tuple) {
+        super(tuple);
+    }
+
+    /**
+     * Creates a new instance from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static NodeTupleWritable read(DataInput input) throws IOException {
+        NodeTupleWritable t = new NodeTupleWritable();
+        t.readFields(input);
+        return t;
+    }
+
+    @Override
+    protected Tuple<Node> createTuple(Node[] ns) {
+        return Tuple.create(ns);
+    }
+
+    @Override
+    protected Node[] createNodes(Tuple<Node> tuple) {
+        return tuple.tuple();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
new file mode 100644
index 0000000..cf00f8d
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/NodeWritable.java
@@ -0,0 +1,188 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
+import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
+import org.apache.jena.riot.thrift.TRDF;
+import org.apache.jena.riot.thrift.ThriftConvert;
+import org.apache.jena.riot.thrift.wire.RDF_Term;
+import org.apache.thrift.TException;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.util.NodeUtils;
+
+/**
+ * A writable for {@link Node} instances
+ * <p>
+ * This uses <a
+ * href="http://afs.github.io/rdf-thrift/rdf-binary-thrift.html">RDF Thrift</a>
+ * for the binary encoding of terms. The in-memory storage for this type is both
+ * a {@link Node} and a {@link RDF_Term} with lazy conversion between the two
+ * forms as necessary.
+ * </p>
+ */
+public class NodeWritable implements WritableComparable<NodeWritable> {
+
+    static {
+        WritableComparator.define(NodeWritable.class, new SimpleBinaryComparator());
+    }
+
+    private Node node;
+    private RDF_Term term = new RDF_Term();
+
+    /**
+     * Creates an empty writable
+     */
+    public NodeWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static NodeWritable read(DataInput input) throws IOException {
+        NodeWritable nw = new NodeWritable();
+        nw.readFields(input);
+        return nw;
+    }
+
+    /**
+     * Creates a new writable with the given value
+     * 
+     * @param n
+     *            Node
+     */
+    public NodeWritable(Node n) {
+        this.set(n);
+    }
+
+    /**
+     * Gets the node
+     * 
+     * @return Node
+     */
+    public Node get() {
+        // We may not have yet loaded the node
+        if (this.node == null) {
+            // If term is set to undefined then node is supposed to be null
+            if (this.term.isSet() && !this.term.isSetUndefined()) {
+                this.node = ThriftConvert.convert(this.term);
+            }
+        }
+        return this.node;
+    }
+
+    /**
+     * Sets the node
+     * 
+     * @param n
+     *            Node
+     */
+    public void set(Node n) {
+        this.node = n;
+        // Clear the term for now
+        // We only convert the Node to a term as and when we want to write it
+        // out in order to not waste effort if the value is never written out
+        this.term.clear();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        // Clear previous value
+        this.node = null;
+        this.term.clear();
+
+        // Read in the new value
+        int termLength = input.readInt();
+        byte[] buffer = new byte[termLength];
+        input.readFully(buffer);
+        try {
+            ThriftConverter.fromBytes(buffer, this.term);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+
+        // Note that we don't convert it back into a Node at this time
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        // May not yet have prepared the Thrift term
+        if (!this.term.isSet()) {
+            if (this.node == null) {
+                this.term.setUndefined(TRDF.UNDEF);
+            } else {
+                ThriftConvert.toThrift(this.node, null, this.term, false);
+            }
+        }
+
+        // Write out the Thrift term
+        byte[] buffer;
+        try {
+            buffer = ThriftConverter.toBytes(this.term);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        output.writeInt(buffer.length);
+        output.write(buffer);
+    }
+
+    @Override
+    public int compareTo(NodeWritable other) {
+        // Use get() rather than accessing the field directly because the node
+        // field is lazily instantiated from the Thrift term
+        return NodeUtils.compareRDFTerms(this.get(), other.get());
+    }
+
+    @Override
+    public String toString() {
+        // Use get() rather than accessing the field directly because the node
+        // field is lazily instantiated from the Thrift term
+        Node n = this.get();
+        if (n == null)
+            return "";
+        return n.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        // Use get() rather than accessing the field directly because the node
+        // field is lazily instantiated from the Thrift term
+        Node n = this.get();
+        return n != null ? this.get().hashCode() : 0;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof NodeWritable))
+            return false;
+        return this.compareTo((NodeWritable) other) == 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
new file mode 100644
index 0000000..3d9dd00
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/QuadWritable.java
@@ -0,0 +1,136 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
+import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
+import org.apache.jena.riot.thrift.ThriftConvert;
+import org.apache.jena.riot.thrift.wire.RDF_Quad;
+import org.apache.thrift.TException;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A writable quad
+ */
+public class QuadWritable extends AbstractNodeTupleWritable<Quad> {
+
+    static {
+        WritableComparator.define(QuadWritable.class, new SimpleBinaryComparator());
+    }
+
+    private RDF_Quad quad = new RDF_Quad();
+
+    /**
+     * Creates a new empty instance
+     */
+    public QuadWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance with the given value
+     * 
+     * @param q
+     *            Quad
+     */
+    public QuadWritable(Quad q) {
+        super(q);
+    }
+
+    /**
+     * Creates a new instance from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static QuadWritable read(DataInput input) throws IOException {
+        QuadWritable q = new QuadWritable();
+        q.readFields(input);
+        return q;
+    }
+
+    @Override
+    public void set(Quad tuple) {
+        super.set(tuple);
+        this.quad.clear();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.quad.clear();
+        int tripleLength = input.readInt();
+        byte[] buffer = new byte[tripleLength];
+        input.readFully(buffer);
+        try {
+            ThriftConverter.fromBytes(buffer, this.quad);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        this.setInternal(new Quad(ThriftConvert.convert(this.quad.getG()), ThriftConvert.convert(this.quad.getS()),
+                ThriftConvert.convert(this.quad.getP()), ThriftConvert.convert(this.quad.getO())));
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        if (this.get() == null)
+            throw new IOException(
+                    "Null quads cannot be written using this class, consider using NodeTupleWritable instead");
+
+        // May not have yet prepared the Thrift triple
+        if (!this.quad.isSetS()) {
+            Quad tuple = this.get();
+            this.quad.setG(ThriftConvert.convert(tuple.getGraph(), false));
+            this.quad.setS(ThriftConvert.convert(tuple.getSubject(), false));
+            this.quad.setP(ThriftConvert.convert(tuple.getPredicate(), false));
+            this.quad.setO(ThriftConvert.convert(tuple.getObject(), false));
+        }
+
+        byte[] buffer;
+        try {
+            buffer = ThriftConverter.toBytes(this.quad);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        output.writeInt(buffer.length);
+        output.write(buffer);
+    }
+
+    @Override
+    protected Quad createTuple(Node[] ns) {
+        if (ns.length != 4)
+            throw new IllegalArgumentException(String.format(
+                    "Incorrect number of nodes to form a quad - got %d but expected 4", ns.length));
+        return new Quad(ns[0], ns[1], ns[2], ns[3]);
+    }
+
+    @Override
+    protected Node[] createNodes(Quad tuple) {
+        return new Node[] { tuple.getGraph(), tuple.getSubject(), tuple.getPredicate(), tuple.getObject() };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
new file mode 100644
index 0000000..a17052b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/TripleWritable.java
@@ -0,0 +1,138 @@
+/*
+ * 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.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.jena.hadoop.rdf.types.comparators.SimpleBinaryComparator;
+import org.apache.jena.hadoop.rdf.types.converters.ThriftConverter;
+import org.apache.jena.riot.thrift.ThriftConvert;
+import org.apache.jena.riot.thrift.wire.RDF_Triple;
+import org.apache.thrift.TException;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A writable triple
+ * 
+ * 
+ * 
+ */
+public class TripleWritable extends AbstractNodeTupleWritable<Triple> {
+    
+    static {
+        WritableComparator.define(TripleWritable.class, new SimpleBinaryComparator());
+    }
+
+    private RDF_Triple triple = new RDF_Triple();
+
+    /**
+     * Creates a new instance using the default NTriples node formatter
+     */
+    public TripleWritable() {
+        this(null);
+    }
+
+    /**
+     * Creates a new instance with a given value that uses a specific node
+     * formatter
+     * 
+     * @param t
+     *            Triple
+     */
+    public TripleWritable(Triple t) {
+        super(t);
+    }
+
+    /**
+     * Creates a new instance from the given input
+     * 
+     * @param input
+     *            Input
+     * @return New instance
+     * @throws IOException
+     */
+    public static TripleWritable read(DataInput input) throws IOException {
+        TripleWritable t = new TripleWritable();
+        t.readFields(input);
+        return t;
+    }
+
+    @Override
+    public void set(Triple tuple) {
+        super.set(tuple);
+        this.triple.clear();
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.triple.clear();
+        int tripleLength = input.readInt();
+        byte[] buffer = new byte[tripleLength];
+        input.readFully(buffer);
+        try {
+            ThriftConverter.fromBytes(buffer, this.triple);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        this.setInternal(new Triple(ThriftConvert.convert(this.triple.getS()),
+                ThriftConvert.convert(this.triple.getP()), ThriftConvert.convert(this.triple.getO())));
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        if (this.get() == null)
+            throw new IOException(
+                    "Null triples cannot be written using this class, consider using NodeTupleWritable instead");
+        
+        // May not have yet prepared the Thrift triple
+        if (!this.triple.isSetS()) {
+            Triple tuple = this.get();
+            this.triple.setS(ThriftConvert.convert(tuple.getSubject(), false));
+            this.triple.setP(ThriftConvert.convert(tuple.getPredicate(), false));
+            this.triple.setO(ThriftConvert.convert(tuple.getObject(), false));
+        }
+
+        byte[] buffer;
+        try {
+            buffer = ThriftConverter.toBytes(this.triple);
+        } catch (TException e) {
+            throw new IOException(e);
+        }
+        output.writeInt(buffer.length);
+        output.write(buffer);
+    }
+
+    @Override
+    protected Triple createTuple(Node[] ns) {
+        if (ns.length != 3)
+            throw new IllegalArgumentException(String.format(
+                    "Incorrect number of nodes to form a triple - got %d but expected 3", ns.length));
+        return new Triple(ns[0], ns[1], ns[2]);
+    }
+
+    @Override
+    protected Node[] createNodes(Triple tuple) {
+        return new Node[] { tuple.getSubject(), tuple.getPredicate(), tuple.getObject() };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.java
new file mode 100644
index 0000000..6c46714
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/comparators/SimpleBinaryComparator.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.types.comparators;
+
+import org.apache.hadoop.io.WritableComparator;
+
+/**
+ * A general purpose comparator that may be used with any types which can be
+ * compared directly on their binary encodings
+ */
+public class SimpleBinaryComparator extends WritableComparator {
+
+    @Override
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+        return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
new file mode 100644
index 0000000..0675afc
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/main/java/org/apache/jena/hadoop/rdf/types/converters/ThriftConverter.java
@@ -0,0 +1,147 @@
+/*
+ * 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.types.converters;
+
+import java.io.ByteArrayOutputStream;
+
+import org.apache.jena.riot.thrift.wire.RDF_Quad;
+import org.apache.jena.riot.thrift.wire.RDF_Term;
+import org.apache.jena.riot.thrift.wire.RDF_Triple;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TIOStreamTransport;
+import org.apache.thrift.transport.TMemoryInputTransport;
+import org.apache.thrift.transport.TTransport;
+
+/**
+ * Helper for converting between the binary representation of Nodes, Triples and
+ * Quads and their Jena API equivalents
+ * 
+ */
+public class ThriftConverter {
+
+    private static ThreadLocal<TMemoryInputTransport> inputTransports = new ThreadLocal<>();
+    private static ThreadLocal<TProtocol> inputProtocols = new ThreadLocal<>();
+
+    private static ThreadLocal<ByteArrayOutputStream> outputStreams = new ThreadLocal<>();
+    private static ThreadLocal<TTransport> outputTransports = new ThreadLocal<>();
+    private static ThreadLocal<TProtocol> outputProtocols = new ThreadLocal<>();
+
+    private static TMemoryInputTransport getInputTransport() {
+        TMemoryInputTransport transport = inputTransports.get();
+        if (transport != null)
+            return transport;
+
+        transport = new TMemoryInputTransport();
+        inputTransports.set(transport);
+        return transport;
+    }
+
+    private static TProtocol getInputProtocol() {
+        TProtocol protocol = inputProtocols.get();
+        if (protocol != null)
+            return protocol;
+
+        protocol = new TCompactProtocol(getInputTransport());
+        inputProtocols.set(protocol);
+        return protocol;
+    }
+
+    private static ByteArrayOutputStream getOutputStream() {
+        ByteArrayOutputStream output = outputStreams.get();
+        if (output != null)
+            return output;
+
+        output = new ByteArrayOutputStream();
+        outputStreams.set(output);
+        return output;
+    }
+
+    private static TTransport getOutputTransport() {
+        TTransport transport = outputTransports.get();
+        if (transport != null)
+            return transport;
+
+        transport = new TIOStreamTransport(getOutputStream());
+        outputTransports.set(transport);
+        return transport;
+    }
+
+    private static TProtocol getOutputProtocol() {
+        TProtocol protocol = outputProtocols.get();
+        if (protocol != null)
+            return protocol;
+
+        protocol = new TCompactProtocol(getOutputTransport());
+        outputProtocols.set(protocol);
+        return protocol;
+    }
+
+    public static byte[] toBytes(RDF_Term term) throws TException {
+        ByteArrayOutputStream output = getOutputStream();
+        output.reset();
+
+        TProtocol protocol = getOutputProtocol();
+        term.write(protocol);
+
+        return output.toByteArray();
+    }
+
+    public static void fromBytes(byte[] bs, RDF_Term term) throws TException {
+        TMemoryInputTransport transport = getInputTransport();
+        transport.reset(bs);
+        TProtocol protocol = getInputProtocol();
+        term.read(protocol);
+    }
+
+    public static void fromBytes(byte[] buffer, RDF_Triple triple) throws TException {
+        TMemoryInputTransport transport = getInputTransport();
+        transport.reset(buffer);
+        TProtocol protocol = getInputProtocol();
+        triple.read(protocol);
+    }
+
+    public static byte[] toBytes(RDF_Triple triple) throws TException {
+        ByteArrayOutputStream output = getOutputStream();
+        output.reset();
+
+        TProtocol protocol = getOutputProtocol();
+        triple.write(protocol);
+
+        return output.toByteArray();
+    }
+
+    public static void fromBytes(byte[] buffer, RDF_Quad quad) throws TException {
+        TMemoryInputTransport transport = getInputTransport();
+        transport.reset(buffer);
+        TProtocol protocol = getInputProtocol();
+        quad.read(protocol);
+    }
+
+    public static byte[] toBytes(RDF_Quad quad) throws TException {
+        ByteArrayOutputStream output = getOutputStream();
+        output.reset();
+
+        TProtocol protocol = getOutputProtocol();
+        quad.write(protocol);
+
+        return output.toByteArray();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java b/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
new file mode 100644
index 0000000..7214b14
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
@@ -0,0 +1,210 @@
+/*
+ * 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.types;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
+import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * Tests for {@link CharacteristicWritable} and
+ * {@link CharacteristicSetWritable}
+ * 
+ * 
+ * 
+ */
+public class CharacteristicTests {
+
+    /**
+     * Checks whether a writable round trips successfully
+     * 
+     * @param cw
+     *            Characteristic writable
+     * @throws IOException
+     */
+    private void checkRoundTrip(CharacteristicWritable cw) throws IOException {
+        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        DataOutputStream output = new DataOutputStream(outputStream);
+        cw.write(output);
+
+        ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray());
+        DataInputStream input = new DataInputStream(inputStream);
+        CharacteristicWritable actual = CharacteristicWritable.read(input);
+        Assert.assertEquals(cw, actual);
+    }
+
+    /**
+     * Tests characteristic round tripping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_writable_01() throws IOException {
+        Node n = NodeFactory.createURI("http://example.org");
+        CharacteristicWritable expected = new CharacteristicWritable(n);
+        Assert.assertEquals(1, expected.getCount().get());
+
+        this.checkRoundTrip(expected);
+    }
+
+    /**
+     * Tests characteristic properties
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_writable_02() throws IOException {
+        Node n = NodeFactory.createURI("http://example.org");
+        CharacteristicWritable cw1 = new CharacteristicWritable(n);
+        CharacteristicWritable cw2 = new CharacteristicWritable(n, 100);
+        this.checkRoundTrip(cw1);
+        this.checkRoundTrip(cw2);
+
+        // Should still be equal since equality is only on the node not the
+        // count
+        Assert.assertEquals(cw1, cw2);
+    }
+
+    /**
+     * Tests characteristic properties
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_writable_03() throws IOException {
+        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
+        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
+        this.checkRoundTrip(cw1);
+        this.checkRoundTrip(cw2);
+
+        // Should not be equal as different nodes
+        Assert.assertNotEquals(cw1, cw2);
+    }
+
+    /**
+     * Checks that a writable round trips
+     * 
+     * @param set
+     *            Characteristic set
+     * @throws IOException
+     */
+    private void checkRoundTrip(CharacteristicSetWritable set) throws IOException {
+        // Test round trip
+        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        DataOutputStream output = new DataOutputStream(outputStream);
+        set.write(output);
+
+        ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray());
+        DataInputStream input = new DataInputStream(inputStream);
+        CharacteristicSetWritable actual = CharacteristicSetWritable.read(input);
+        Assert.assertEquals(set, actual);
+    }
+
+    /**
+     * Checks a characteristic set
+     * 
+     * @param set
+     *            Set
+     * @param expectedItems
+     *            Expected number of characteristics
+     * @param expectedCounts
+     *            Expected counts for characteristics
+     */
+    protected final void checkCharacteristicSet(CharacteristicSetWritable set, int expectedItems, long[] expectedCounts) {
+        Assert.assertEquals(expectedItems, set.size());
+        Assert.assertEquals(expectedItems, expectedCounts.length);
+        Iterator<CharacteristicWritable> iter = set.getCharacteristics();
+        int i = 0;
+        while (iter.hasNext()) {
+            CharacteristicWritable cw = iter.next();
+            Assert.assertEquals(expectedCounts[i], cw.getCount().get());
+            i++;
+        }
+    }
+
+    /**
+     * Tests characteristic sets
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_writable_01() throws IOException {
+        CharacteristicSetWritable set = new CharacteristicSetWritable();
+
+        // Add some characteristics
+        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
+        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
+        set.add(cw1);
+        set.add(cw2);
+        this.checkCharacteristicSet(set, 2, new long[] { 1, 1 });
+        this.checkRoundTrip(set);
+    }
+
+    /**
+     * Tests characteristic sets
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_writable_02() throws IOException {
+        CharacteristicSetWritable set = new CharacteristicSetWritable();
+
+        // Add some characteristics
+        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
+        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"), 2);
+        set.add(cw1);
+        set.add(cw2);
+        this.checkCharacteristicSet(set, 1, new long[] { 3 });
+        this.checkRoundTrip(set);
+    }
+    
+    /**
+     * Tests characteristic sets
+     * 
+     * @throws IOException
+     */
+    @Test
+    public void characteristic_set_writable_03() throws IOException {
+        CharacteristicSetWritable set1 = new CharacteristicSetWritable();
+        CharacteristicSetWritable set2 = new CharacteristicSetWritable();
+
+        // Add some characteristics
+        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
+        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
+        set1.add(cw1);
+        set2.add(cw2);
+        this.checkCharacteristicSet(set1, 1, new long[] { 1 });
+        this.checkCharacteristicSet(set2, 1, new long[] { 1 });
+        this.checkRoundTrip(set1);
+        this.checkRoundTrip(set2);
+        
+        Assert.assertNotEquals(set1, set2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java b/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
new file mode 100644
index 0000000..a70dfb0
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
@@ -0,0 +1,406 @@
+/*
+ * 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.types;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableComparable;
+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.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Tests for the various RDF types defined by the
+ * {@link org.apache.jena.hadoop.rdf.types} package
+ * 
+ * 
+ * 
+ */
+public class RdfTypesTest {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RdfTypesTest.class);
+
+    private ByteArrayOutputStream outputStream;
+    private ByteArrayInputStream inputStream;
+
+    /**
+     * Prepare for output
+     * 
+     * @return Data output
+     */
+    private DataOutput prepareOutput() {
+        this.outputStream = new ByteArrayOutputStream();
+        return new DataOutputStream(this.outputStream);
+    }
+
+    /**
+     * Prepare for input from the previously written output
+     * 
+     * @return Data Input
+     */
+    private DataInput prepareInput() {
+        this.inputStream = new ByteArrayInputStream(this.outputStream.toByteArray());
+        return new DataInputStream(this.inputStream);
+    }
+
+    /**
+     * Prepare for input from the given data
+     * 
+     * @param data
+     *            Data
+     * @return Data Input
+     */
+    @SuppressWarnings("unused")
+    private DataInput prepareInput(byte[] data) {
+        this.inputStream = new ByteArrayInputStream(data);
+        return new DataInputStream(this.inputStream);
+    }
+
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    private <T extends WritableComparable> void testWriteRead(T writable, T expected) throws IOException, InstantiationException, IllegalAccessException,
+            ClassNotFoundException {
+        // Write out data
+        DataOutput output = this.prepareOutput();
+        writable.write(output);
+
+        // Read back in data
+        DataInput input = this.prepareInput();
+        T actual = (T) Class.forName(writable.getClass().getName()).newInstance();
+        actual.readFields(input);
+
+        LOG.info("Original = " + writable.toString());
+        LOG.info("Round Tripped = " + actual.toString());
+
+        // Check equivalent
+        Assert.assertEquals(0, expected.compareTo(actual));
+    }
+    
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_null() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = null;
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+    
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    @Ignore
+    public void node_writable_variable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createVariable("x");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+    
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    @Ignore
+    public void node_writable_variable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createVariable("really-log-variable-name-asddsfr4545egfdgdfgfdgdtgvdg-dfgfdgdfgdfgdfg4-dfvdfgdfgdfgfdgfdgdfgdfgfdg");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_uri_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createURI("http://example.org");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_uri_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createURI("http://user:password@example.org/some/path?key=value#id");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("simple");
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("language", "en", null);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_03() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("string", XSDDatatype.XSDstring);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_04() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("1234", XSDDatatype.XSDinteger);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_05() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("123.4", XSDDatatype.XSDdecimal);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_06() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("12.3e4", XSDDatatype.XSDdouble);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_literal_07() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createLiteral("true", XSDDatatype.XSDboolean);
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_bnode_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createAnon();
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+    }
+
+    /**
+     * Basic node writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void node_writable_bnode_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Node n = NodeFactory.createAnon();
+        NodeWritable nw = new NodeWritable(n);
+        testWriteRead(nw, nw);
+        NodeWritable nw2 = new NodeWritable(n);
+        testWriteRead(nw2, nw2);
+
+        Assert.assertEquals(0, nw.compareTo(nw2));
+    }
+
+    /**
+     * Basic triple writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void triple_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Triple t = new Triple(NodeFactory.createURI("http://example"), NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral("value"));
+        TripleWritable tw = new TripleWritable(t);
+        testWriteRead(tw, tw);
+    }
+    
+    /**
+     * Basic triple writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void triple_writable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Triple t = new Triple(NodeFactory.createAnon(), NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral("value"));
+        TripleWritable tw = new TripleWritable(t);
+        testWriteRead(tw, tw);
+    }
+
+    /**
+     * Basic quad writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void quad_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Quad q = new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://example"), NodeFactory.createURI("http://predicate"),
+                NodeFactory.createLiteral("value"));
+        QuadWritable qw = new QuadWritable(q);
+        testWriteRead(qw, qw);
+    }
+    
+    /**
+     * Basic quad writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void quad_writable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Quad q = new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createAnon(), NodeFactory.createURI("http://predicate"),
+                NodeFactory.createLiteral("value"));
+        QuadWritable qw = new QuadWritable(q);
+        testWriteRead(qw, qw);
+    }
+
+    /**
+     * Basic tuple writable round tripping test
+     * 
+     * @throws IOException
+     * @throws InstantiationException
+     * @throws IllegalAccessException
+     * @throws ClassNotFoundException
+     */
+    @Test
+    public void tuple_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
+        Tuple<Node> t = Tuple.createTuple(NodeFactory.createURI("http://one"), NodeFactory.createURI("http://two"), NodeFactory.createLiteral("value"),
+                NodeFactory.createLiteral("foo"), NodeFactory.createURI("http://three"));
+        NodeTupleWritable tw = new NodeTupleWritable(t);
+        testWriteRead(tw, tw);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/pom.xml b/jena-hadoop-rdf/jena-elephas-io/pom.xml
new file mode 100644
index 0000000..2be37f9
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/pom.xml
@@ -0,0 +1,67 @@
+<!--
+   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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.jena</groupId>
+    <artifactId>jena-elephas</artifactId>
+    <version>0.9.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>jena-elephas-io</artifactId>
+  <name>Apache Jena - Elephas - I/O</name>
+  <description>RDF Input/Output formats library for Hadoop</description>
+
+	<!-- Note that versions are managed by parent POMs -->
+  <dependencies>
+		<!-- Internal Project Dependencies -->
+    <dependency>
+      <groupId>org.apache.jena</groupId>
+      <artifactId>jena-hadoop-rdf-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+	<!-- Hadoop Dependencies -->
+	<!-- Note these will be provided on the Hadoop cluster hence the provided 
+		scope -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+	<!-- Jena dependencies -->
+    <dependency>
+      <groupId>org.apache.jena</groupId>
+      <artifactId>jena-arq</artifactId>
+    </dependency>
+
+	<!-- Test Dependencies -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>
\ 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/HadoopIOConstants.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java
new file mode 100644
index 0000000..5c1b41c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.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;
+
+/**
+ * Hadoop IO related constants
+ * 
+ * 
+ * 
+ */
+public class HadoopIOConstants {
+
+    /**
+     * Private constructor prevents instantiation
+     */
+    private HadoopIOConstants() {
+    }
+
+    /**
+     * Map Reduce configuration setting for max line length
+     */
+    public static final String MAX_LINE_LENGTH = "mapreduce.input.linerecordreader.line.maxlength";
+
+    /**
+     * Run ID
+     */
+    public static final String RUN_ID = "runId";
+    
+    /**
+     * Compression codecs to use
+     */
+    public static final String IO_COMPRESSION_CODECS = "io.compression.codecs";
+}

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/RdfIOConstants.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
new file mode 100644
index 0000000..27c2bb2
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+/**
+ * RDF IO related constants
+ * 
+ * 
+ * 
+ */
+public class RdfIOConstants {
+
+    /**
+     * Private constructor prevents instantiation
+     */
+    private RdfIOConstants() {
+    }
+
+    /**
+     * Configuration key used to set whether bad tuples are ignored. This is the
+     * default behaviour, when explicitly set to {@code false} bad tuples will
+     * result in {@link IOException} being thrown by the relevant record
+     * readers.
+     */
+    public static final String INPUT_IGNORE_BAD_TUPLES = "rdf.io.input.ignore-bad-tuples";
+
+    /**
+     * Configuration key used to set the batch size used for RDF output formats
+     * that take a batched writing approach. Default value is given by the
+     * constant {@link #DEFAULT_OUTPUT_BATCH_SIZE}.
+     */
+    public static final String OUTPUT_BATCH_SIZE = "rdf.io.output.batch-size";
+
+    /**
+     * Default batch size for batched output formats
+     */
+    public static final long DEFAULT_OUTPUT_BATCH_SIZE = 10000;
+
+    /**
+     * Configuration key used to control behaviour with regards to how blank
+     * nodes are handled.
+     * <p>
+     * The default behaviour is that blank nodes are file scoped which is what
+     * the RDF specifications require.
+     * </p>
+     * <p>
+     * However in the case of a multi-stage pipeline this behaviour can cause
+     * blank nodes to diverge over several jobs and introduce spurious blank
+     * nodes over time. This is described in <a
+     * href="https://issues.apache.org/jira/browse/JENA-820">JENA-820</a> and
+     * enabling this flag for jobs in your pipeline allow you to work around
+     * this problem.
+     * </p>
+     * <h3>Warning</h3> You should only enable this flag for jobs that take in
+     * RDF output originating from previous jobs since our normal blank node
+     * allocation policy ensures that blank nodes will be file scoped and unique
+     * over all files (barring unfortunate hasing collisions). If you enable
+     * this for jobs that take in RDF originating from other sources you may
+     * incorrectly conflate blank nodes that are supposed to distinct and
+     * separate nodes.
+     */
+    public static final String GLOBAL_BNODE_IDENTITY = "rdf.io.input.bnodes.global-identity";
+}

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/input/AbstractNLineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
new file mode 100644
index 0000000..1fcb030
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
@@ -0,0 +1,70 @@
+/*
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract line based input format that reuses the machinery from
+ * {@link NLineInputFormat} to calculate the splits
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Value type
+ */
+public abstract class AbstractNLineFileInputFormat<TKey, TValue> extends FileInputFormat<TKey, TValue> {
+    
+    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractNLineFileInputFormat.class);
+
+    /**
+     * Logically splits the set of input files for the job, splits N lines of
+     * the input as one split.
+     * 
+     * @see FileInputFormat#getSplits(JobContext)
+     */
+    public final List<InputSplit> getSplits(JobContext job) throws IOException {
+        boolean debug = LOGGER.isDebugEnabled();
+        if (debug && FileInputFormat.getInputDirRecursive(job)) {
+            LOGGER.debug("Recursive searching for input data is enabled");
+        }
+        
+        List<InputSplit> splits = new ArrayList<InputSplit>();
+        int numLinesPerSplit = NLineInputFormat.getNumLinesPerSplit(job);
+        for (FileStatus status : listStatus(job)) {
+            if (debug) {
+                LOGGER.debug("Determining how to split input file/directory {}", status.getPath());
+            }
+            splits.addAll(NLineInputFormat.getSplitsForFile(status, job.getConfiguration(), numLinesPerSplit));
+        }
+        return splits;
+    }
+}

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/input/AbstractWholeFileInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
new file mode 100644
index 0000000..e561cdb
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+/**
+ * Abstract implementation of a while file input format where each file is a
+ * single split
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Value type
+ */
+public abstract class AbstractWholeFileInputFormat<TKey, TValue> extends FileInputFormat<TKey, TValue> {
+
+    @Override
+    protected final boolean isSplitable(JobContext context, Path filename) {
+        return 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/input/QuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
new file mode 100644
index 0000000..b8fdbd5
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.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.QuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * RDF input format that can handle any RDF quads format that ARQ supports
+ * selecting the format to use for each file based upon the file extension
+ * 
+ * 
+ * 
+ */
+public class QuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new QuadsReader();
+    }
+
+}

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/input/TriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
new file mode 100644
index 0000000..03f394a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.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.TriplesReader;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+/**
+ * RDF input format that can handle any RDF triples format that ARQ supports
+ * selecting the format to use for each file based upon the file extension
+ */
+public class TriplesInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
+
+    @Override
+    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TriplesReader();
+    }
+
+}

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/input/TriplesOrQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
new file mode 100644
index 0000000..bfd643e
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.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.TriplesOrQuadsReader;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * RDF input format that can handle any RDF triple/quads format that ARQ
+ * supports selecting the format to use for each file based upon the file
+ * extension. Triples are converted into quads in the default graph.
+ * 
+ * 
+ * 
+ */
+public class TriplesOrQuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
+
+    @Override
+    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        return new TriplesOrQuadsReader();
+    }
+
+}


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

Posted by rv...@apache.org.
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);
+
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
new file mode 100644
index 0000000..50b8bcf
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.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.input.nquads;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for NQuads input
+ * 
+ * 
+ * 
+ */
+public class WholeFileNQuadsInputTest extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nq";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new WholeFileNQuadsInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
new file mode 100644
index 0000000..ab92873
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.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.input.ntriples;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractBlockedTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for blocked NTriples input
+ * 
+ *
+ */
+public class BlockedNTriplesInputTest extends AbstractBlockedTripleInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nt";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new BlockedNTriplesInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
new file mode 100644
index 0000000..24fb731
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.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.input.ntriples;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractTriplesInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Tests for the {@link NTriplesInputFormat}
+ * 
+ * 
+ * 
+ */
+public class NTriplesInputTest extends AbstractTriplesInputFormatTests {
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new NTriplesInputFormat();
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nt";
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
new file mode 100644
index 0000000..fcec570
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.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.input.ntriples;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for the {@link NTriplesInputFormat}
+ * 
+ * 
+ * 
+ */
+public class WholeFileNTriplesInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new WholeFileNTriplesInputFormat();
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nt";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+    
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
new file mode 100644
index 0000000..4731832
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.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.input.rdfjson;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for RDF/JSON input
+ * 
+ * 
+ * 
+ */
+public class RdfJsonInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFJSON;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".rj";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new RdfJsonInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
new file mode 100644
index 0000000..f8edcc4
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.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.input.rdfxml;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for RDF/XML input
+ * 
+ * 
+ * 
+ */
+public class RdfXmlInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFXML;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".rdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new RdfXmlInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
new file mode 100644
index 0000000..8d79295
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.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.input.thrift;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+
+/**
+ * Tests for JSON-LD input
+ * 
+ *
+ */
+public class ThriftQuadInputTest extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new ThriftQuadInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
new file mode 100644
index 0000000..6b5e0b7
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.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.input.thrift;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+
+/**
+ * Tests for JSON-LD input
+ * 
+ *
+ */
+public class ThriftTripleInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new ThriftTripleInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
new file mode 100644
index 0000000..1fad0dc
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.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.input.trig;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for TriG input
+ * 
+ *
+ */
+public class TriGInputTest extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIG;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trig";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new TriGInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
new file mode 100644
index 0000000..4a3a66a
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.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.input.trix;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for TriX input
+ * 
+ *
+ */
+public class TriXInputTest extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIX;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trix";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new TriXInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
new file mode 100644
index 0000000..e6211ba
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.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.input.turtle;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for turtle input format
+ * 
+ * 
+ * 
+ */
+public class TurtleInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected final String getFileExtension() {
+        return ".ttl";
+    }
+
+    @Override
+    protected final Lang getRdfLanguage() {
+        return Lang.TURTLE;
+    }
+    
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new TurtleInputFormat();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
new file mode 100644
index 0000000..9532d56
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
@@ -0,0 +1,701 @@
+/*
+ * 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.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Abstract tests for {@link TrackableInputStream} implementations
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractTrackableInputStreamTests {
+
+    protected static final int KILO = 1024;
+    protected static final int BYTES_PER_KB = KILO;
+    protected static final int BYTES_PER_MB = BYTES_PER_KB * KILO;
+
+    /**
+     * Gets the instance to test using the given input as the stream to track
+     * 
+     * @param input
+     *            Input Stream
+     * @return Trackable Input Stream
+     */
+    protected abstract TrackableInputStream getInstance(InputStream input);
+
+    /**
+     * Generates an input stream containing the given number of bytes
+     * 
+     * @param length
+     *            Number of bytes
+     * @return Input stream
+     */
+    protected final InputStream generateData(int length) {
+        ByteArrayOutputStream output = new ByteArrayOutputStream(length);
+        byte b = (byte) 'b';
+        for (int i = 0; i < length; i++) {
+            output.write(b);
+        }
+        return new ByteArrayInputStream(output.toByteArray());
+    }
+
+    protected final void testSingleByteRead(int length) throws IOException {
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input);
+        long count = 0;
+        while (trackable.read() >= 0) {
+            count++;
+        }
+        Assert.assertEquals(length, count);
+        Assert.assertEquals(length, trackable.getBytesRead());
+        trackable.close();
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_single_01() throws IOException {
+        this.testSingleByteRead(0);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_single_02() throws IOException {
+        this.testSingleByteRead(100);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_single_03() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_single_04() throws IOException {
+        // 1 MB
+        this.testSingleByteRead(BYTES_PER_MB);
+    }
+
+    protected final void testMultiByteRead(int length, int bufferSize) throws IOException {
+        if (bufferSize < 1)
+            throw new IllegalArgumentException("bufferSize must be >= 1");
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input);
+        long count = 0;
+        byte[] buffer = new byte[bufferSize];
+        long read;
+        do {
+            read = trackable.read(buffer);
+            if (read > 0)
+                count += read;
+        } while (read >= 0);
+        Assert.assertEquals(length, count);
+        Assert.assertEquals(length, trackable.getBytesRead());
+        trackable.close();
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_01() throws IOException {
+        this.testMultiByteRead(0, 1);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_02() throws IOException {
+        this.testMultiByteRead(0, 16);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_03() throws IOException {
+        this.testMultiByteRead(0, BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_04() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_05() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_06() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_07() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_08() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_09() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB);
+    }
+
+    protected final void testMultiByteRead(int length, int bufferSize, int readSize) throws IOException {
+        if (bufferSize < 1)
+            throw new IllegalArgumentException("bufferSize must be >= 1");
+        if (readSize < 1 || readSize > bufferSize)
+            throw new IllegalArgumentException("readSize must be >= 1 and <= bufferSize");
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input);
+        long count = 0;
+        byte[] buffer = new byte[bufferSize];
+        long read;
+        do {
+            read = trackable.read(buffer, 0, readSize);
+            if (read > 0)
+                count += read;
+        } while (read >= 0);
+        Assert.assertEquals(length, count);
+        Assert.assertEquals(length, trackable.getBytesRead());
+        trackable.close();
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_01() throws IOException {
+        this.testMultiByteRead(0, 1, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_02() throws IOException {
+        this.testMultiByteRead(0, 16, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_03() throws IOException {
+        this.testMultiByteRead(0, 16, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_04() throws IOException {
+        this.testMultiByteRead(0, BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_05() throws IOException {
+        this.testMultiByteRead(0, BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_06() throws IOException {
+        this.testMultiByteRead(0, BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_07() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 1, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_08() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 16, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_09() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 16, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_10() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_11() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_12() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_13() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 1, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_14() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 16, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_15() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 16, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_16() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_17() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_18() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    protected final void testSkip(int length, long skipSize) throws IOException {
+        if (skipSize < 1)
+            throw new IllegalArgumentException("skipSize must be >= 1");
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input);
+        long count = 0;
+        long skipped;
+        do {
+            skipped = trackable.skip(skipSize);
+            if (skipped > 0)
+                count += skipped;
+        } while (skipped > 0);
+        Assert.assertEquals(length, count);
+        Assert.assertEquals(length, trackable.getBytesRead());
+        trackable.close();
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_01() throws IOException {
+        this.testSkip(0, 1);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_02() throws IOException {
+        this.testSkip(100, 1);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_03() throws IOException {
+        this.testSkip(100, 16);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_04() throws IOException {
+        this.testSkip(100, BYTES_PER_KB);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_05() throws IOException {
+        // 1KB
+        this.testSkip(BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_06() throws IOException {
+        // 1KB
+        this.testSkip(BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_07() throws IOException {
+        // 1KB
+        this.testSkip(BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_08() throws IOException {
+        // 1KB
+        this.testSkip(BYTES_PER_KB, BYTES_PER_MB);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_09() throws IOException {
+        // 1 MB
+        this.testSkip(BYTES_PER_MB, 1);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_10() throws IOException {
+        // 1 MB
+        this.testSkip(BYTES_PER_MB, 16);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_11() throws IOException {
+        // 1 MB
+        this.testSkip(BYTES_PER_MB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_12() throws IOException {
+        // 1 MB
+        this.testSkip(BYTES_PER_MB, BYTES_PER_MB);
+    }
+
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_01() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(-1, trackable.read());
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_02() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(0, trackable.read(new byte[0]));
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_03() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(-1, trackable.read(new byte[1]));
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_04() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(0, trackable.read(new byte[16], 0, 0));
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_05() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(-1, trackable.read(new byte[16], 0, 8));
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_06() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(0, trackable.skip(0));
+    }
+    
+    /**
+     * Tests exceptions are thrown trying to perform actions after closing the
+     * input
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_07() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(0, trackable.skip(1));
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
new file mode 100644
index 0000000..f8819bc
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
@@ -0,0 +1,240 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.jena.hadoop.rdf.io.input.util.BlockInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the {@link BlockInputStream}
+ * 
+ * 
+ * 
+ */
+public class BlockInputStreamTest extends AbstractTrackableInputStreamTests {
+
+    @Override
+    protected TrackableInputStream getInstance(InputStream input) {
+        return new BlockInputStream(input, Long.MAX_VALUE);
+    }
+
+    /**
+     * Gets an instance of a block input stream
+     * 
+     * @param input
+     *            Underlying input stream
+     * @param limit
+     *            Limit on bytes to read
+     * @return Block input stream
+     */
+    protected BlockInputStream getInstance(InputStream input, long limit) {
+        return new BlockInputStream(input, limit);
+    }
+    
+    protected final void testSingleByteRead(int length, long limit) throws IOException {
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input, limit);
+        long count = 0;
+        while (trackable.read() >= 0) {
+            count++;
+        }
+        int expected = (int) Math.min(length, limit);
+        Assert.assertEquals(expected, count);
+        Assert.assertEquals(expected, trackable.getBytesRead());
+        trackable.close();
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_01() throws IOException {
+        this.testSingleByteRead(0, 0);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_02() throws IOException {
+        this.testSingleByteRead(100, 0);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_03() throws IOException {
+        this.testSingleByteRead(100, 50);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_04() throws IOException {
+        this.testSingleByteRead(100, 100);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_05() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, 1);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_06() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, 100);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_07() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_KB / 2);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_08() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_KB);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_09() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_MB);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_10() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_MB * 10);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_11() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, 1);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_12() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, 100);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_13() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_KB);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_14() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB / 2);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_15() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_16() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB * 10);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
new file mode 100644
index 0000000..f453aa5
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.util;
+
+import java.io.InputStream;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
+
+/**
+ * Tests for the {@link TrackedInputStream}
+ * 
+ * 
+ * 
+ */
+public class TrackedInputStreamTest extends AbstractTrackableInputStreamTests {
+
+    @Override
+    protected TrackableInputStream getInstance(InputStream input) {
+        return new TrackedInputStream(input);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
new file mode 100644
index 0000000..b5ea2d8
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
@@ -0,0 +1,255 @@
+/*
+ * 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.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.lang.StreamRDFCounting;
+import org.apache.jena.riot.system.StreamRDFLib;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract node tuple output format tests
+ * 
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ * 
+ */
+public abstract class AbstractNodeTupleOutputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> {
+
+    @SuppressWarnings("unused")
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleOutputFormatTests.class);
+
+    protected static final int EMPTY_SIZE = 0, SMALL_SIZE = 100, LARGE_SIZE = 10000, VERY_LARGE_SIZE = 100000;
+
+    /**
+     * Temporary folder for the tests
+     */
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+
+    /**
+     * Prepares a fresh configuration
+     * 
+     * @return Configuration
+     */
+    protected Configuration prepareConfiguration() {
+        Configuration config = new Configuration(true);
+        // Nothing else to do
+        return config;
+    }
+
+    /**
+     * Gets the extra file extension to add to the filenames
+     * 
+     * @return File extension
+     */
+    protected abstract String getFileExtension();
+
+    /**
+     * Generates tuples to be output for testing
+     * 
+     * @param num
+     *            Number of tuples to generate
+     * @return Iterator of tuples
+     */
+    protected abstract Iterator<T> generateTuples(int num);
+
+    /**
+     * Counts tuples in the output file
+     * 
+     * @param f
+     *            Output file
+     * @return Tuple count
+     */
+    protected final long countTuples(File f) {
+        StreamRDFCounting counter = StreamRDFLib.count();
+        RDFDataMgr.parse(counter, f.getAbsolutePath(), this.getRdfLanguage(), null);
+        return counter.count();
+    }
+
+    /**
+     * Checks that tuples are as expected
+     * 
+     * @param f
+     *            File
+     * @param expected
+     *            Expected number of tuples
+     */
+    protected void checkTuples(File f, long expected) {
+        Assert.assertEquals(expected, this.countTuples(f));
+    }
+
+    /**
+     * Gets the RDF language of the produced output which is used to parse back
+     * in the output to validate the correct amount of output was produced
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+    /**
+     * Gets the output format to test
+     * 
+     * @return Output format
+     */
+    protected abstract OutputFormat<NullWritable, T> getOutputFormat();
+
+    /**
+     * Adds an output path to the job configuration
+     * 
+     * @param f
+     *            File
+     * @param config
+     *            Configuration
+     * @param job
+     *            Job
+     * @throws IOException
+     */
+    protected void addOutputPath(File f, Configuration config, Job job) throws IOException {
+        FileSystem fs = FileSystem.getLocal(config);
+        Path outputPath = fs.makeQualified(new Path(f.getAbsolutePath()));
+        FileOutputFormat.setOutputPath(job, outputPath);
+    }
+
+    protected File findOutputFile(File dir, JobContext context) throws FileNotFoundException, IOException {
+        Path outputPath = FileOutputFormat.getOutputPath(context);
+        RemoteIterator<LocatedFileStatus> files = outputPath.getFileSystem(context.getConfiguration()).listFiles(
+                outputPath, true);
+        while (files.hasNext()) {
+            LocatedFileStatus status = files.next();
+            if (status.isFile() && !status.getPath().getName().startsWith("_")) {
+                return new File(status.getPath().toUri());
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Tests output
+     * 
+     * @param f
+     *            File to output to
+     * @param num
+     *            Number of tuples to output
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected final void testOutput(File f, int num) throws IOException, InterruptedException {
+        // Prepare configuration
+        Configuration config = this.prepareConfiguration();
+
+        // Set up fake job
+        OutputFormat<NullWritable, T> outputFormat = this.getOutputFormat();
+        Job job = Job.getInstance(config);
+        job.setOutputFormatClass(outputFormat.getClass());
+        this.addOutputPath(f, job.getConfiguration(), job);
+        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+        Assert.assertNotNull(FileOutputFormat.getOutputPath(context));
+
+        // Output the data
+        TaskAttemptID id = new TaskAttemptID("outputTest", 1, TaskType.MAP, 1, 1);
+        TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), id);
+        RecordWriter<NullWritable, T> writer = outputFormat.getRecordWriter(taskContext);
+        Iterator<T> tuples = this.generateTuples(num);
+        while (tuples.hasNext()) {
+            writer.write(NullWritable.get(), tuples.next());
+        }
+        writer.close(taskContext);
+
+        // Check output
+        File outputFile = this.findOutputFile(this.folder.getRoot(), context);
+        Assert.assertNotNull(outputFile);
+        this.checkTuples(outputFile, num);
+    }
+
+    /**
+     * Basic output tests
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void output_01() throws IOException, InterruptedException {
+        this.testOutput(this.folder.getRoot(), EMPTY_SIZE);
+    }
+
+    /**
+     * Basic output tests
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void output_02() throws IOException, InterruptedException {
+        this.testOutput(this.folder.getRoot(), SMALL_SIZE);
+    }
+
+    /**
+     * Basic output tests
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void output_03() throws IOException, InterruptedException {
+        this.testOutput(this.folder.getRoot(), LARGE_SIZE);
+    }
+
+    /**
+     * Basic output tests
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    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/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
new file mode 100644
index 0000000..f1822f6
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.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;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for quad output formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractQuadOutputFormatTests extends AbstractNodeTupleOutputFormatTests<Quad, QuadWritable> {
+
+    @Override
+    protected Iterator<QuadWritable> generateTuples(int num) {
+        List<QuadWritable> qs = new ArrayList<QuadWritable>();
+        for (int i = 0; i < num; i++) {
+            Quad q = new Quad(NodeFactory.createURI("http://example.org/graphs/" + i),
+                    NodeFactory.createURI("http://example.org/subjects/" + i),
+                    NodeFactory.createURI("http://example.org/predicate"), NodeFactory.createLiteral(Integer.toString(i),
+                            XSDDatatype.XSDinteger));
+            qs.add(new QuadWritable(q));
+        }
+        return qs.iterator();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
new file mode 100644
index 0000000..90eb531
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for triple output formats
+ * 
+ *
+ */
+public abstract class AbstractTripleOutputFormatTests extends AbstractNodeTupleOutputFormatTests<Triple, TripleWritable> {
+
+    @Override
+    protected Iterator<TripleWritable> generateTuples(int num) {
+        List<TripleWritable> ts = new ArrayList<TripleWritable>();
+        for (int i = 0; i < num; i++) {
+            Triple t = new Triple(NodeFactory.createURI("http://example.org/subjects/" + i), NodeFactory.createURI("http://example.org/predicate"), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
+            ts.add(new TripleWritable(t));
+        }
+        return ts.iterator();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
new file mode 100644
index 0000000..c6784a5
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.jsonld;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * 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>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
new file mode 100644
index 0000000..d157409
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.jsonld;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * 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>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
new file mode 100644
index 0000000..1a7ffa4
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.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.nquads;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for NQuads output format
+ * 
+ * 
+ * 
+ */
+public class NQuadsOutputTest extends AbstractQuadOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".nq";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
+        return new NQuadsOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
new file mode 100644
index 0000000..ad9be56
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.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.ntriples;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for NTriples output format
+ * 
+ * 
+ * 
+ */
+public class NTriplesOutputTest extends AbstractTripleOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".nt";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new NTriplesOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
new file mode 100644
index 0000000..833f89b
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.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.rdfjson;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for RDF/JSON output
+ * 
+ * 
+ * 
+ */
+public class RdfJsonOutputTest extends AbstractTripleOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".rj";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFJSON;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new RdfJsonOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
new file mode 100644
index 0000000..40bc937
--- /dev/null
+++ b/jena-elephas/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.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.rdfxml;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for RDF/XML output
+ * 
+ * 
+ * 
+ */
+public class RdfXmlOutputTest extends AbstractTripleOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".rdf";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFXML;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new RdfXmlOutputFormat<NullWritable>();
+    }
+
+}

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

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


[20/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
deleted file mode 100644
index 2279444..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedQuadsStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract record reader for whole file triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractBlockBasedQuadReader extends AbstractBlockBasedNodeTupleReader<Quad, QuadWritable> {
-
-    @Override
-    protected PipedRDFIterator<Quad> getPipedIterator() {
-        return new PipedRDFIterator<Quad>();
-    }
-
-    @Override
-    protected TrackedPipedRDFStream<Quad> getPipedStream(PipedRDFIterator<Quad> iterator, TrackableInputStream input) {
-        return new TrackedPipedQuadsStream(iterator, input);
-    }
-
-    @Override
-    protected QuadWritable createInstance(Quad tuple) {
-        return new QuadWritable(tuple);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
deleted file mode 100644
index 2afd329..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedTriplesStream;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * An abstract record reader for whole file triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractBlockBasedTripleReader extends AbstractBlockBasedNodeTupleReader<Triple, TripleWritable> {
-
-    @Override
-    protected PipedRDFIterator<Triple> getPipedIterator() {
-        return new PipedRDFIterator<Triple>();
-    }
-
-    @Override
-    protected TrackedPipedRDFStream<Triple> getPipedStream(PipedRDFIterator<Triple> iterator, TrackableInputStream input) {
-        return new TrackedPipedTriplesStream(iterator, input);
-    }
-
-    @Override
-    protected TripleWritable createInstance(Triple tuple) {
-        return new TripleWritable(tuple);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
deleted file mode 100644
index 6c1abe9..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-import java.util.Iterator;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.util.LineReader;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.system.ParserProfile;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract implementation of a record reader that reads records from line
- * based tuple formats. This only supports reading from file splits currently.
- * <p>
- * The keys produced are the position of the line in the file and the values
- * will be node tuples
- * </p>
- * 
- * 
- * 
- * @param <TValue>
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractLineBasedNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractLineBasedNodeTupleReader.class);
-    private CompressionCodecFactory compressionCodecs = null;
-    private long start, pos, end, estLength;
-    private int maxLineLength;
-    private LineReader in;
-    private LongWritable key = null;
-    private Text value = null;
-    private T tuple = null;
-    private ParserProfile profile = null;
-    private boolean ignoreBadTuples = true;
-
-    @Override
-    public final void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
-        LOG.debug("initialize({}, {})", genericSplit, context);
-
-        // Assuming file split
-        if (!(genericSplit instanceof FileSplit))
-            throw new IOException("This record reader only supports FileSplit inputs");
-        FileSplit split = (FileSplit) genericSplit;
-
-        // Configuration
-        profile = RdfIOUtils.createParserProfile(context, split.getPath());
-        Configuration config = context.getConfiguration();
-        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
-        if (this.ignoreBadTuples)
-            LOG.warn(
-                    "Configured to ignore bad tuples, parsing errors will be logged and the bad line skipped but no errors will be thrownConsider setting {} to false to disable this behaviour",
-                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
-
-        // Figure out what portion of the file to read
-        this.maxLineLength = config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE);
-        start = split.getStart();
-        end = start + split.getLength();
-        final Path file = split.getPath();
-        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
-        compressionCodecs = new CompressionCodecFactory(config);
-        final CompressionCodec codec = compressionCodecs.getCodec(file);
-
-        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { start, split.getLength(), totalLength }));
-
-        // Open the file and seek to the start of the split
-        FileSystem fs = file.getFileSystem(config);
-        FSDataInputStream fileIn = fs.open(file);
-        boolean skipFirstLine = false;
-        if (codec != null) {
-            // Compressed input
-            // For compressed input NLineInputFormat will have failed to find
-            // any line breaks and will give us a split from 0 -> (length - 1)
-            // Add 1 and verify we got complete split
-            if (totalLength > split.getLength() + 1)
-                throw new IOException("This record reader can only be used with compressed input where the split covers the whole file");
-            in = new LineReader(codec.createInputStream(fileIn), config);
-            estLength = end;
-            end = Long.MAX_VALUE;
-        } else {
-            // Uncompressed input
-            if (start != 0) {
-                skipFirstLine = true;
-                --start;
-                fileIn.seek(start);
-            }
-            in = new LineReader(fileIn, config);
-        }
-        // Skip first line and re-establish "start".
-        // This is to do with how line reader reads lines and how
-        // NLineInputFormat will provide the split information to use
-        if (skipFirstLine) {
-            start += in.readLine(new Text(), 0, (int) Math.min((long) Integer.MAX_VALUE, end - start));
-        }
-        this.pos = start;
-    }
-
-    /**
-     * Gets an iterator over the data on the current line
-     * 
-     * @param line
-     *            Line
-     * @param profile
-     *            Parser profile
-     * @return Iterator
-     */
-    protected abstract Iterator<TValue> getIterator(String line, ParserProfile profile);
-
-    /**
-     * Creates an instance of a writable tuple from the given tuple value
-     * 
-     * @param tuple
-     *            Tuple value
-     * @return Writable tuple
-     */
-    protected abstract T createInstance(TValue tuple);
-
-    @Override
-    public final boolean nextKeyValue() throws IOException, InterruptedException {
-        // Reuse key for efficiency
-        if (key == null) {
-            key = new LongWritable();
-        }
-
-        // Reset value which we use for reading lines
-        if (value == null) {
-            value = new Text();
-        }
-        tuple = null;
-
-        // Try to read the next valid line
-        int newSize = 0;
-        while (pos < end) {
-            // Read next line
-            newSize = in.readLine(value, maxLineLength, Math.max((int) Math.min(Integer.MAX_VALUE, end - pos), maxLineLength));
-
-            // Once we get an empty line we've reached the end of our input
-            if (newSize == 0) {
-                break;
-            }
-
-            // Update position, remember that where inputs are compressed we may
-            // be at a larger position then we expected because the length of
-            // the split is likely less than the length of the data once
-            // decompressed
-            key.set(pos);
-            pos += newSize;
-            if (pos > estLength)
-                estLength = pos + 1;
-
-            // Skip lines that exceed the line length limit that has been set
-            if (newSize >= maxLineLength) {
-                LOG.warn("Skipped oversized line of size {} at position {}", newSize, (pos - newSize));
-                continue;
-            }
-
-            // Attempt to read the tuple from current line
-            try {
-                Iterator<TValue> iter = this.getIterator(value.toString(), profile);
-                if (iter.hasNext()) {
-                    tuple = this.createInstance(iter.next());
-
-                    // If we reach here we've found a valid tuple so we can
-                    // break out of the loop
-                    break;
-                } else {
-                    // Empty line/Comment line
-                    LOG.debug("Valid line with no triple at position {}", (pos - newSize));
-                    continue;
-                }
-            } catch (Throwable e) {
-                // Failed to read the tuple on this line
-                LOG.error("Bad tuple at position " + (pos - newSize), e);
-                if (this.ignoreBadTuples)
-                    continue;
-                throw new IOException(String.format("Bad tuple at position %d", (pos - newSize)), e);
-            }
-        }
-        boolean result = this.tuple != null;
-
-        // End of input
-        if (newSize == 0) {
-            key = null;
-            value = null;
-            tuple = null;
-            result = false;
-            estLength = pos;
-        }
-        LOG.debug("nextKeyValue() --> {}", result);
-        return result;
-    }
-
-    @Override
-    public LongWritable getCurrentKey() throws IOException, InterruptedException {
-        LOG.debug("getCurrentKey() --> {}", key);
-        return key;
-    }
-
-    @Override
-    public T getCurrentValue() throws IOException, InterruptedException {
-        LOG.debug("getCurrentValue() --> {}", tuple);
-        return tuple;
-    }
-
-    @Override
-    public float getProgress() throws IOException, InterruptedException {
-        float progress = 0.0f;
-        if (start != end) {
-            if (end == Long.MAX_VALUE) {
-                if (estLength == 0)
-                    return 1.0f;
-                // Use estimated length
-                progress = Math.min(1.0f, (pos - start) / (float) (estLength - start));
-            } else {
-                // Use actual length
-                progress = Math.min(1.0f, (pos - start) / (float) (end - start));
-            }
-        }
-        LOG.debug("getProgress() --> {}", progress);
-        return progress;
-    }
-
-    @Override
-    public void close() throws IOException {
-        LOG.debug("close()");
-        if (in != null) {
-            in.close();
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
deleted file mode 100644
index ac93865..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.readers;
-
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.tokens.Tokenizer;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract reader for line based quad formats
- * 
- * 
- * 
- */
-public abstract class AbstractLineBasedQuadReader extends AbstractLineBasedNodeTupleReader<Quad, QuadWritable> {
-
-    @Override
-    protected Iterator<Quad> getIterator(String line, ParserProfile profile) {
-        Tokenizer tokenizer = getTokenizer(line);
-        return getQuadsIterator(tokenizer, profile);
-    }
-
-    @Override
-    protected QuadWritable createInstance(Quad q) {
-        return new QuadWritable(q);
-    }
-
-    protected abstract Tokenizer getTokenizer(String line);
-
-    protected abstract Iterator<Quad> getQuadsIterator(Tokenizer tokenizer, ParserProfile profile);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
deleted file mode 100644
index a0232f5..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.tokens.Tokenizer;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * An abstract record reader for line based triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractLineBasedTripleReader extends AbstractLineBasedNodeTupleReader<Triple, TripleWritable> {
-
-    @Override
-    protected Iterator<Triple> getIterator(String line, ParserProfile profile) {
-        Tokenizer tokenizer = getTokenizer(line);
-        return getTriplesIterator(tokenizer, profile);
-    }
-
-    @Override
-    protected TripleWritable createInstance(Triple t) {
-        return new TripleWritable(t);
-    }
-    
-    protected abstract Tokenizer getTokenizer(String line);
-
-    protected abstract Iterator<Triple> getTriplesIterator(Tokenizer tokenizer, ParserProfile profile);
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
deleted file mode 100644
index d0ffed8..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.Path;
-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.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract record reader for arbitrary RDF which provides support for
- * selecting the actual record reader to use based on detecting the RDF language
- * from the file name
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractRdfReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        RecordReader<LongWritable, T> {
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractRdfReader.class);
-
-    private RecordReader<LongWritable, T> reader;
-
-    @Override
-    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException,
-            InterruptedException {
-        LOG.debug("initialize({}, {})", genericSplit, context);
-
-        // Assuming file split
-        if (!(genericSplit instanceof FileSplit))
-            throw new IOException("This record reader only supports FileSplit inputs");
-
-        // Find RDF language
-        FileSplit split = (FileSplit) genericSplit;
-        Path path = split.getPath();
-        Lang lang = RDFLanguages.filenameToLang(path.getName());
-        if (lang == null)
-            throw new IOException("There is no registered RDF language for the input file " + path.toString());
-
-        // Select the record reader and initialize
-        this.reader = this.selectRecordReader(lang);
-        this.reader.initialize(split, context);
-    }
-
-    /**
-     * Selects the appropriate record reader to use for the given RDF language
-     * 
-     * @param lang
-     *            RDF language
-     * @return Record reader
-     * @throws IOException
-     *             Should be thrown if no record reader can be selected
-     */
-    protected abstract RecordReader<LongWritable, T> selectRecordReader(Lang lang) throws IOException;
-
-    @Override
-    public final boolean nextKeyValue() throws IOException, InterruptedException {
-        return this.reader.nextKeyValue();
-    }
-
-    @Override
-    public final LongWritable getCurrentKey() throws IOException, InterruptedException {
-        return this.reader.getCurrentKey();
-    }
-
-    @Override
-    public final T getCurrentValue() throws IOException, InterruptedException {
-        return this.reader.getCurrentValue();
-    }
-
-    @Override
-    public final float getProgress() throws IOException, InterruptedException {
-        return this.reader.getProgress();
-    }
-
-    @Override
-    public final void close() throws IOException {
-        this.reader.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
deleted file mode 100644
index c2da3f7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFDataMgr;
-import org.apache.jena.riot.ReaderRIOT;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-import org.apache.jena.riot.lang.PipedRDFStream;
-import org.apache.jena.riot.system.ParserProfile;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract implementation for a record reader that reads records from whole
- * files i.e. the whole file must be kept together to allow tuples to be
- * successfully read. This only supports reading from file splits currently.
- * <p>
- * The keys produced are the approximate position in the file at which a tuple
- * was found and the values will be node tuples. Positions are approximate
- * because they are recorded after the point at which the most recent tuple was
- * parsed from the input thus they reflect the approximate position in the
- * stream immediately after which the triple was found.
- * </p>
- * <p>
- * You should also be aware that with whole file formats syntax compressions in
- * the format may mean that there are multiple triples produced with the same
- * position and thus key.
- * </p>
- * 
- * 
- * 
- * @param <TValue>
- *            Value type
- * @param <T>
- *            Tuple type
- */
-public abstract class AbstractWholeFileNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractLineBasedNodeTupleReader.class);
-    private CompressionCodec compressionCodecs;
-    private TrackedInputStream input;
-    private LongWritable key;
-    private long length;
-    private T tuple;
-    private TrackedPipedRDFStream<TValue> stream;
-    private PipedRDFIterator<TValue> iter;
-    private Thread parserThread;
-    private boolean finished = false;
-    private boolean ignoreBadTuples = true;
-    private boolean parserFinished = false;
-    private Throwable parserError = null;
-
-    @Override
-    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
-        LOG.debug("initialize({}, {})", genericSplit, context);
-
-        // Assuming file split
-        if (!(genericSplit instanceof FileSplit))
-            throw new IOException("This record reader only supports FileSplit inputs");
-        FileSplit split = (FileSplit) genericSplit;
-
-        // Configuration
-        Configuration config = context.getConfiguration();
-        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
-        if (this.ignoreBadTuples)
-            LOG.warn(
-                    "Configured to ignore bad tuples, parsing errors will be logged and further parsing aborted but no user visible errors will be thrown.  Consider setting {} to false to disable this behaviour",
-                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
-
-        // Figure out what portion of the file to read
-        if (split.getStart() > 0)
-            throw new IOException("This record reader requires a file split which covers the entire file");
-        final Path file = split.getPath();
-        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
-        CompressionCodecFactory factory = new CompressionCodecFactory(config);
-        this.compressionCodecs = factory.getCodec(file);
-
-        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { split.getStart(), split.getLength(),
-                totalLength }));
-
-        if (totalLength > split.getLength())
-            throw new IOException("This record reader requires a file split which covers the entire file");
-
-        // Open the file and prepare the input stream
-        FileSystem fs = file.getFileSystem(config);
-        FSDataInputStream fileIn = fs.open(file);
-        this.length = split.getLength();
-        if (this.compressionCodecs != null) {
-            // Compressed input
-            input = new TrackedInputStream(this.compressionCodecs.createInputStream(fileIn));
-        } else {
-            // Uncompressed input
-            input = new TrackedInputStream(fileIn);
-        }
-
-        // Set up background thread for parser
-        iter = this.getPipedIterator();
-        this.stream = this.getPipedStream(iter, this.input);
-        ParserProfile profile = RdfIOUtils.createParserProfile(context, file);
-        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage(), profile);
-        this.parserThread = new Thread(parserRunnable);
-        this.parserThread.setDaemon(true);
-        this.parserThread.start();
-    }
-
-    /**
-     * Gets the RDF iterator to use
-     * 
-     * @return Iterator
-     */
-    protected abstract PipedRDFIterator<TValue> getPipedIterator();
-
-    /**
-     * Gets the RDF stream to parse to
-     * 
-     * @param iterator
-     *            Iterator
-     * @return RDF stream
-     */
-    protected abstract TrackedPipedRDFStream<TValue> getPipedStream(PipedRDFIterator<TValue> iterator, TrackableInputStream input);
-
-    /**
-     * Gets the RDF language to use for parsing
-     * 
-     * @return
-     */
-    protected abstract Lang getRdfLanguage();
-
-    /**
-     * Creates the runnable upon which the parsing will run
-     * 
-     * @param input
-     *            Input
-     * @param stream
-     *            Stream
-     * @param lang
-     *            Language to use for parsing
-     * @return Parser runnable
-     */
-    private Runnable createRunnable(@SuppressWarnings("rawtypes") final AbstractWholeFileNodeTupleReader reader, final InputStream input,
-            final PipedRDFStream<TValue> stream, final Lang lang, final ParserProfile profile) {
-        return new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    ReaderRIOT riotReader = RDFDataMgr.createReader(lang);
-                    riotReader.setParserProfile(profile);
-                    riotReader.read(input, null, lang.getContentType(), stream, null);
-                    reader.setParserFinished(null);
-                } catch (Throwable e) {
-                    reader.setParserFinished(e);
-                }
-            }
-        };
-    }
-
-    /**
-     * Sets the parser thread finished state
-     * 
-     * @param e
-     *            Error (if any)
-     */
-    private void setParserFinished(Throwable e) {
-        synchronized (this.parserThread) {
-            this.parserError = e;
-            this.parserFinished = true;
-        }
-    }
-
-    /**
-     * Waits for the parser thread to have reported as finished
-     * 
-     * @throws InterruptedException
-     */
-    private void waitForParserFinished() throws InterruptedException {
-        do {
-            synchronized (this.parserThread) {
-                if (this.parserFinished)
-                    return;
-            }
-            Thread.sleep(50);
-        } while (true);
-    }
-
-    /**
-     * Creates an instance of a writable tuple from the given tuple value
-     * 
-     * @param tuple
-     *            Tuple value
-     * @return Writable tuple
-     */
-    protected abstract T createInstance(TValue tuple);
-
-    @Override
-    public boolean nextKeyValue() throws IOException, InterruptedException {
-        // Reuse key for efficiency
-        if (key == null) {
-            key = new LongWritable();
-        }
-
-        if (this.finished)
-            return false;
-
-        try {
-            if (this.iter.hasNext()) {
-                Long l = this.stream.getPosition();
-                if (l != null) {
-                    this.key.set(l);
-                    // For compressed input the actual length from which we
-                    // calculate progress is likely less than the actual
-                    // uncompressed length so we may need to increment the
-                    // length as we go along
-                    // We always add 1 more than the current length because we
-                    // don't want to report 100% progress until we really have
-                    // finished
-                    if (this.compressionCodecs != null && l > this.length)
-                        this.length = l + 1;
-                }
-                this.tuple = this.createInstance(this.iter.next());
-                return true;
-            } else {
-                // Need to ensure that the parser thread has finished in order
-                // to determine whether we finished without error
-                this.waitForParserFinished();
-                if (this.parserError != null) {
-                    LOG.error("Error parsing whole file, aborting further parsing", this.parserError);
-                    if (!this.ignoreBadTuples)
-                        throw new IOException("Error parsing whole file at position " + this.input.getBytesRead() + ", aborting further parsing",
-                                this.parserError);
-
-                }
-
-                this.key = null;
-                this.tuple = null;
-                this.finished = true;
-                // This is necessary so that when compressed input is used we
-                // report 100% progress once we've reached the genuine end of
-                // the stream
-                if (this.compressionCodecs != null)
-                    this.length--;
-                return false;
-            }
-        } catch (Throwable e) {
-            // Failed to read the tuple on this line
-            LOG.error("Error parsing whole file, aborting further parsing", e);
-            if (!this.ignoreBadTuples) {
-                this.iter.close();
-                throw new IOException("Error parsing whole file at position " + this.input.getBytesRead() + ", aborting further parsing", e);
-            }
-            this.key = null;
-            this.tuple = null;
-            this.finished = true;
-            return false;
-        }
-    }
-
-    @Override
-    public LongWritable getCurrentKey() throws IOException, InterruptedException {
-        return this.key;
-    }
-
-    @Override
-    public T getCurrentValue() throws IOException, InterruptedException {
-        return this.tuple;
-    }
-
-    @Override
-    public float getProgress() throws IOException, InterruptedException {
-        float progress = 0.0f;
-        if (this.key == null) {
-            // We've either not started or we've finished
-            progress = (this.finished ? 1.0f : 0.0f);
-        } else if (this.key.get() == Long.MIN_VALUE) {
-            // We don't have a position so we've either in-progress or finished
-            progress = (this.finished ? 1.0f : 0.5f);
-        } else {
-            // We're some way through the file
-            progress = this.key.get() / (float) this.length;
-        }
-        LOG.debug("getProgress() --> {}", progress);
-        return progress;
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.iter.close();
-        this.input.close();
-        this.finished = true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
deleted file mode 100644
index e525bea..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedQuadsStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract record reader for whole file triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractWholeFileQuadReader extends AbstractWholeFileNodeTupleReader<Quad, QuadWritable> {
-
-    @Override
-    protected PipedRDFIterator<Quad> getPipedIterator() {
-        return new PipedRDFIterator<Quad>();
-    }
-
-    @Override
-    protected TrackedPipedRDFStream<Quad> getPipedStream(PipedRDFIterator<Quad> iterator, TrackableInputStream input) {
-        return new TrackedPipedQuadsStream(iterator, input);
-    }
-
-    @Override
-    protected QuadWritable createInstance(Quad tuple) {
-        return new QuadWritable(tuple);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
deleted file mode 100644
index 8710b99..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.readers;
-
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedTriplesStream;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * An abstract record reader for whole file triple formats
- * 
- * 
- * 
- */
-public abstract class AbstractWholeFileTripleReader extends AbstractWholeFileNodeTupleReader<Triple, TripleWritable> {
-
-    @Override
-    protected PipedRDFIterator<Triple> getPipedIterator() {
-        return new PipedRDFIterator<Triple>();
-    }
-
-    @Override
-    protected TrackedPipedRDFStream<Triple> getPipedStream(PipedRDFIterator<Triple> iterator, TrackableInputStream input) {
-        return new TrackedPipedTriplesStream(iterator, input);
-    }
-
-    @Override
-    protected TripleWritable createInstance(Triple tuple) {
-        return new TripleWritable(tuple);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
deleted file mode 100644
index 26b0a8b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-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;
-
-/**
- * A record reader that reads triples from any RDF quads format
- */
-public class QuadsReader extends AbstractRdfReader<Quad, QuadWritable> {
-
-    @Override
-    protected RecordReader<LongWritable, QuadWritable> selectRecordReader(Lang lang) throws IOException {
-        if (!RDFLanguages.isQuads(lang))
-            throw new IOException(
-                    lang.getLabel()
-                            + " is not a RDF quads format, perhaps you wanted TriplesInputFormat or TriplesOrQuadsInputFormat instead?");
-
-        // This will throw an appropriate error if the language does not support
-        // triples
-        return HadoopRdfIORegistry.createQuadReader(lang);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
deleted file mode 100644
index b071f67..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-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.graph.Node;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A record reader that reads RDF from any triples/quads format. Triples are
- * converted into quads in the default graph. This behaviour can be changed by
- * deriving from this class and overriding the {@link #getGraphNode()} method
- * 
- * 
- * 
- */
-@SuppressWarnings("javadoc")
-public class TriplesOrQuadsReader extends AbstractRdfReader<Quad, QuadWritable> {
-
-    @Override
-    protected RecordReader<LongWritable, QuadWritable> selectRecordReader(Lang lang) throws IOException {
-        if (!RDFLanguages.isQuads(lang) && !RDFLanguages.isTriples(lang))
-            throw new IOException(lang.getLabel() + " is not a RDF triples/quads format");
-
-        if (HadoopRdfIORegistry.hasQuadReader(lang)) {
-            // Supports quads directly
-            return HadoopRdfIORegistry.createQuadReader(lang);
-        } else {
-            // Try to create a triples reader and wrap upwards into quads
-            // This will throw an error if a triple reader is not available
-            return new TriplesToQuadsReader(HadoopRdfIORegistry.createTripleReader(lang));
-        }
-    }
-
-    /**
-     * Gets the graph node which represents the graph into which triples will be
-     * indicated to belong to when they are converting into quads.
-     * <p>
-     * Defaults to {@link Quad#defaultGraphNodeGenerated} which represents the
-     * default graph
-     * </p>
-     * 
-     * @return Graph node
-     */
-    protected Node getGraphNode() {
-        return Quad.defaultGraphNodeGenerated;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
deleted file mode 100644
index 96c4a65..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-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;
-
-/**
- * A record reader that reads triples from any RDF triples format
- */
-public class TriplesReader extends AbstractRdfReader<Triple, TripleWritable> {
-
-    @Override
-    protected RecordReader<LongWritable, TripleWritable> selectRecordReader(Lang lang) throws IOException {
-        if (!RDFLanguages.isTriples(lang))
-            throw new IOException(
-                    lang.getLabel()
-                            + " is not a RDF triples format, perhaps you wanted QuadsInputFormat or TriplesOrQuadsInputFormat instead?");
-
-        // This will throw an appropriate error if the language does not support
-        // triples
-        return HadoopRdfIORegistry.createTripleReader(lang);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
deleted file mode 100644
index a388f0e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.readers;
-
-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.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A record reader that converts triples into quads by wrapping a
- * {@code RecordReader<LongWritable, TripleWritable>} implementation
- * 
- * 
- * 
- */
-public class TriplesToQuadsReader extends RecordReader<LongWritable, QuadWritable> {
-
-    private final RecordReader<LongWritable, TripleWritable> reader;
-    private Node graph;
-
-    /**
-     * Creates a new reader
-     * 
-     * @param reader
-     *            Triple reader
-     */
-    public TriplesToQuadsReader(RecordReader<LongWritable, TripleWritable> reader) {
-        this(reader, Quad.defaultGraphNodeGenerated);
-    }
-
-    /**
-     * Creates a new reader
-     * 
-     * @param reader
-     *            Triple reader
-     * @param graphNode
-     *            Graph node
-     */
-    public TriplesToQuadsReader(RecordReader<LongWritable, TripleWritable> reader, Node graphNode) {
-        if (reader == null)
-            throw new NullPointerException("reader cannot be null");
-        if (graphNode == null)
-            throw new NullPointerException("Graph node cannot be null");
-        this.reader = reader;
-        this.graph = graphNode;
-    }
-
-    @Override
-    public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
-        this.reader.initialize(split, context);
-    }
-
-    @Override
-    public final boolean nextKeyValue() throws IOException, InterruptedException {
-        return this.reader.nextKeyValue();
-    }
-
-    @Override
-    public final LongWritable getCurrentKey() throws IOException, InterruptedException {
-        return this.reader.getCurrentKey();
-    }
-
-    @Override
-    public final QuadWritable getCurrentValue() throws IOException, InterruptedException {
-        TripleWritable t = this.reader.getCurrentValue();
-        return new QuadWritable(new Quad(this.graph, t.get()));
-    }
-
-    @Override
-    public final float getProgress() throws IOException, InterruptedException {
-        return this.reader.getProgress();
-    }
-
-    @Override
-    public final void close() throws IOException {
-        this.reader.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
deleted file mode 100644
index 1b3f467..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDQuadReader.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.readers.jsonld;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
deleted file mode 100644
index 7cdea9e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/jsonld/JsonLDTripleReader.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.readers.jsonld;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
deleted file mode 100644
index cef8ef1..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/BlockedNQuadsReader.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.readers.nquads;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractBlockBasedQuadReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record record for NQuads
- * <p>
- * This is a hybrid of the {@link NQuadsReader} and the
- * {@link WholeFileNQuadsReader} in that it does not process individual lines
- * rather it processes the inputs in blocks of lines parsing the whole block
- * rather than individual lines. This provides a compromise between the higher
- * parser setup of creating more parsers and the benefit of being able to split
- * input files over multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class BlockedNQuadsReader extends AbstractBlockBasedQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java
deleted file mode 100644
index e00e318..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/NQuadsReader.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.readers.nquads;
-
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractLineBasedQuadReader;
-import org.apache.jena.riot.lang.LangNQuads;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.tokens.Tokenizer;
-import org.apache.jena.riot.tokens.TokenizerFactory;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A record reader for NQuads
- * 
- * 
- * 
- */
-public class NQuadsReader extends AbstractLineBasedQuadReader {
-
-    @Override
-    protected Tokenizer getTokenizer(String line) {
-        return TokenizerFactory.makeTokenizerString(line);
-    }
-
-    @Override
-    protected Iterator<Quad> getQuadsIterator(Tokenizer tokenizer, ParserProfile profile) {
-        return new LangNQuads(tokenizer, profile, null);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
deleted file mode 100644
index 96e6f80..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/nquads/WholeFileNQuadsReader.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.readers.nquads;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record record for NQuads
- * <p>
- * Unlike the {@link NQuadsReader} this processes files as a whole rather than
- * individual lines. This has the advantage of less parser setup overhead but
- * the disadvantage that the input cannot be split between multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class WholeFileNQuadsReader extends AbstractWholeFileQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java
deleted file mode 100644
index 7268d5a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/BlockedNTriplesReader.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.readers.ntriples;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractBlockBasedTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record record for NTriples
- * <p>
- * This is a hybrid of the {@link NTriplesReader} and the
- * {@link WholeFileNTriplesReader} in that it does not process individual lines
- * rather it processes the inputs in blocks of lines parsing the whole block
- * rather than individual lines. This provides a compromise between the higher
- * parser setup of creating more parsers and the benefit of being able to split
- * input files over multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class BlockedNTriplesReader extends AbstractBlockBasedTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
deleted file mode 100644
index bfc8503..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/NTriplesReader.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.readers.ntriples;
-
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractLineBasedTripleReader;
-import org.apache.jena.riot.lang.LangNTriples;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.tokens.Tokenizer;
-import org.apache.jena.riot.tokens.TokenizerFactory;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A record reader for NTriples
- * 
- * 
- * 
- */
-public class NTriplesReader extends AbstractLineBasedTripleReader {
-
-    @Override
-    protected Iterator<Triple> getTriplesIterator(Tokenizer tokenizer, ParserProfile profile) {
-        return new LangNTriples(tokenizer, profile, null);
-    }
-
-    @Override
-    protected Tokenizer getTokenizer(String line) {
-        return TokenizerFactory.makeTokenizerString(line);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
deleted file mode 100644
index c200d93..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/ntriples/WholeFileNTriplesReader.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.readers.ntriples;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record record for NTriples
- * <p>
- * Unlike the {@link NTriplesReader} this processes files as a whole rather than
- * individual lines. This has the advantage of less parser setup overhead but
- * the disadvantage that the input cannot be split between multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class WholeFileNTriplesReader extends AbstractWholeFileTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
deleted file mode 100644
index 009024b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfjson/RdfJsonReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.rdfjson;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for RDF/JSON files
- * 
- * 
- * 
- */
-public class RdfJsonReader extends AbstractWholeFileTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFJSON;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
deleted file mode 100644
index 9c374c6..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/rdfxml/RdfXmlReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.rdfxml;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for RDF/XML files
- * 
- * 
- * 
- */
-public class RdfXmlReader extends AbstractWholeFileTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFXML;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
deleted file mode 100644
index 084b1ec..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftQuadReader.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.readers.thrift;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-public class ThriftQuadReader extends AbstractWholeFileQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
deleted file mode 100644
index 713bfa7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/thrift/ThriftTripleReader.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.readers.thrift;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-public class ThriftTripleReader extends AbstractWholeFileTripleReader {
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
deleted file mode 100644
index b1b0c3c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trig/TriGReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.trig;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for TriG files
- * 
- * 
- * 
- */
-public class TriGReader extends AbstractWholeFileQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIG;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
deleted file mode 100644
index 6873c64..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/trix/TriXReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.trix;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileQuadReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for TriX files
- * 
- * 
- * 
- */
-public class TriXReader extends AbstractWholeFileQuadReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TRIX;
-    }
-
-}


[16/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
deleted file mode 100644
index 2e1e865..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedTripleInputFormatTests.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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;
-
-/**
- * Abstract tests for blocked triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractBlockedTripleInputFormatTests extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected boolean canSplitInputs() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
deleted file mode 100644
index e22650f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractNodeTupleInputFormatTests.java
+++ /dev/null
@@ -1,612 +0,0 @@
-/*
- * 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 java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Abstract node tuple input format tests
- * 
- * 
- * 
- * @param <TValue>
- * @param <T>
- */
-public abstract class AbstractNodeTupleInputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleInputFormatTests.class);
-
-    protected static final int EMPTY_SIZE = 0, SMALL_SIZE = 100, LARGE_SIZE = 10000, BAD_SIZE = 100, MIXED_SIZE = 100;
-    protected static final String EMPTY = "empty";
-    protected static final String SMALL = "small";
-    protected static final String LARGE = "large";
-    protected static final String BAD = "bad";
-    protected static final String MIXED = "mixed";
-
-    /**
-     * Temporary folder for the tests
-     */
-    @Rule
-    public TemporaryFolder folder = new TemporaryFolder();
-
-    protected File empty, small, large, bad, mixed;
-
-    /**
-     * Prepares the inputs for the tests
-     * 
-     * @throws IOException
-     */
-    @Before
-    public void beforeTest() throws IOException {
-        this.prepareInputs();
-    }
-
-    /**
-     * Cleans up the inputs after each test
-     */
-    @After
-    public void afterTest() {
-        // Should be unnecessary since JUnit will clean up the temporary folder
-        // anyway but best to do this regardless
-        if (empty != null)
-            empty.delete();
-        if (small != null)
-            small.delete();
-        if (large != null)
-            large.delete();
-        if (bad != null)
-            bad.delete();
-        if (mixed != null)
-            mixed.delete();
-    }
-
-    /**
-     * Prepares a fresh configuration
-     * 
-     * @return Configuration
-     */
-    protected Configuration prepareConfiguration() {
-        Configuration config = new Configuration(true);
-        // Nothing else to do
-        return config;
-    }
-
-    /**
-     * Prepares the inputs
-     * 
-     * @throws IOException
-     */
-    protected void prepareInputs() throws IOException {
-        String ext = this.getFileExtension();
-        empty = folder.newFile(EMPTY + ext);
-        this.generateTuples(empty, EMPTY_SIZE);
-        small = folder.newFile(SMALL + ext);
-        this.generateTuples(small, SMALL_SIZE);
-        large = folder.newFile(LARGE + ext);
-        this.generateTuples(large, LARGE_SIZE);
-        bad = folder.newFile(BAD + ext);
-        this.generateBadTuples(bad, BAD_SIZE);
-        mixed = folder.newFile(MIXED + ext);
-        this.generateMixedTuples(mixed, MIXED_SIZE);
-    }
-
-    /**
-     * Gets the extra file extension to add to the filenames
-     * 
-     * @return File extension
-     */
-    protected abstract String getFileExtension();
-
-    /**
-     * Generates tuples used for tests
-     * 
-     * @param f
-     *            File
-     * @param num
-     *            Number of tuples to generate
-     * @throws IOException
-     */
-    protected final void generateTuples(File f, int num) throws IOException {
-        this.generateTuples(this.getOutputStream(f), num);
-    }
-
-    /**
-     * Gets the output stream to use for generating tuples
-     * 
-     * @param f
-     *            File
-     * @return Output Stream
-     * @throws IOException
-     */
-    protected OutputStream getOutputStream(File f) throws IOException {
-        return new FileOutputStream(f, false);
-    }
-
-    /**
-     * Generates tuples used for tests
-     * 
-     * @param output
-     *            Output Stream to write to
-     * @param num
-     *            Number of tuples to generate
-     * @throws IOException
-     */
-    protected abstract void generateTuples(OutputStream output, int num) throws IOException;
-
-    /**
-     * Generates bad tuples used for tests
-     * 
-     * @param f
-     *            File
-     * @param num
-     *            Number of bad tuples to generate
-     * @throws IOException
-     */
-    protected final void generateBadTuples(File f, int num) throws IOException {
-        this.generateBadTuples(this.getOutputStream(f), num);
-    }
-
-    /**
-     * Generates bad tuples used for tests
-     * 
-     * @param output
-     *            Output Stream to write to
-     * @param num
-     *            Number of bad tuples to generate
-     * @throws IOException
-     */
-    protected abstract void generateBadTuples(OutputStream output, int num) throws IOException;
-
-    /**
-     * Generates a mixture of good and bad tuples used for tests
-     * 
-     * @param f
-     *            File
-     * @param num
-     *            Number of tuples to generate, they should be a 50/50 mix of
-     *            good and bad tuples
-     * @throws IOException
-     */
-    protected final void generateMixedTuples(File f, int num) throws IOException {
-        this.generateMixedTuples(this.getOutputStream(f), num);
-    }
-
-    /**
-     * Generates a mixture of good and bad tuples used for tests
-     * 
-     * @param output
-     *            Output Stream to write to
-     * @param num
-     *            Number of tuples to generate, they should be a 50/50 mix of
-     *            good and bad tuples
-     * @throws IOException
-     */
-    protected abstract void generateMixedTuples(OutputStream output, int num) throws IOException;
-
-    /**
-     * Adds an input path to the job configuration
-     * 
-     * @param f
-     *            File
-     * @param config
-     *            Configuration
-     * @param job
-     *            Job
-     * @throws IOException
-     */
-    protected void addInputPath(File f, Configuration config, Job job) throws IOException {
-        FileSystem fs = FileSystem.getLocal(config);
-        Path inputPath = fs.makeQualified(new Path(f.getAbsolutePath()));
-        FileInputFormat.addInputPath(job, inputPath);
-    }
-
-    protected final int countTuples(RecordReader<LongWritable, T> reader) throws IOException, InterruptedException {
-        int count = 0;
-
-        // Check initial progress
-        LOG.info(String.format("Initial Reported Progress %f", reader.getProgress()));
-        float progress = reader.getProgress();
-        if (Float.compare(0.0f, progress) == 0) {
-            Assert.assertEquals(0.0d, reader.getProgress(), 0.0d);
-        } else if (Float.compare(1.0f, progress) == 0) {
-            // If reader is reported 1.0 straight away then we expect there to
-            // be no key values
-            Assert.assertEquals(1.0d, reader.getProgress(), 0.0d);
-            Assert.assertFalse(reader.nextKeyValue());
-        } else {
-            Assert.fail(String.format(
-                    "Expected progress of 0.0 or 1.0 before reader has been accessed for first time but got %f",
-                    progress));
-        }
-
-        // Count tuples
-        boolean debug = LOG.isDebugEnabled();
-        while (reader.nextKeyValue()) {
-            count++;
-            progress = reader.getProgress();
-            if (debug)
-                LOG.debug(String.format("Current Reported Progress %f", progress));
-            Assert.assertTrue(String.format("Progress should be in the range 0.0 < p <= 1.0 but got %f", progress),
-                    progress > 0.0f && progress <= 1.0f);
-        }
-        reader.close();
-        LOG.info(String.format("Got %d tuples from this record reader", count));
-
-        // Check final progress
-        LOG.info(String.format("Final Reported Progress %f", reader.getProgress()));
-        Assert.assertEquals(1.0d, reader.getProgress(), 0.0d);
-
-        return count;
-    }
-
-    protected final void checkTuples(RecordReader<LongWritable, T> reader, int expected) throws IOException,
-            InterruptedException {
-        Assert.assertEquals(expected, this.countTuples(reader));
-    }
-
-    /**
-     * Runs a test with a single input
-     * 
-     * @param input
-     *            Input
-     * @param expectedTuples
-     *            Expected tuples
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected final void testSingleInput(File input, int expectedSplits, int expectedTuples) throws IOException,
-            InterruptedException {
-        // Prepare configuration
-        Configuration config = this.prepareConfiguration();
-        this.testSingleInput(config, input, expectedSplits, expectedTuples);
-    }
-
-    /**
-     * Runs a test with a single input
-     * 
-     * @param config
-     *            Configuration
-     * @param input
-     *            Input
-     * @param expectedTuples
-     *            Expected tuples
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected final void testSingleInput(Configuration config, File input, int expectedSplits, int expectedTuples)
-            throws IOException, InterruptedException {
-        // Set up fake job
-        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
-        Job job = Job.getInstance(config);
-        job.setInputFormatClass(inputFormat.getClass());
-        this.addInputPath(input, job.getConfiguration(), job);
-        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-        Assert.assertEquals(1, FileInputFormat.getInputPaths(context).length);
-        NLineInputFormat.setNumLinesPerSplit(job, LARGE_SIZE);
-
-        // Check splits
-        List<InputSplit> splits = inputFormat.getSplits(context);
-        Assert.assertEquals(expectedSplits, splits.size());
-
-        // Check tuples
-        for (InputSplit split : splits) {
-            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
-            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
-            reader.initialize(split, taskContext);
-            this.checkTuples(reader, expectedTuples);
-        }
-    }
-
-    protected abstract InputFormat<LongWritable, T> getInputFormat();
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_01() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(empty, this.canSplitInputs() ? 0 : 1, EMPTY_SIZE);
-    }
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_02() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(small, 1, SMALL_SIZE);
-    }
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_03() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(large, 1, LARGE_SIZE);
-    }
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_04() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(bad, 1, 0);
-    }
-
-    /**
-     * Basic tuples input test
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void single_input_05() throws IOException, InterruptedException, ClassNotFoundException {
-        testSingleInput(mixed, 1, MIXED_SIZE / 2);
-    }
-
-    /**
-     * Tests behaviour when ignoring bad tuples is disabled
-     * 
-     * @throws InterruptedException
-     * @throws IOException
-     */
-    @Test(expected = IOException.class)
-    public final void fail_on_bad_input_01() throws IOException, InterruptedException {
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        Assert.assertFalse(config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true));
-        testSingleInput(config, bad, 1, 0);
-    }
-
-    /**
-     * Tests behaviour when ignoring bad tuples is disabled
-     * 
-     * @throws InterruptedException
-     * @throws IOException
-     */
-    @Test(expected = IOException.class)
-    public final void fail_on_bad_input_02() throws IOException, InterruptedException {
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        Assert.assertFalse(config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true));
-        testSingleInput(config, mixed, 1, MIXED_SIZE / 2);
-    }
-
-    /**
-     * Runs a multiple input test
-     * 
-     * @param inputs
-     *            Inputs
-     * @param expectedSplits
-     *            Number of splits expected
-     * @param expectedTuples
-     *            Number of tuples expected
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    protected final void testMultipleInputs(File[] inputs, int expectedSplits, int expectedTuples) throws IOException,
-            InterruptedException {
-        // Prepare configuration and inputs
-        Configuration config = this.prepareConfiguration();
-
-        // Set up fake job
-        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
-        Job job = Job.getInstance(config);
-        job.setInputFormatClass(inputFormat.getClass());
-        for (File input : inputs) {
-            this.addInputPath(input, job.getConfiguration(), job);
-        }
-        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-        Assert.assertEquals(inputs.length, FileInputFormat.getInputPaths(context).length);
-        NLineInputFormat.setNumLinesPerSplit(job, expectedTuples);
-
-        // Check splits
-        List<InputSplit> splits = inputFormat.getSplits(context);
-        Assert.assertEquals(expectedSplits, splits.size());
-
-        // Check tuples
-        int count = 0;
-        for (InputSplit split : splits) {
-            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
-            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
-            reader.initialize(split, taskContext);
-            count += this.countTuples(reader);
-        }
-        Assert.assertEquals(expectedTuples, count);
-    }
-
-    /**
-     * tuples test with multiple inputs
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void multiple_inputs_01() throws IOException, InterruptedException, ClassNotFoundException {
-        testMultipleInputs(new File[] { empty, small, large }, this.canSplitInputs() ? 2 : 3, EMPTY_SIZE + SMALL_SIZE
-                + LARGE_SIZE);
-    }
-
-    /**
-     * tuples test with multiple inputs
-     * 
-     * @throws IOException
-     * @throws ClassNotFoundException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void multiple_inputs_02() throws IOException, InterruptedException, ClassNotFoundException {
-        testMultipleInputs(new File[] { folder.getRoot() }, this.canSplitInputs() ? 4 : 5, EMPTY_SIZE + SMALL_SIZE
-                + LARGE_SIZE + (MIXED_SIZE / 2));
-    }
-
-    protected final void testSplitInputs(Configuration config, File[] inputs, int expectedSplits, int expectedTuples)
-            throws IOException, InterruptedException {
-        // Set up fake job
-        InputFormat<LongWritable, T> inputFormat = this.getInputFormat();
-        Job job = Job.getInstance(config);
-        job.setInputFormatClass(inputFormat.getClass());
-        for (File input : inputs) {
-            this.addInputPath(input, job.getConfiguration(), job);
-        }
-        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-        Assert.assertEquals(inputs.length, FileInputFormat.getInputPaths(context).length);
-
-        // Check splits
-        List<InputSplit> splits = inputFormat.getSplits(context);
-        Assert.assertEquals(expectedSplits, splits.size());
-
-        // Check tuples
-        int count = 0;
-        for (InputSplit split : splits) {
-            // Validate split
-            Assert.assertTrue(this.isValidSplit(split, config));
-
-            // Read split
-            TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
-            RecordReader<LongWritable, T> reader = inputFormat.createRecordReader(split, taskContext);
-            reader.initialize(split, taskContext);
-            count += this.countTuples(reader);
-        }
-        Assert.assertEquals(expectedTuples, count);
-    }
-
-    /**
-     * Determines whether an input split is valid
-     * 
-     * @param split
-     *            Input split
-     * @return True if a valid split, false otherwise
-     * @throws IOException
-     */
-    protected boolean isValidSplit(InputSplit split, Configuration config) throws IOException {
-        return split instanceof FileSplit;
-    }
-
-    /**
-     * Indicates whether inputs can be split, defaults to true
-     * 
-     * @return Whether inputs can be split
-     */
-    protected boolean canSplitInputs() {
-        return true;
-    }
-
-    /**
-     * Tests for input splitting
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public final void split_input_01() throws IOException, InterruptedException, ClassNotFoundException {
-        Assume.assumeTrue(this.canSplitInputs());
-
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
-        this.testSplitInputs(config, new File[] { small }, 100, SMALL_SIZE);
-    }
-
-    /**
-     * Tests for input splitting
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public final void split_input_02() throws IOException, InterruptedException, ClassNotFoundException {
-        Assume.assumeTrue(this.canSplitInputs());
-
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        config.setLong(NLineInputFormat.LINES_PER_MAP, 10);
-        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
-        this.testSplitInputs(config, new File[] { small }, 10, SMALL_SIZE);
-    }
-
-    /**
-     * Tests for input splitting
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public final void split_input_03() throws IOException, InterruptedException, ClassNotFoundException {
-        Assume.assumeTrue(this.canSplitInputs());
-
-        Configuration config = this.prepareConfiguration();
-        config.setBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, false);
-        config.setLong(NLineInputFormat.LINES_PER_MAP, 100);
-        Assert.assertEquals(Integer.MAX_VALUE, config.getInt(HadoopIOConstants.MAX_LINE_LENGTH, Integer.MAX_VALUE));
-        this.testSplitInputs(config, new File[] { large }, 100, LARGE_SIZE);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
deleted file mode 100644
index 78d7f33..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractQuadsInputFormatTests.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for Quad input formats
- * 
- *
- */
-public abstract class AbstractQuadsInputFormatTests extends AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected void generateTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-    
-    @Override
-    protected void generateBadTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write("<http://broken\n".getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
-        boolean bad = false;
-        for (int i = 0; i < num; i++, bad = !bad) {
-            if (bad) {
-                output.write("<http://broken\n".getBytes(utf8));
-            } else {
-                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
-            }
-        }
-        output.flush();
-        output.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
deleted file mode 100644
index 65a9889..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractTriplesInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for Triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractTriplesInputFormatTests extends AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected void generateTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "<http://broken\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
-        boolean bad = false;
-        for (int i = 0; i < num; i++, bad = !bad) {
-            if (bad) {
-                output.write("<http://broken\n".getBytes(utf8));
-            } else {
-                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
-            }
-        }
-        output.flush();
-        output.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
deleted file mode 100644
index 0b6cfde..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileQuadInputFormatTests.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-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.query.Dataset;
-import com.hp.hpl.jena.query.DatasetFactory;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ModelFactory;
-import com.hp.hpl.jena.rdf.model.Property;
-import com.hp.hpl.jena.rdf.model.Resource;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for Quad input formats
- * 
- * 
- * 
- */
-public abstract class AbstractWholeFileQuadInputFormatTests extends AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-
-    private void writeTuples(Dataset ds, OutputStream output) {
-        RDFDataMgr.write(output, ds, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
-    }
-
-    /**
-     * Gets the RDF language to write out generate tuples in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-    private void writeGoodTuples(OutputStream output, int num) throws IOException {
-        Dataset ds = DatasetFactory.createMem();
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num; i++) {
-            if (i % 100 == 0) {
-                ds.addNamedModel("http://example.org/graphs/" + (i / 100), m);
-                m = ModelFactory.createDefaultModel();
-            }
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        if (!m.isEmpty()) {
-            ds.addNamedModel("http://example.org/graphs/extra", m);
-        }
-        this.writeTuples(ds, output);
-    }
-
-    @Override
-    protected final void generateTuples(OutputStream output, int num) throws IOException {
-        this.writeGoodTuples(output, num);
-        output.close();
-    }
-
-    @Override
-    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
-        // Write good data
-        this.writeGoodTuples(output, num / 2);
-
-        // Write junk data
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num / 2; i++) {
-            output.write(junk);
-        }
-
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
deleted file mode 100644
index b68d662..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileTripleInputFormatTests.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-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.Triple;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ModelFactory;
-import com.hp.hpl.jena.rdf.model.Property;
-import com.hp.hpl.jena.rdf.model.Resource;
-
-/**
- * Abstract tests for Triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractWholeFileTripleInputFormatTests extends AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-    
-    private void writeTuples(Model m, OutputStream output) {
-        RDFDataMgr.write(output, m, this.getRdfLanguage());
-    }
-        
-    /**
-     * Gets the RDF language to write out generate tuples in
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-    
-    @Override
-    protected final void generateTuples(OutputStream output, int num) throws IOException {
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num; i++) {
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        this.writeTuples(m, output);
-        output.close();
-    }
-    
-    @Override
-    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
-        // Write good data
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num / 2; i++) {
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        this.writeTuples(m, output);
-        
-        // Write junk data
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num / 2; i++) {
-            output.write(junk);
-        }
-        
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
deleted file mode 100644
index 4bb0939..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractBlankNodeTests.java
+++ /dev/null
@@ -1,636 +0,0 @@
-/*
- * 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.bnodes;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.nio.file.StandardCopyOption;
-import java.nio.file.attribute.FileAttribute;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.log4j.BasicConfigurator;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * Test case that embodies the scenario described in JENA-820
- */
-@SuppressWarnings("unused")
-public abstract class AbstractBlankNodeTests<T, TValue extends AbstractNodeTupleWritable<T>> {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractBlankNodeTests.class);
-
-    @BeforeClass
-    public static void setup() {
-        // Enable if you need to diagnose test failures
-        // Useful since it includes printing the file names of the temporary
-        // files being used
-        // BasicConfigurator.resetConfiguration();
-        // BasicConfigurator.configure();
-    }
-
-    /**
-     * Gets the extension for the initial input files
-     * 
-     * @return Extension including the {@code .}
-     */
-    protected abstract String getInitialInputExtension();
-
-    /**
-     * Creates a tuple
-     * 
-     * @param s
-     *            Subject
-     * @param p
-     *            Predicate
-     * @param o
-     *            Object
-     * @return Tuple
-     */
-    protected abstract T createTuple(Node s, Node p, Node o);
-
-    /**
-     * Writes out the given tuples to the given file
-     * 
-     * @param f
-     *            File
-     * @param tuples
-     *            Tuples
-     * @throws FileNotFoundException
-     */
-    protected abstract void writeTuples(File f, List<T> tuples) throws FileNotFoundException;
-
-    /**
-     * Creates the input format for reading the initial inputs
-     * 
-     * @return Input format
-     */
-    protected abstract InputFormat<LongWritable, TValue> createInitialInputFormat();
-
-    /**
-     * Creates the output format for writing the intermediate output
-     * 
-     * @return Output format
-     */
-    protected abstract OutputFormat<LongWritable, TValue> createIntermediateOutputFormat();
-
-    /**
-     * Creates the input format for reading the intermediate outputs back in
-     * 
-     * @return Input format
-     */
-    protected abstract InputFormat<LongWritable, TValue> createIntermediateInputFormat();
-
-    /**
-     * Gets the subject of the tuple
-     * 
-     * @param value
-     *            Tuple
-     * @return Subject
-     */
-    protected abstract Node getSubject(T value);
-
-    /**
-     * Gets whether the format being tested respects the RIOT
-     * {@link ParserProfile}
-     * 
-     * @return True if parser profile is respected, false otherwise
-     */
-    protected boolean respectsParserProfile() {
-        return true;
-    }
-
-    /**
-     * Gets whether the format being tested preserves blank node identity
-     * 
-     * @return True if identity is presereved, false otherwise
-     */
-    protected boolean preservesBlankNodeIdentity() {
-        return false;
-    }
-
-    /**
-     * Test that starts with two blank nodes with the same identity in a single
-     * file, splits them over two files and checks that we can workaround
-     * JENA-820 successfully by setting the
-     * {@link RdfIOConstants#GLOBAL_BNODE_IDENTITY} flag for our subsequent job
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public final void blank_node_divergence_01() throws IOException, InterruptedException {
-        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
-        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
-        
-        // Temporary files
-        File a = File.createTempFile("bnode_divergence", getInitialInputExtension());
-        File intermediateOutputDir = Files.createTempDirectory("bnode_divergence", new FileAttribute[0]).toFile();
-
-        try {
-            // Prepare the input data
-            // Two mentions of the same blank node in the same file
-            List<T> tuples = new ArrayList<>();
-            Node bnode = NodeFactory.createAnon();
-            Node pred = NodeFactory.createURI("http://example.org/predicate");
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
-            writeTuples(a, tuples);
-
-            // Set up fake job which will process the file as a single split
-            Configuration config = new Configuration(true);
-            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
-            Job job = Job.getInstance(config);
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()));
-            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            List<InputSplit> splits = inputFormat.getSplits(context);
-            Assert.assertEquals(1, splits.size());
-
-            for (InputSplit split : splits) {
-                // Initialize the input reading
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        createAttemptID(1, 1, 1));
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                // Copy the input to the output - each triple goes to a separate
-                // output file
-                // This is how we force multiple files to be produced
-                int taskID = 1;
-                while (reader.nextKeyValue()) {
-                    // Prepare the output writing
-                    OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
-                    TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                            createAttemptID(1, ++taskID, 1));
-                    RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
-
-                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
-                    writer.close(outputTaskContext);
-                }
-            }
-
-            // Promote outputs from temporary status
-            promoteInputs(intermediateOutputDir);
-
-            // Now we need to create a subsequent job that reads the
-            // intermediate outputs
-            // As described in JENA-820 at this point the blank nodes are
-            // consistent, however when we read them from different files they
-            // by default get treated as different nodes and so the blank nodes
-            // diverge which is incorrect and undesirable behaviour in
-            // multi-stage pipelines
-            System.out.println(intermediateOutputDir.getAbsolutePath());
-            job = Job.getInstance(config);
-            inputFormat = createIntermediateInputFormat();
-            job.setInputFormatClass(inputFormat.getClass());
-            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
-
-            // Enabling this flag works around the JENA-820 issue
-            job.getConfiguration().setBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, true);
-            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            splits = inputFormat.getSplits(context);
-            Assert.assertEquals(2, splits.size());
-
-            // Expect to end up with a single blank node
-            Set<Node> nodes = new HashSet<Node>();
-            for (InputSplit split : splits) {
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        new TaskAttemptID());
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                while (reader.nextKeyValue()) {
-                    nodes.add(getSubject(reader.getCurrentValue().get()));
-                }
-            }
-            // Nodes should not have diverged
-            Assert.assertEquals(1, nodes.size());
-
-        } finally {
-            a.delete();
-            deleteDirectory(intermediateOutputDir);
-        }
-    }
-
-    /**
-     * Test that starts with two blank nodes with the same identity in a single
-     * file, splits them over two files and shows that they diverge in the
-     * subsequent job when the JENA-820 workaround is not enabled
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void blank_node_divergence_02() throws IOException, InterruptedException {
-        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
-        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
-        
-        // Temporary files
-        File a = File.createTempFile("bnode_divergence", getInitialInputExtension());
-        File intermediateOutputDir = Files.createTempDirectory("bnode_divergence", new FileAttribute[0]).toFile();
-
-        try {
-            // Prepare the input data
-            // Two mentions of the same blank node in the same file
-            List<T> tuples = new ArrayList<>();
-            Node bnode = NodeFactory.createAnon();
-            Node pred = NodeFactory.createURI("http://example.org/predicate");
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
-            writeTuples(a, tuples);
-
-            // Set up fake job which will process the file as a single split
-            Configuration config = new Configuration(true);
-            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
-            Job job = Job.getInstance(config);
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()));
-            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            List<InputSplit> splits = inputFormat.getSplits(context);
-            Assert.assertEquals(1, splits.size());
-
-            for (InputSplit split : splits) {
-                // Initialize the input reading
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        createAttemptID(1, 1, 1));
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                // Copy the input to the output - each triple goes to a separate
-                // output file
-                // This is how we force multiple files to be produced
-                int taskID = 1;
-                while (reader.nextKeyValue()) {
-                    // Prepare the output writing
-                    OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
-                    TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                            createAttemptID(1, ++taskID, 1));
-                    RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
-
-                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
-                    writer.close(outputTaskContext);
-                }
-            }
-
-            // Promote outputs from temporary status
-            promoteInputs(intermediateOutputDir);
-
-            // Now we need to create a subsequent job that reads the
-            // intermediate outputs
-            // As described in JENA-820 at this point the blank nodes are
-            // consistent, however when we read them from different files they
-            // by default get treated as different nodes and so the blank nodes
-            // diverge which is incorrect and undesirable behaviour in
-            // multi-stage pipelines. However it is the default behaviour
-            // because when we start from external inputs we want them to be
-            // file scoped.
-            System.out.println(intermediateOutputDir.getAbsolutePath());
-            job = Job.getInstance(config);
-            inputFormat = createIntermediateInputFormat();
-            job.setInputFormatClass(inputFormat.getClass());
-            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
-
-            // Make sure JENA-820 flag is disabled
-            job.getConfiguration().setBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, false);
-            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            splits = inputFormat.getSplits(context);
-            Assert.assertEquals(2, splits.size());
-
-            // Expect to end up with a single blank node
-            Set<Node> nodes = new HashSet<Node>();
-            for (InputSplit split : splits) {
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        new TaskAttemptID());
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                while (reader.nextKeyValue()) {
-                    nodes.add(getSubject(reader.getCurrentValue().get()));
-                }
-            }
-            // Nodes should have diverged
-            Assert.assertEquals(2, nodes.size());
-
-        } finally {
-            a.delete();
-            deleteDirectory(intermediateOutputDir);
-        }
-    }
-
-    /**
-     * Test that starts with two blank nodes in two different files and checks
-     * that writing them to a single file does not conflate them
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void blank_node_identity_01() throws IOException, InterruptedException {
-        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
-        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
-        
-        // Temporary files
-        File a = File.createTempFile("bnode_identity", getInitialInputExtension());
-        File b = File.createTempFile("bnode_identity", getInitialInputExtension());
-        File intermediateOutputDir = Files.createTempDirectory("bnode_identity", new FileAttribute[0]).toFile();
-
-        try {
-            // Prepare the input data
-            // Different blank nodes in different files
-            List<T> tuples = new ArrayList<>();
-            Node bnode1 = NodeFactory.createAnon();
-            Node bnode2 = NodeFactory.createAnon();
-            Node pred = NodeFactory.createURI("http://example.org/predicate");
-
-            tuples.add(createTuple(bnode1, pred, NodeFactory.createLiteral("first")));
-            writeTuples(a, tuples);
-
-            tuples.clear();
-            tuples.add(createTuple(bnode2, pred, NodeFactory.createLiteral("second")));
-            writeTuples(b, tuples);
-
-            // Set up fake job which will process the two files
-            Configuration config = new Configuration(true);
-            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
-            Job job = Job.getInstance(config);
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()), new Path(b.getAbsolutePath()));
-            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            List<InputSplit> splits = inputFormat.getSplits(context);
-            Assert.assertEquals(2, splits.size());
-
-            // Prepare the output writing - putting all output to a single file
-            OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
-            TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(), createAttemptID(
-                    1, 2, 1));
-            RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
-
-            for (InputSplit split : splits) {
-                // Initialize the input reading
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        createAttemptID(1, 1, 1));
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                // Copy the input to the output - all triples go to a single
-                // output
-                while (reader.nextKeyValue()) {
-                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
-                }
-            }
-            writer.close(outputTaskContext);
-
-            // Promote outputs from temporary status
-            promoteInputs(intermediateOutputDir);
-
-            // Now we need to create a subsequent job that reads the
-            // intermediate outputs
-            // The Blank nodes should have been given separate identities so we
-            // should not be conflating them, this is the opposite problem to
-            // that described in JENA-820
-            System.out.println(intermediateOutputDir.getAbsolutePath());
-            job = Job.getInstance(config);
-            inputFormat = createIntermediateInputFormat();
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            splits = inputFormat.getSplits(context);
-            Assert.assertEquals(1, splits.size());
-
-            // Expect to end up with a single blank node
-            Set<Node> nodes = new HashSet<Node>();
-            for (InputSplit split : splits) {
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        new TaskAttemptID());
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                while (reader.nextKeyValue()) {
-                    nodes.add(getSubject(reader.getCurrentValue().get()));
-                }
-            }
-            // Nodes must not have converged
-            Assert.assertEquals(2, nodes.size());
-
-        } finally {
-            a.delete();
-            b.delete();
-            deleteDirectory(intermediateOutputDir);
-        }
-    }
-
-    /**
-     * Test that starts with two blank nodes in two different files and checks
-     * that writing them to a single file does not conflate them
-     * 
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    @Test
-    public void blank_node_identity_02() throws IOException, InterruptedException {
-        Assume.assumeTrue("Requires ParserProfile be respected", this.respectsParserProfile());
-        Assume.assumeFalse("Requires that Blank Node identity not be preserved", this.preservesBlankNodeIdentity());
-        
-        // Temporary files
-        File a = File.createTempFile("bnode_identity", getInitialInputExtension());
-        File b = File.createTempFile("bnode_identity", getInitialInputExtension());
-        File intermediateOutputDir = Files.createTempDirectory("bnode_identity", new FileAttribute[0]).toFile();
-
-        try {
-            // Prepare the input data
-            // Same blank node but in different files so must be treated as
-            // different blank nodes and not converge
-            List<T> tuples = new ArrayList<>();
-            Node bnode = NodeFactory.createAnon();
-            Node pred = NodeFactory.createURI("http://example.org/predicate");
-
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("first")));
-            writeTuples(a, tuples);
-
-            tuples.clear();
-            tuples.add(createTuple(bnode, pred, NodeFactory.createLiteral("second")));
-            writeTuples(b, tuples);
-
-            // Set up fake job which will process the two files
-            Configuration config = new Configuration(true);
-            InputFormat<LongWritable, TValue> inputFormat = createInitialInputFormat();
-            Job job = Job.getInstance(config);
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(a.getAbsolutePath()), new Path(b.getAbsolutePath()));
-            FileOutputFormat.setOutputPath(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            List<InputSplit> splits = inputFormat.getSplits(context);
-            Assert.assertEquals(2, splits.size());
-
-            // Prepare the output writing - putting all output to a single file
-            OutputFormat<LongWritable, TValue> outputFormat = createIntermediateOutputFormat();
-            TaskAttemptContext outputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(), createAttemptID(
-                    1, 2, 1));
-            RecordWriter<LongWritable, TValue> writer = outputFormat.getRecordWriter(outputTaskContext);
-
-            for (InputSplit split : splits) {
-                // Initialize the input reading
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        createAttemptID(1, 1, 1));
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                // Copy the input to the output - all triples go to a single
-                // output
-                while (reader.nextKeyValue()) {
-                    writer.write(reader.getCurrentKey(), reader.getCurrentValue());
-                }
-            }
-            writer.close(outputTaskContext);
-
-            // Promote outputs from temporary status
-            promoteInputs(intermediateOutputDir);
-
-            // Now we need to create a subsequent job that reads the
-            // intermediate outputs
-            // The Blank nodes should have been given separate identities so we
-            // should not be conflating them, this is the opposite problem to
-            // that described in JENA-820
-            System.out.println(intermediateOutputDir.getAbsolutePath());
-            job = Job.getInstance(config);
-            inputFormat = createIntermediateInputFormat();
-            job.setInputFormatClass(inputFormat.getClass());
-            NLineInputFormat.setNumLinesPerSplit(job, 100);
-            FileInputFormat.setInputPaths(job, new Path(intermediateOutputDir.getAbsolutePath()));
-            context = new JobContextImpl(job.getConfiguration(), job.getJobID());
-
-            // Get the splits
-            splits = inputFormat.getSplits(context);
-            Assert.assertEquals(1, splits.size());
-
-            // Expect to end up with a single blank node
-            Set<Node> nodes = new HashSet<Node>();
-            for (InputSplit split : splits) {
-                TaskAttemptContext inputTaskContext = new TaskAttemptContextImpl(job.getConfiguration(),
-                        new TaskAttemptID());
-                RecordReader<LongWritable, TValue> reader = inputFormat.createRecordReader(split, inputTaskContext);
-                reader.initialize(split, inputTaskContext);
-
-                while (reader.nextKeyValue()) {
-                    nodes.add(getSubject(reader.getCurrentValue().get()));
-                }
-            }
-            // Nodes must not diverge
-            Assert.assertEquals(2, nodes.size());
-
-        } finally {
-            a.delete();
-            b.delete();
-            deleteDirectory(intermediateOutputDir);
-        }
-    }
-
-    private TaskAttemptID createAttemptID(int jobID, int taskID, int id) {
-        return new TaskAttemptID("outputTest", jobID, TaskType.MAP, taskID, 1);
-    }
-
-    private void promoteInputs(File baseDir) throws IOException {
-        for (File f : baseDir.listFiles()) {
-            if (f.isDirectory()) {
-                promoteInputs(baseDir, f);
-            }
-        }
-    }
-
-    private void promoteInputs(File targetDir, File dir) throws IOException {
-        java.nio.file.Path target = Paths.get(targetDir.toURI());
-        for (File f : dir.listFiles()) {
-            if (f.isDirectory()) {
-                promoteInputs(targetDir, f);
-            } else {
-                LOGGER.debug("Moving {} to {}", f.getAbsolutePath(), target.resolve(f.getName()));
-                Files.move(Paths.get(f.toURI()), target.resolve(f.getName()), StandardCopyOption.REPLACE_EXISTING);
-            }
-        }
-
-        // Remove defunct sub-directory
-        dir.delete();
-    }
-
-    private void deleteDirectory(File dir) throws IOException {
-        for (File f : dir.listFiles()) {
-            if (f.isFile())
-                f.delete();
-            if (f.isDirectory())
-                deleteDirectory(f);
-        }
-        dir.delete();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
deleted file mode 100644
index bbd6742..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/AbstractTripleBlankNodeTests.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.bnodes;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.util.List;
-
-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.Node;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.graph.GraphFactory;
-
-/**
- *
- */
-public abstract class AbstractTripleBlankNodeTests extends AbstractBlankNodeTests<Triple, TripleWritable> {
-    
-    /**
-     * Gets the language to use
-     * 
-     * @return Language
-     */
-    protected abstract Lang getLanguage();
-
-    @Override
-    protected Triple createTuple(Node s, Node p, Node o) {
-        return new Triple(s, p, o);
-    }
-
-    @Override
-    protected void writeTuples(File f, List<Triple> tuples) throws FileNotFoundException {
-        Graph g = GraphFactory.createGraphMem();
-        for (Triple t : tuples) {
-            g.add(t);
-        }
-        RDFDataMgr.write(new FileOutputStream(f), g, getLanguage());
-    }
-
-    @Override
-    protected Node getSubject(Triple value) {
-        return value.getSubject();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
deleted file mode 100644
index f234127..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/JsonLdTripleBlankNodeTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDTripleInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.jsonld.JsonLDTripleOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link RdfJsonInputFormat}
- */
-public class JsonLdTripleBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.JSONLD;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".jsonld";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new JsonLDTripleInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new JsonLDTripleOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new JsonLDTripleInputFormat();
-    }
-
-    @Override
-    protected boolean respectsParserProfile() {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
deleted file mode 100644
index 4c350c7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/NTriplesBlankNodeTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.ntriples.NTriplesOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link NTriplesInputFormat}
- */
-public class NTriplesBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".nt";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new NTriplesInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new NTriplesOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new NTriplesInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
deleted file mode 100644
index 2be1e0e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfJsonBlankNodeTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.rdfjson.RdfJsonOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link RdfJsonInputFormat}
- */
-public class RdfJsonBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.RDFJSON;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".rj";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new RdfJsonInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new RdfJsonOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new RdfJsonInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
deleted file mode 100644
index d6f32a2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfThriftBlankNodeTest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftTripleInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.thrift.ThriftTripleOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Tests blank node divergence when using the {@link RdfThriftInputFormat}
- */
-public class RdfThriftBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".trdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new ThriftTripleInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new ThriftTripleOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new ThriftTripleInputFormat();
-    }
-
-    @Override
-    protected boolean respectsParserProfile() {
-        return false;
-    }
-    
-    @Override
-    protected boolean preservesBlankNodeIdentity() {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
deleted file mode 100644
index da70007..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.rdfxml.RdfXmlInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.rdfxml.RdfXmlOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link RdfXmlInputFormat}
- */
-public class RdfXmlBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.RDFXML;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".rdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new RdfXmlInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new RdfXmlOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new RdfXmlInputFormat();
-    }
-
-    @Override
-    protected boolean respectsParserProfile() {
-        return false;
-    }
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java
new file mode 100644
index 0000000..7c56d1c
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountMapper.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * Abstract mapper class for mappers which split node tuple values into pairs of
+ * node keys with a long value of 1. Can be used in conjunction with a
+ * {@link NodeCountReducer} to count the usages of each unique node.
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+public abstract class AbstractNodeTupleNodeCountMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, NodeWritable, LongWritable> {
+    
+    private LongWritable initialCount = new LongWritable(1);
+
+    @Override
+    protected void map(TKey key, T value, Context context) throws IOException,
+            InterruptedException {
+        NodeWritable[] ns = this.getNodes(value);
+        for (NodeWritable n : ns) {
+            context.write(n, this.initialCount);
+        }
+    }
+
+    /**
+     * Gets the nodes of the tuple which are to be counted
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] getNodes(T tuple);
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java
new file mode 100644
index 0000000..a5460f6
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/NodeCountReducer.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+
+/**
+ * A reducer which takes node keys with a sequence of longs representing counts
+ * as the values and sums the counts together into pairs consisting of a node
+ * key and a count value.
+ * 
+ * 
+ * 
+ */
+public class NodeCountReducer extends Reducer<NodeWritable, LongWritable, NodeWritable, LongWritable> {
+
+    @Override
+    protected void reduce(NodeWritable key, Iterable<LongWritable> values, Context context) throws IOException,
+            InterruptedException {
+        long count = 0;
+        Iterator<LongWritable> iter = values.iterator();
+        while (iter.hasNext()) {
+            count += iter.next().get();
+        }
+        context.write(key, new LongWritable(count));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java
new file mode 100644
index 0000000..5c2485b
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapper.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper for counting node usages within quads designed primarily for use in
+ * conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadNodeCountMapper<TKey> extends AbstractNodeTupleNodeCountMapper<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java
new file mode 100644
index 0000000..4108841
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapper.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count;
+
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper for counting node usages within triples designed primarily for use
+ * in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleNodeCountMapper<TKey> extends AbstractNodeTupleNodeCountMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java
new file mode 100644
index 0000000..7ea7ead
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/QuadDataTypeCountMapper.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.datatypes;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A mapper for counting data type usages within quads designed primarily for
+ * use in conjunction with {@link NodeCountReducer}
+ * <p>
+ * This mapper extracts the data types for typed literal objects and converts
+ * them into nodes so they can be counted
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadDataTypeCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
+
+    private static final NodeWritable[] EMPTY = new NodeWritable[0];
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Node object = tuple.get().getObject();
+        if (!object.isLiteral())
+            return EMPTY;
+        String dtUri = object.getLiteralDatatypeURI();
+        if (dtUri == null)
+            return EMPTY;
+        return new NodeWritable[] { new NodeWritable(NodeFactory.createURI(dtUri)) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java
new file mode 100644
index 0000000..2c294be
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/datatypes/TripleDataTypeCountMapper.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.datatypes;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A mapper for counting data type usages within triples designed primarily for
+ * use in conjunction with {@link NodeCountReducer}
+ * <p>
+ * This mapper extracts the data types for typed literal objects and converts
+ * them into nodes so they can be counted
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleDataTypeCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
+
+    private static final NodeWritable[] EMPTY = new NodeWritable[0];
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Node object = tuple.get().getObject();
+        if (!object.isLiteral())
+            return EMPTY;
+        String dtUri = object.getLiteralDatatypeURI();
+        if (dtUri == null)
+            return EMPTY;
+        return new NodeWritable[] { new NodeWritable(NodeFactory.createURI(dtUri)) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java
new file mode 100644
index 0000000..ba47765
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/AbstractNodeTupleNamespaceCountMapper.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.namespaces;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+
+import com.hp.hpl.jena.graph.Node;
+
+/**
+ * Abstract mapper class for mappers which split node tuple values and extract
+ * the namespace URIs they use and outputs pairs of namespaces keys with a long
+ * value of 1. Can be used in conjunction with a {@link TextCountReducer} to
+ * count the usages of each unique namespace.
+ * 
+ * 
+ * 
+ * @param <TKey>
+ * @param <TValue>
+ * @param <T>
+ */
+public abstract class AbstractNodeTupleNamespaceCountMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, Text, LongWritable> {
+
+    private LongWritable initialCount = new LongWritable(1);
+    protected static final String NO_NAMESPACE = null;
+
+    @Override
+    protected void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        NodeWritable[] ns = this.getNodes(value);
+        for (NodeWritable n : ns) {
+            String namespace = this.extractNamespace(n);
+            if (namespace != null) {
+                context.write(new Text(namespace), this.initialCount);
+            }
+        }
+    }
+
+    /**
+     * Extracts the namespace from a node
+     * <p>
+     * Finds the URI for the node (if any) and then invokes
+     * {@link #extractNamespace(String)} to extract the actual namespace URI.
+     * </p>
+     * <p>
+     * Derived classes may override this to change the logic of how namespaces
+     * are extracted.
+     * </p>
+     * 
+     * @param nw
+     *            Node
+     * @return Namespace
+     */
+    protected String extractNamespace(NodeWritable nw) {
+        Node n = nw.get();
+        if (n.isBlank() || n.isVariable())
+            return NO_NAMESPACE;
+        if (n.isLiteral()) {
+            String dtUri = n.getLiteralDatatypeURI();
+            if (dtUri == null)
+                return NO_NAMESPACE;
+            return extractNamespace(dtUri);
+        }
+        return extractNamespace(n.getURI());
+    }
+
+    /**
+     * Extracts the namespace from a URI
+     * <p>
+     * First tries to extract a hash based namespace. If that is not possible it
+     * tries to extract a slash based namespace, if this is not possible then
+     * the full URI is returned.
+     * </p>
+     * <p>
+     * Derived classes may override this to change the logic of how namespaces
+     * are extracted.
+     * </p>
+     * 
+     * @param uri
+     *            URI
+     * @return Namespace
+     */
+    protected String extractNamespace(String uri) {
+        if (uri.contains("#")) {
+            // Extract hash namespace
+            return uri.substring(0, uri.lastIndexOf('#') + 1);
+        } else if (uri.contains("/")) {
+            // Ensure that this is not immediately after the scheme component or
+            // at end of URI
+            int index = uri.lastIndexOf('/');
+            int schemeSepIndex = uri.indexOf(':');
+            if (index - schemeSepIndex <= 2 || index == uri.length() - 1) {
+                // Use full URI
+                return uri;
+            }
+
+            // Otherwise safe to extract slash namespace
+            return uri.substring(0, uri.lastIndexOf('/') + 1);
+        } else {
+            // Use full URI
+            return uri;
+        }
+    }
+
+    /**
+     * Gets the nodes of the tuple whose namespaces are to be counted
+     * 
+     * @param tuple
+     *            Tuple
+     * @return Nodes
+     */
+    protected abstract NodeWritable[] getNodes(T tuple);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java
new file mode 100644
index 0000000..97ee546
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/QuadNamespaceCountMapper.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.namespaces;
+
+import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * A mapper for counting namespace usages within quads designed primarily for
+ * use in conjunction with the {@link TextCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class QuadNamespaceCountMapper<TKey> extends AbstractNodeTupleNamespaceCountMapper<TKey, Quad, QuadWritable> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        Quad q = tuple.get();
+        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
+                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java
new file mode 100644
index 0000000..eacef7a
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/namespaces/TripleNamespaceCountMapper.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.namespaces;
+
+import org.apache.jena.hadoop.rdf.mapreduce.TextCountReducer;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * A mapper for counting namespace usages within triples designed primarily for
+ * use in conjunction with the {@link TextCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ */
+public class TripleNamespaceCountMapper<TKey> extends AbstractNodeTupleNamespaceCountMapper<TKey, Triple, TripleWritable> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        Triple t = tuple.get();
+        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
+                new NodeWritable(t.getObject()) };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java
new file mode 100644
index 0000000..ef19623
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadObjectCountMapper.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * A mapper for counting object node usages within quads designed primarily for
+ * use in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadObjectCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java
new file mode 100644
index 0000000..0ef0731
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadPredicateCountMapper.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * A mapper for counting predicate node usages within quads designed primarily
+ * for use in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadPredicateCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getPredicate()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java
new file mode 100644
index 0000000..2b48e79
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/QuadSubjectCountMapper.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * A mapper for counting subject node usages within quads designed primarily for use
+ * in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class QuadSubjectCountMapper<TKey> extends QuadNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(QuadWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getSubject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java
new file mode 100644
index 0000000..16250eb
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleObjectCountMapper.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * A mapper for counting object node usages within triples designed primarily for use
+ * in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleObjectCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getObject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java
new file mode 100644
index 0000000..de97a9b
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TriplePredicateCountMapper.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * A mapper for counting predicate node usages within triples designed primarily
+ * for use in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TriplePredicateCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getPredicate()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java
new file mode 100644
index 0000000..5dc3838
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/count/positional/TripleSubjectCountMapper.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.count.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
+import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
+import org.apache.jena.hadoop.rdf.types.NodeWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * A mapper for counting subject node usages within triples designed primarily for use
+ * in conjunction with {@link NodeCountReducer}
+ * 
+ * 
+ * 
+ * @param <TKey> Key type
+ */
+public class TripleSubjectCountMapper<TKey> extends TripleNodeCountMapper<TKey> {
+
+    @Override
+    protected NodeWritable[] getNodes(TripleWritable tuple) {
+        return new NodeWritable[] { new NodeWritable(tuple.get().getSubject()) };
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java
new file mode 100644
index 0000000..1ecec75
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractNodeTupleFilterMapper.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+
+
+/**
+ * Abstract mapper implementation which helps in filtering tuples from the
+ * input, derived implementations provide an implementation of the
+ * {@link #accepts(TKey, T)}
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ */
+@SuppressWarnings("javadoc")
+public abstract class AbstractNodeTupleFilterMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
+        Mapper<TKey, T, TKey, T> {
+
+    private boolean invert = false;
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        this.invert = context.getConfiguration().getBoolean(RdfMapReduceConstants.FILTER_INVERT, this.invert);
+    }
+
+    @Override
+    protected final void map(TKey key, T value, Context context) throws IOException, InterruptedException {
+        if (this.accepts(key, value)) {
+            if (!this.invert)
+                context.write(key, value);
+        } else if (this.invert) {
+            context.write(key, value);
+        }
+    }
+
+    /**
+     * Gets whether the mapper accepts the key value pair and will pass it as
+     * output
+     * 
+     * @param key
+     *            Key
+     * @param tuple
+     *            Tuple value
+     * @return True if the mapper accepts the given key value pair, false
+     *         otherwise
+     */
+    protected abstract boolean accepts(TKey key, T tuple);
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java
new file mode 100644
index 0000000..3caf051
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractQuadFilterMapper.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract mapper implementation for filtering quads
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractQuadFilterMapper<TKey> extends AbstractNodeTupleFilterMapper<TKey, Quad, QuadWritable> {
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java
new file mode 100644
index 0000000..e99e369
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/AbstractTripleFilterMapper.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract mapper implementation for filtering triples
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractTripleFilterMapper<TKey> extends AbstractNodeTupleFilterMapper<TKey, Triple, TripleWritable> {
+
+}

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

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

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

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

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
new file mode 100644
index 0000000..f5b5876
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractQuadFilterByPositionMapper.java
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.filter.AbstractQuadFilterMapper;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * An abstract triple filter that filters quads based on different criteria for
+ * each position (graph, subject, predicate and object) within the quad.
+ * <p>
+ * By default this implementation eliminates all quads it sees, derived
+ * implementations need to override one or more of the specific accept methods
+ * in order to actually accept some triples. See
+ * {@link QuadFilterByPredicateMapper} for an example implementation.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractQuadFilterByPositionMapper<TKey> extends AbstractQuadFilterMapper<TKey> {
+
+    @Override
+    protected final boolean accepts(Object key, QuadWritable tuple) {
+        Quad q = tuple.get();
+        if (!this.acceptsAllGraphs()) {
+            Node g = q.getGraph();
+            if (!this.acceptsGraph(g))
+                return false;
+        }
+        if (!this.acceptsAllSubjects()) {
+            Node s = q.getSubject();
+            if (!this.acceptsSubject(s))
+                return false;
+        }
+        if (!this.acceptsAllPredicates()) {
+            Node p = q.getPredicate();
+            if (!this.acceptsPredicate(p))
+                return false;
+        }
+        if (!this.acceptsAllObjects()) {
+            Node o = q.getObject();
+            if (!this.acceptsObject(o))
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Gets whether this filter accepts all graphs, if false then the
+     * {@link #acceptsGraph(Node)} method will be called to determine if a
+     * specific graph is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all graphs are accepted, false otherwise
+     */
+    protected boolean acceptsAllGraphs() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific graph is acceptable
+     * 
+     * @param graph
+     *            Graph
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsGraph(Node graph) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all subjects, if false then the
+     * {@link #acceptsSubject(Node)} method will be called to determine if a
+     * specific subject is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all subjects are accepted, false otherwise
+     */
+    protected boolean acceptsAllSubjects() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific subject is acceptable
+     * 
+     * @param subject
+     *            Subject
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsSubject(Node subject) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all predicate, if false then the
+     * {@link #acceptsPredicate(Node)} method will be called to determine if a
+     * specific predicate is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all predicates are accepted, false otherwise
+     */
+    protected boolean acceptsAllPredicates() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific predicate is acceptable
+     * 
+     * @param predicate
+     *            Predicate
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsPredicate(Node predicate) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all objects, if false then the
+     * {@link #acceptsObject(Node)} method will be called to determine if a
+     * specific object is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all objects are accepted, false otherwise
+     */
+    protected boolean acceptsAllObjects() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific object is acceptable
+     * 
+     * @param object
+     *            Object
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsObject(Node object) {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java
new file mode 100644
index 0000000..973d651
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/AbstractTripleFilterByPositionMapper.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import org.apache.jena.hadoop.rdf.mapreduce.filter.AbstractTripleFilterMapper;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * An abstract triple filter that filters triples based on different criteria
+ * for each position (subject, predicate and object) within the triple.
+ * <p>
+ * By default this implementation eliminates all triples it sees, derived
+ * implementations need to override one or more of the specific accept methods
+ * in order to actually accept some triples. See
+ * {@link TripleFilterByPredicateUriMapper} for an example implementation.
+ * </p>
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public abstract class AbstractTripleFilterByPositionMapper<TKey> extends AbstractTripleFilterMapper<TKey> {
+
+    @Override
+    protected final boolean accepts(Object key, TripleWritable tuple) {
+        Triple t = tuple.get();
+        if (!this.acceptsAllSubjects()) {
+            Node s = t.getSubject();
+            if (!this.acceptsSubject(s))
+                return false;
+        }
+        if (!this.acceptsAllPredicates()) {
+            Node p = t.getPredicate();
+            if (!this.acceptsPredicate(p))
+                return false;
+        }
+        if (!this.acceptsAllObjects()) {
+            Node o = t.getObject();
+            if (!this.acceptsObject(o))
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Gets whether this filter accepts all subjects, if false then the
+     * {@link #acceptsSubject(Node)} method will be called to determine if a
+     * specific subject is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all subjects are accepted, false otherwise
+     */
+    protected boolean acceptsAllSubjects() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific subject is acceptable
+     * 
+     * @param subject
+     *            Subject
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsSubject(Node subject) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all predicate, if false then the
+     * {@link #acceptsPredicate(Node)} method will be called to determine if a
+     * specific predicate is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all predicates are accepted, false otherwise
+     */
+    protected boolean acceptsAllPredicates() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific predicate is acceptable
+     * 
+     * @param predicate
+     *            Predicate
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsPredicate(Node predicate) {
+        return false;
+    }
+
+    /**
+     * Gets whether this filter accepts all objects, if false then the
+     * {@link #acceptsObject(Node)} method will be called to determine if a
+     * specific object is acceptable
+     * <p>
+     * Default behaviour if not overridden is to return {@code false}
+     * </p>
+     * 
+     * @return True if all objects are accepted, false otherwise
+     */
+    protected boolean acceptsAllObjects() {
+        return false;
+    }
+
+    /**
+     * Gets whether a specific object is acceptable
+     * 
+     * @param object
+     *            Object
+     * @return True if accepted, false otherwise
+     */
+    protected boolean acceptsObject(Node object) {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
new file mode 100644
index 0000000..1c2b29e
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByGraphUriMapper.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A quad filter which selects quads which have matching subjects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadFilterByGraphUriMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
+
+    private List<Node> graphs = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the subject URIs we are filtering on
+        String[] graphUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_GRAPH_URIS);
+        if (graphUris != null) {
+            for (String graphUri : graphUris) {
+                this.graphs.add(NodeFactory.createURI(graphUri));
+            }
+        }
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsGraph(Node graph) {
+        if (this.graphs.size() == 0)
+            return false;
+        return this.graphs.contains(graph);
+    }
+
+    @Override
+    protected boolean acceptsAllPredicates() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllObjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
new file mode 100644
index 0000000..a3be806
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByObjectUriMapper.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A quad filter which selects quads which have matching objects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadFilterByObjectUriMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
+
+    private List<Node> objects = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the subject URIs we are filtering on
+        String[] objectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_OBJECT_URIS);
+        if (objectUris != null) {
+            for (String objectUri : objectUris) {
+                this.objects.add(NodeFactory.createURI(objectUri));
+            }
+        }
+    }
+    
+    @Override
+    protected boolean acceptsAllGraphs() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsObject(Node object) {
+        if (this.objects.size() == 0)
+            return false;
+        return this.objects.contains(object);
+    }
+
+    @Override
+    protected boolean acceptsAllPredicates() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
new file mode 100644
index 0000000..1be64bf
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterByPredicateMapper.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A quad filter which selects quads which have matching predicates
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadFilterByPredicateMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
+
+    private List<Node> predicates = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the predicate URIs we are filtering on
+        String[] predicateUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS);
+        if (predicateUris != null) {
+            for (String predicateUri : predicateUris) {
+                this.predicates.add(NodeFactory.createURI(predicateUri));
+            }
+        }
+    }
+    
+    @Override
+    protected boolean acceptsAllGraphs() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsPredicate(Node predicate) {
+        if (this.predicates.size() == 0)
+            return false;
+        return this.predicates.contains(predicate);
+    }
+
+    @Override
+    protected boolean acceptsAllObjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
new file mode 100644
index 0000000..2a4f37b
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/QuadFilterBySubjectUriMapper.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A quad filter which selects quads which have matching subjects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class QuadFilterBySubjectUriMapper<TKey> extends AbstractQuadFilterByPositionMapper<TKey> {
+
+    private List<Node> subjects = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the subject URIs we are filtering on
+        String[] subjectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_SUBJECT_URIS);
+        if (subjectUris != null) {
+            for (String subjectUri : subjectUris) {
+                this.subjects.add(NodeFactory.createURI(subjectUri));
+            }
+        }
+    }
+    
+    @Override
+    protected boolean acceptsAllGraphs() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsSubject(Node subject) {
+        if (this.subjects.size() == 0)
+            return false;
+        return this.subjects.contains(subject);
+    }
+
+    @Override
+    protected boolean acceptsAllPredicates() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllObjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java
new file mode 100644
index 0000000..035aabc
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByObjectUriMapper.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A triple filter which selects triples which have matching objects
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleFilterByObjectUriMapper<TKey> extends AbstractTripleFilterByPositionMapper<TKey> {
+
+    private List<Node> objects = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the subject URIs we are filtering on
+        String[] objectUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_OBJECT_URIS);
+        if (objectUris != null) {
+            for (String objectUri : objectUris) {
+                this.objects.add(NodeFactory.createURI(objectUri));
+            }
+        }
+    }
+
+    @Override
+    protected boolean acceptsObject(Node object) {
+        if (this.objects.size() == 0)
+            return false;
+        return this.objects.contains(object);
+    }
+
+    @Override
+    protected boolean acceptsAllPredicates() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java
----------------------------------------------------------------------
diff --git a/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java
new file mode 100644
index 0000000..061a3e7
--- /dev/null
+++ b/jena-elephas/jena-elephas-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/filter/positional/TripleFilterByPredicateUriMapper.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.mapreduce.filter.positional;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.mapreduce.RdfMapReduceConstants;
+
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+
+/**
+ * A triple filter which selects triples which have matching predicates
+ * 
+ * 
+ * 
+ * @param <TKey>
+ *            Key type
+ */
+public class TripleFilterByPredicateUriMapper<TKey> extends AbstractTripleFilterByPositionMapper<TKey> {
+
+    private List<Node> predicates = new ArrayList<Node>();
+
+    @Override
+    protected void setup(Context context) throws IOException, InterruptedException {
+        super.setup(context);
+
+        // Get the predicate URIs we are filtering on
+        String[] predicateUris = context.getConfiguration().getStrings(RdfMapReduceConstants.FILTER_PREDICATE_URIS);
+        if (predicateUris != null) {
+            for (String predicateUri : predicateUris) {
+                this.predicates.add(NodeFactory.createURI(predicateUri));
+            }
+        }
+    }
+
+    @Override
+    protected boolean acceptsAllSubjects() {
+        return true;
+    }
+
+    @Override
+    protected boolean acceptsPredicate(Node predicate) {
+        if (this.predicates.size() == 0)
+            return false;
+        return this.predicates.contains(predicate);
+    }
+
+    @Override
+    protected boolean acceptsAllObjects() {
+        return true;
+    }
+}


[17/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
deleted file mode 100644
index 7b04ef4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
+++ /dev/null
@@ -1,310 +0,0 @@
-/*
- * 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.registry;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.ServiceLoader;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * A registry which is used by various classes to dynamically select record
- * readers and writers based on a provided {@link Lang}
- * <p>
- * Readers and writers are dynamically discovered using the Java
- * {@link ServiceLoader} mechanism. This will look for files under
- * {@code META-INF/services} named
- * {@code org.apache.jena.hadoop.rdf.io.registry.ReaderFactory} and
- * {@code org.apache.jena.hadoop.rdf.io.registry.WriterFactory}. This follows
- * the standard {@linkplain ServiceLoader} format of provided one class name per
- * line which implements the relevant interface.
- * </p>
- * 
- */
-public class HadoopRdfIORegistry {
-
-    private static Map<Lang, ReaderFactory> readerFactories = new HashMap<>();
-    private static Map<Lang, WriterFactory> writerFactories = new HashMap<>();
-    private static boolean init = false;
-
-    static {
-        init();
-    }
-
-    private static synchronized void init() {
-        if (init)
-            return;
-
-        // Dynamically load and register reader factories
-        ServiceLoader<ReaderFactory> readerFactoryLoader = ServiceLoader.load(ReaderFactory.class);
-        Iterator<ReaderFactory> readerFactoryIterator = readerFactoryLoader.iterator();
-        while (readerFactoryIterator.hasNext()) {
-            ReaderFactory f = readerFactoryIterator.next();
-            addReaderFactory(f);
-        }
-
-        // Dynamically load and register writer factories
-        ServiceLoader<WriterFactory> writerFactoryLoader = ServiceLoader.load(WriterFactory.class);
-        Iterator<WriterFactory> writerFactoryIterator = writerFactoryLoader.iterator();
-        while (writerFactoryIterator.hasNext()) {
-            WriterFactory f = writerFactoryIterator.next();
-            addWriterFactory(f);
-        }
-
-        init = true;
-    }
-
-    /**
-     * Resets the registry to the default configuration
-     */
-    public static synchronized void reset() {
-        if (!init)
-            return;
-
-        init = false;
-        init();
-    }
-
-    /**
-     * Registers the reader factory for all the languages it declares itself as
-     * supporting
-     * 
-     * @param f
-     *            Reader factory
-     */
-    public static void addReaderFactory(ReaderFactory f) {
-        if (f == null)
-            throw new NullPointerException("Factory cannot be null");
-
-        readerFactories.put(f.getPrimaryLanguage(), f);
-        for (Lang altLang : f.getAlternativeLanguages()) {
-            readerFactories.put(altLang, f);
-        }
-    }
-
-    /**
-     * Registers the writer factory for all the languages it declares itself as
-     * supporting
-     * 
-     * @param f
-     *            Writer factory
-     */
-    public static void addWriterFactory(WriterFactory f) {
-        if (f == null)
-            throw new NullPointerException("Factory cannot be null");
-
-        writerFactories.put(f.getPrimaryLanguage(), f);
-        for (Lang altLang : f.getAlternativeLanguages()) {
-            writerFactories.put(altLang, f);
-        }
-    }
-
-    /**
-     * Gets whether there is a quad reader available for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return True if available, false otherwise
-     */
-    public static boolean hasQuadReader(Lang lang) {
-        if (lang == null)
-            return false;
-
-        ReaderFactory f = readerFactories.get(lang);
-        if (f == null)
-            return false;
-        return f.canReadQuads();
-    }
-
-    /**
-     * Gets whether there is a triple reader available for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return True if available, false otherwise
-     */
-    public static boolean hasTriplesReader(Lang lang) {
-        if (lang == null)
-            return false;
-
-        ReaderFactory f = readerFactories.get(lang);
-        if (f == null)
-            return false;
-        return f.canReadTriples();
-    }
-
-    /**
-     * Tries to create a quad reader for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return Quad reader if one is available
-     * @throws IOException
-     *             Thrown if a quad reader is not available or the given
-     *             language does not support quads
-     */
-    public static RecordReader<LongWritable, QuadWritable> createQuadReader(Lang lang) throws IOException {
-        if (lang == null)
-            throw new IOException("Cannot create a quad reader for an undefined language");
-
-        ReaderFactory f = readerFactories.get(lang);
-        if (f == null)
-            throw new IOException("No factory registered for language " + lang.getName());
-        if (!f.canReadQuads())
-            throw new IOException(lang.getName() + " does not support reading quads");
-
-        RecordReader<LongWritable, QuadWritable> reader = f.createQuadReader();
-        if (reader == null)
-            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples reader");
-        return reader;
-    }
-
-    /**
-     * Tries to create a triple reader for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return Triple reader if one is available
-     * @throws IOException
-     *             Thrown if a triple reader is not available or the given
-     *             language does not support triple
-     */
-    public static RecordReader<LongWritable, TripleWritable> createTripleReader(Lang lang) throws IOException {
-        if (lang == null)
-            throw new IOException("Cannot create a triple reader for an undefined language");
-
-        ReaderFactory f = readerFactories.get(lang);
-        if (f == null)
-            throw new IOException("No factory registered for language " + lang.getName());
-        if (!f.canReadTriples())
-            throw new IOException(lang.getName() + " does not support reading triples");
-
-        RecordReader<LongWritable, TripleWritable> reader = f.createTripleReader();
-        if (reader == null)
-            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples reader");
-        return reader;
-    }
-
-    /**
-     * Gets whether there is a quad writer available for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return True if available, false otherwise
-     */
-    public static boolean hasQuadWriter(Lang lang) {
-        if (lang == null)
-            return false;
-
-        WriterFactory f = writerFactories.get(lang);
-        if (f == null)
-            return false;
-        return f.canWriteQuads();
-    }
-
-    /**
-     * Gets whether there is a triple writer available for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return True if available, false otherwise
-     */
-    public static boolean hasTriplesWriter(Lang lang) {
-        if (lang == null)
-            return false;
-
-        WriterFactory f = writerFactories.get(lang);
-        if (f == null)
-            return false;
-        return f.canWriteTriples();
-    }
-
-    /**
-     * Tries to create a quad writer for the given language
-     * 
-     * @param lang
-     *            Language
-     * @param writer
-     *            Writer
-     * @param config
-     *            Configuration
-     * 
-     * @return Quad writer if one is available
-     * @throws IOException
-     *             Thrown if a quad writer is not available or the given
-     *             language does not support quads
-     */
-    public static <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Lang lang, Writer writer,
-            Configuration config) throws IOException {
-        if (lang == null)
-            throw new IOException("Cannot create a quad writer for an undefined language");
-
-        WriterFactory f = writerFactories.get(lang);
-        if (f == null)
-            throw new IOException("No factory registered for language " + lang.getName());
-        if (!f.canWriteQuads())
-            throw new IOException(lang.getName() + " does not support writeing quads");
-
-        RecordWriter<TKey, QuadWritable> rwriter = f.<TKey> createQuadWriter(writer, config);
-        if (rwriter == null)
-            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples writer");
-        return rwriter;
-    }
-
-    /**
-     * Tries to create a triple writer for the given language
-     * 
-     * @param lang
-     *            Language
-     * @param writer
-     *            Writer
-     * @param config
-     *            Configuration
-     * @return Triple writer if one is available
-     * @throws IOException
-     *             Thrown if a triple writer is not available or the given
-     *             language does not support triple
-     */
-    public static <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Lang lang, Writer writer,
-            Configuration config) throws IOException {
-        if (lang == null)
-            throw new IOException("Cannot create a triple writer for an undefined language");
-
-        WriterFactory f = writerFactories.get(lang);
-        if (f == null)
-            throw new IOException("No factory registered for language " + lang.getName());
-        if (!f.canWriteTriples())
-            throw new IOException(lang.getName() + " does not support writing triples");
-
-        RecordWriter<TKey, TripleWritable> rwriter = f.<TKey> createTripleWriter(writer, config);
-        if (rwriter == null)
-            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples writer");
-        return rwriter;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
deleted file mode 100644
index e1c98c7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.registry;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Interface for reader factories
- * 
- */
-public interface ReaderFactory {
-
-    /**
-     * Gets the primary language this factory produces readers for
-     * 
-     * @return Primary language
-     */
-    public abstract Lang getPrimaryLanguage();
-
-    /**
-     * Gets the alternative languages this factory can produce readers for
-     * 
-     * @return Alternative languages
-     */
-    public abstract Collection<Lang> getAlternativeLanguages();
-
-    /**
-     * Gets whether this factory can produce readers that are capable of reading
-     * quads
-     * 
-     * @return True if quads can be read, false if not
-     */
-    public abstract boolean canReadQuads();
-
-    /**
-     * Gets whether this factory can produce readers that are capable of reading
-     * triples
-     * 
-     * @return True if triples can be read, false if not
-     */
-    public abstract boolean canReadTriples();
-
-    /**
-     * Creates a quad reader
-     * 
-     * @return Quad reader
-     * @throws IOException
-     *             May be thrown if a quad reader cannot be created
-     */
-    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
-
-    /**
-     * Creates a triples reader
-     * 
-     * @return Triples reader
-     * @throws IOException
-     *             May be thrown if a triple reader cannot be created
-     */
-    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java
deleted file mode 100644
index db5635f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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.registry;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Interface for writer factories
- * 
- */
-public interface WriterFactory {
-
-    /**
-     * Gets the primary language this factory produces writers for
-     * 
-     * @return Primary language
-     */
-    public abstract Lang getPrimaryLanguage();
-
-    /**
-     * Gets the alternative languages this factory can produce writers for
-     * 
-     * @return Alternative languages
-     */
-    public abstract Collection<Lang> getAlternativeLanguages();
-
-    /**
-     * Gets whether this factory can produce writers that are capable of reading
-     * quads
-     * 
-     * @return True if quads can be read, false if not
-     */
-    public abstract boolean canWriteQuads();
-
-    /**
-     * Gets whether this factory can produce writers that are capable of reading
-     * triples
-     * 
-     * @return True if triples can be read, false if not
-     */
-    public abstract boolean canWriteTriples();
-
-    /**
-     * Creates a quad writer
-     * 
-     * @param writer
-     *            Writer
-     * @param config
-     *            Configuration
-     * 
-     * @return Quad writer
-     * @throws IOException
-     *             May be thrown if a quad writer cannot be created
-     */
-    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException;
-
-    /**
-     * Creates a triples writer
-     * 
-     * @param writer
-     *            Writer
-     * @param config
-     *            Configuration
-     * 
-     * @return Triples writer
-     * @throws IOException
-     *             May be thrown if a triple writer cannot be created
-     */
-    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
deleted file mode 100644
index 7fe15a9..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract reader factory for languages that only support quads
- */
-public abstract class AbstractQuadsOnlyReaderFactory implements ReaderFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
-
-    public AbstractQuadsOnlyReaderFactory(Lang lang) {
-        this(lang, (Collection<Lang>)null);
-    }
-    
-    public AbstractQuadsOnlyReaderFactory(Lang lang, Lang...altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractQuadsOnlyReaderFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-    
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canReadQuads() {
-        return true;
-    }
-
-    @Override
-    public final boolean canReadTriples() {
-        return false;
-    }
-
-    @Override
-    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
-
-    @Override
-    public final RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        throw new IOException(this.lang.getName() + " does not support reading triples");
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java
deleted file mode 100644
index 60e45af..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract reader factory for languages that support triples and quads
- */
-public abstract class AbstractReaderFactory implements ReaderFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
-
-    public AbstractReaderFactory(Lang lang) {
-        this(lang, (Collection<Lang>)null);
-    }
-    
-    public AbstractReaderFactory(Lang lang, Lang...altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractReaderFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-    
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canReadQuads() {
-        return true;
-    }
-
-    @Override
-    public final boolean canReadTriples() {
-        return true;
-    }
-
-    @Override
-    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
-
-    @Override
-    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
deleted file mode 100644
index 7fb8131..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract reader factory for languages that only support triples
- */
-public abstract class AbstractTriplesOnlyReaderFactory implements ReaderFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
-
-    public AbstractTriplesOnlyReaderFactory(Lang lang) {
-        this(lang, (Collection<Lang>)null);
-    }
-    
-    public AbstractTriplesOnlyReaderFactory(Lang lang, Lang...altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractTriplesOnlyReaderFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-    
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-    
-    @Override
-    public final boolean canReadQuads() {
-        return false;
-    }
-
-    @Override
-    public final boolean canReadTriples() {
-        return true;
-    }
-
-    @Override
-    public final RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        throw new IOException(this.lang.getName() + " does not support reading quads");
-    }
-
-    @Override
-    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java
deleted file mode 100644
index 6b064a4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.JsonLDQuadReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.JsonLDTripleReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class JsonLDReaderFactory extends AbstractReaderFactory {
-    
-    public JsonLDReaderFactory() {
-        super(Lang.JSONLD);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new JsonLDQuadReader();
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new JsonLDTripleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
deleted file mode 100644
index 2296296..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.nquads.WholeFileNQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class NQuadsReaderFactory extends AbstractQuadsOnlyReaderFactory {
-    
-    public NQuadsReaderFactory() {
-        super(Lang.NQUADS, Lang.NQ);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new WholeFileNQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
deleted file mode 100644
index a98a1ae..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.WholeFileNTriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-public class NTriplesReaderFactory extends AbstractTriplesOnlyReaderFactory {
-
-    public NTriplesReaderFactory() {
-        super(Lang.NTRIPLES, Lang.NT);
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new WholeFileNTriplesReader();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
deleted file mode 100644
index ccf5feb..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.rdfjson.RdfJsonReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class RdfJsonReaderFactory extends AbstractTriplesOnlyReaderFactory {
-
-    public RdfJsonReaderFactory() {
-        super(Lang.RDFJSON);
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new RdfJsonReader();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
deleted file mode 100644
index 1aa88d7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.rdfxml.RdfXmlReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-public class RdfXmlReaderFactory extends AbstractTriplesOnlyReaderFactory {
-
-    public RdfXmlReaderFactory() {
-        super(Lang.RDFXML);
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new RdfXmlReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java
deleted file mode 100644
index 25e8234..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftQuadReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftTripleReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- *
- */
-public class ThriftReaderFactory extends AbstractReaderFactory {
-    
-    public ThriftReaderFactory() {
-        super(RDFLanguages.THRIFT);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new ThriftQuadReader();
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new ThriftTripleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
deleted file mode 100644
index 83ea818..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.trig.TriGReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class TriGReaderFactory extends AbstractQuadsOnlyReaderFactory {
-
-    public TriGReaderFactory() {
-        super(Lang.TRIG);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new TriGReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
deleted file mode 100644
index cb8795c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.trix.TriXReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class TriXReaderFactory extends AbstractQuadsOnlyReaderFactory {
-
-    public TriXReaderFactory() {
-        super(Lang.TRIX);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new TriXReader();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
deleted file mode 100644
index 7800376..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.turtle.TurtleReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-public class TurtleReaderFactory extends AbstractTriplesOnlyReaderFactory {
-    
-    public TurtleReaderFactory() {
-        super(Lang.TURTLE, Lang.TTL, Lang.N3);
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new TurtleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
deleted file mode 100644
index 0cf137e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract writer factory for languages that only support quads
- */
-public abstract class AbstractQuadsOnlyWriterFactory implements WriterFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang> emptyList());
-
-    public AbstractQuadsOnlyWriterFactory(Lang lang) {
-        this(lang, (Collection<Lang>) null);
-    }
-
-    public AbstractQuadsOnlyWriterFactory(Lang lang, Lang... altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractQuadsOnlyWriterFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canWriteQuads() {
-        return true;
-    }
-
-    @Override
-    public final boolean canWriteTriples() {
-        return false;
-    }
-
-    @Override
-    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException;
-
-    @Override
-    public final <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        throw new IOException(this.lang.getName() + " does not support writing triples");
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
deleted file mode 100644
index e45c3da..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract writer factory for languages that only support triples
- */
-public abstract class AbstractTriplesOnlyWriterFactory implements WriterFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang> emptyList());
-
-    public AbstractTriplesOnlyWriterFactory(Lang lang) {
-        this(lang, (Collection<Lang>) null);
-    }
-
-    public AbstractTriplesOnlyWriterFactory(Lang lang, Lang... altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractTriplesOnlyWriterFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canWriteQuads() {
-        return false;
-    }
-
-    @Override
-    public final boolean canWriteTriples() {
-        return true;
-    }
-
-    @Override
-    public final <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        throw new IOException(this.lang.getName() + " does not support writing quads");
-    }
-
-    @Override
-    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
deleted file mode 100644
index 669b9c4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract writer factory for languages that support triples and quads
- */
-public abstract class AbstractWriterFactory implements WriterFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
-
-    public AbstractWriterFactory(Lang lang) {
-        this(lang, (Collection<Lang>)null);
-    }
-    
-    public AbstractWriterFactory(Lang lang, Lang...altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractWriterFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-    
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canWriteQuads() {
-        return true;
-    }
-
-    @Override
-    public final boolean canWriteTriples() {
-        return true;
-    }
-
-    @Override
-    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config) throws IOException;
-
-    @Override
-    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java
deleted file mode 100644
index 89e93ed..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.jsonld.JsonLDQuadWriter;
-import org.apache.jena.hadoop.rdf.io.output.writers.jsonld.JsonLDTripleWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class JsonLDWriterFactory extends AbstractWriterFactory {
-    
-    public JsonLDWriterFactory() {
-        super(Lang.JSONLD);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new JsonLDQuadWriter<>(writer);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new JsonLDTripleWriter<>(writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java
deleted file mode 100644
index abbbd0f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.nquads.NQuadsWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class NQuadsWriterFactory extends AbstractQuadsOnlyWriterFactory {
-    
-    public NQuadsWriterFactory() {
-        super(Lang.NQUADS, Lang.NQ);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new NQuadsWriter<TKey>(writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java
deleted file mode 100644
index 88c9551..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.ntriples.NTriplesWriter;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class NTriplesWriterFactory extends AbstractTriplesOnlyWriterFactory {
-    
-    public NTriplesWriterFactory() {
-        super(Lang.NTRIPLES, Lang.NT);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new NTriplesWriter<TKey>(writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
deleted file mode 100644
index 8252422..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.rdfjson.RdfJsonWriter;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class RdfJsonWriterFactory extends AbstractTriplesOnlyWriterFactory {
-
-    public RdfJsonWriterFactory() {
-        super(Lang.RDFJSON);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new RdfJsonWriter<TKey>(writer);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java
deleted file mode 100644
index b4ac8e3..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.rdfxml.RdfXmlWriter;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class RdfXmlWriterFactory extends AbstractTriplesOnlyWriterFactory {
-    
-    public RdfXmlWriterFactory() {
-        super(Lang.RDFXML);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new RdfXmlWriter<TKey>(writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java
deleted file mode 100644
index 757472c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.writers.StreamRdfQuadWriter;
-import org.apache.jena.hadoop.rdf.io.output.writers.StreamRdfTripleWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.RDFLanguages;
-import org.apache.jena.riot.thrift.StreamRDF2Thrift;
-
-/**
- *
- */
-public class ThriftWriterFactory extends AbstractWriterFactory {
-
-    public ThriftWriterFactory() {
-        super(RDFLanguages.THRIFT);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfQuadWriter<TKey>(new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")),
-                false), writer);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfTripleWriter<TKey>(new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")),
-                false), writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java
deleted file mode 100644
index 6d8b08a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.StreamRdfQuadWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
-
-/**
- *
- */
-public class TriGWriterFactory extends AbstractQuadsOnlyWriterFactory {
-    
-    public TriGWriterFactory() {
-        super(Lang.TRIG);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfQuadWriter<TKey>(new WriterStreamRDFBlocks(writer), writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
deleted file mode 100644
index 0e1b7b2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.writers.StreamRdfQuadWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.writer.StreamWriterTriX;
-
-/**
- *
- */
-public class TriXWriterFactory extends AbstractQuadsOnlyWriterFactory {
-    
-    public TriXWriterFactory() {
-        super(Lang.TRIX);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfQuadWriter<>(new StreamWriterTriX(new WriterOutputStream(writer, Charset.forName("utf-8"))), writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java
deleted file mode 100644
index c837f12..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.StreamRdfTripleWriter;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
-
-/**
- *
- */
-public class TurtleWriterFactory extends AbstractTriplesOnlyWriterFactory {
-    
-    public TurtleWriterFactory() {
-        super(Lang.TURTLE, Lang.TTL, Lang.N3);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfTripleWriter<>(new WriterStreamRDFBlocks(writer), writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory b/jena-hadoop-rdf/hadoop-rdf-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
deleted file mode 100644
index ec0e48a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
+++ /dev/null
@@ -1,10 +0,0 @@
-# Default Reader Factory implementations
-org.apache.jena.hadoop.rdf.io.registry.readers.JsonLDReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.NQuadsReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.NTriplesReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.RdfJsonReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.RdfXmlReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.ThriftReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.TriGReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.TriXReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.TurtleReaderFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory b/jena-hadoop-rdf/hadoop-rdf-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
deleted file mode 100644
index 164880d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
+++ /dev/null
@@ -1,10 +0,0 @@
-# Default Writer Factory implementations
-org.apache.jena.hadoop.rdf.io.registry.writers.JsonLDWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.NQuadsWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.NTriplesWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.RdfJsonWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.RdfXmlWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.ThriftWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.TriGWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.TriXWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.TurtleWriterFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
deleted file mode 100644
index 5762fb7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.log4j.Logger;
-
-
-/**
- * A test mapper which takes in line based RDF triple input and just produces triples
- * 
- *
- */
-public class RdfTriplesInputTestMapper extends Mapper<LongWritable, TripleWritable, NullWritable, TripleWritable> {
-    
-    private static final Logger LOG = Logger.getLogger(RdfTriplesInputTestMapper.class);
-
-    @Override
-    protected void map(LongWritable key, TripleWritable value, Context context)
-            throws IOException, InterruptedException {
-        LOG.info("Line " + key.toString() + " => " + value.toString());
-        context.write(NullWritable.get(), value);
-    }
-
-    
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
deleted file mode 100644
index 1cda0bd..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/AbstractBlockedQuadInputFormatTests.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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;
-
-/**
- * Abstract tests for blocked triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractBlockedQuadInputFormatTests extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected boolean canSplitInputs() {
-        return true;
-    }
-}


[14/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
deleted file mode 100644
index b7acc08..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped NTriples input
- * 
- * 
- * 
- */
-public class GZippedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedWholeFileNTriplesInputTest() {
-        super(".nt.gz", new GzipCodec());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
deleted file mode 100644
index 3b2546d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.rdfjson;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed RDF/JSON input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedRdfJsonInputFormatTests extends AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedRdfJsonInputFormatTests(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.RDFJSON;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new RdfJsonInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
deleted file mode 100644
index 6e58d4b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfjson;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped RDF/JSON input
- * 
- * 
- * 
- */
-public class BZippedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedRdfJsonInputTest() {
-        super(".rj.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
deleted file mode 100644
index 8b7b044..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfjson;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated RDF/JSON input
- * 
- * 
- * 
- */
-public class DeflatedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedRdfJsonInputTest() {
-        super(".rj.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
deleted file mode 100644
index 66996dd..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfjson;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped RDF/JSON input
- * 
- * 
- * 
- */
-public class GZippedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedRdfJsonInputTest() {
-        super(".rj.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
deleted file mode 100644
index a6d1e24..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.rdfxml;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.rdfxml.RdfXmlInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed RDF/XML input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedRdfXmlInputFormatTests extends
- AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedRdfXmlInputFormatTests(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.RDFXML;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new RdfXmlInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
deleted file mode 100644
index 30a6c39..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfxml;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped RDF/XML input
- * 
- * 
- * 
- */
-public class BZippedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedRdfXmlInputTest() {
-        super(".rdf.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
deleted file mode 100644
index a3d747c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfxml;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated RDF/XML input
- * 
- * 
- * 
- */
-public class DeflatedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedRdfXmlInputTest() {
-        super(".rdf.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
deleted file mode 100644
index 748785c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfxml;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped RDF/XML input
- * 
- * 
- * 
- */
-public class GZippedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedRdfXmlInputTest() {
-        super(".rdf.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
deleted file mode 100644
index 78affb2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.thrift;
-
-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftQuadInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Abstract compressed Thrift quad input tests
- */
-public abstract class AbstractCompressedThriftQuadInputFormatTests extends
-        AbstractCompressedWholeFileQuadInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedThriftQuadInputFormatTests(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 RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new ThriftQuadInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
deleted file mode 100644
index f837b89..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.thrift;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftTripleInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Abstract compressed Thrift triple input tests
- */
-public abstract class AbstractCompressedThriftTripleInputFormatTests extends
-        AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedThriftTripleInputFormatTests(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 RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new ThriftTripleInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
deleted file mode 100644
index 320d278..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped Thrift input
- */
-public class BZippedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedThriftQuadInputTest() {
-        super(".trdf.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
deleted file mode 100644
index bb2d65b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped Thrift input
- */
-public class BZippedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedThriftTripleInputTest() {
-        super(".trdf.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
deleted file mode 100644
index 6872583..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated Thrift input
- */
-public class DeflatedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedThriftQuadInputTest() {
-        super(".trdf.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
deleted file mode 100644
index e76d2d7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated Thrift input
- */
-public class DeflatedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedThriftTripleInputTest() {
-        super(".trdf.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
deleted file mode 100644
index 6590f22..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped Thrift input
- */
-public class GZippedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedThriftQuadInputTest() {
-        super(".trdf.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
deleted file mode 100644
index 1ce74f4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped Thrift input
- */
-public class GZippedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedThriftTripleInputTest() {
-        super(".trdf.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
deleted file mode 100644
index 2975f29..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.trig;
-
-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.trig.TriGInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed TriG input tests
- */
-public abstract class AbstractCompressedTriGInputFormatTests extends
-        AbstractCompressedWholeFileQuadInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedTriGInputFormatTests(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.TRIG;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new TriGInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
deleted file mode 100644
index b2b3c33..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.trig;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped TriG input
- * 
- * 
- * 
- */
-public class BZippedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedTriGInputTest() {
-        super(".trig.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
deleted file mode 100644
index c9579a9..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.trig;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated TriG input
- * 
- * 
- * 
- */
-public class DeflatedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedTriGInputTest() {
-        super(".trig.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
deleted file mode 100644
index c3e4106..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.trig;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped TriG input
- * 
- * 
- * 
- */
-public class GZippedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedTriGInputTest() {
-        super(".trig.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
deleted file mode 100644
index ad98e35..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.trix;
-
-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.trix.TriXInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed TriX input tests
- */
-public abstract class AbstractCompressedTriXInputFormatTests extends
-        AbstractCompressedWholeFileQuadInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedTriXInputFormatTests(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.TRIX;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new TriXInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/BZippedTriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/BZippedTriXInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/BZippedTriXInputTest.java
deleted file mode 100644
index fc51ec8..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/BZippedTriXInputTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.trix;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-import org.apache.jena.hadoop.rdf.io.input.compressed.trig.AbstractCompressedTriGInputFormatTests;
-
-/**
- * Tests for BZipped TriX input
- */
-public class BZippedTriXInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedTriXInputTest() {
-        super(".trix.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/DeflatedTriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/DeflatedTriXInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/DeflatedTriXInputTest.java
deleted file mode 100644
index a1a078d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/DeflatedTriXInputTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.trix;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.jena.hadoop.rdf.io.input.compressed.trig.AbstractCompressedTriGInputFormatTests;
-
-/**
- * Tests for Deflated TriX input
- */
-public class DeflatedTriXInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedTriXInputTest() {
-        super(".trix.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/GZippedTriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/GZippedTriXInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/GZippedTriXInputTest.java
deleted file mode 100644
index 10c6980..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/GZippedTriXInputTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.trix;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.jena.hadoop.rdf.io.input.compressed.trig.AbstractCompressedTriGInputFormatTests;
-
-/**
- * Tests for GZipped TriX input
- */
-public class GZippedTriXInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedTriXInputTest() {
-        super(".trix.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
deleted file mode 100644
index 68d776a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.turtle;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.turtle.TurtleInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed Turtle input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedTurtleInputFormatTests extends
-        AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedTurtleInputFormatTests(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.TURTLE;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new TurtleInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
deleted file mode 100644
index 724b847..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.turtle;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped NTriples input
- * 
- * 
- * 
- */
-public class BZippedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedTurtleInputTest() {
-        super(".nt.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
deleted file mode 100644
index eb5ee03..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.turtle;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated NTriples input
- * 
- * 
- * 
- */
-public class DeflatedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedTurtleInputTest() {
-        super(".nt.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
deleted file mode 100644
index 817805c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.turtle;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped NTriples input
- * 
- * 
- * 
- */
-public class GZippedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedTurtleInputTest() {
-        super(".nt.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
deleted file mode 100644
index 92aac53..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.jsonld;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
deleted file mode 100644
index 63b6738..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.jsonld;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
deleted file mode 100644
index 6d1d02a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractBlockedQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for blocked NTriples input
- * 
- * 
- * 
- */
-public class BlockedNQuadsInputTest extends AbstractBlockedQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nq";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new BlockedNQuadsInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
deleted file mode 100644
index 3823728..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractQuadsInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Tests for the NQuads input format
- * 
- *
- */
-public class NQuadsInputTest extends AbstractQuadsInputFormatTests {
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new NQuadsInputFormat();
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nq";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
deleted file mode 100644
index 50b8bcf..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for NQuads input
- * 
- * 
- * 
- */
-public class WholeFileNQuadsInputTest extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nq";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new WholeFileNQuadsInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
deleted file mode 100644
index ab92873..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractBlockedTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for blocked NTriples input
- * 
- *
- */
-public class BlockedNTriplesInputTest extends AbstractBlockedTripleInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nt";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new BlockedNTriplesInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
deleted file mode 100644
index 24fb731..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractTriplesInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Tests for the {@link NTriplesInputFormat}
- * 
- * 
- * 
- */
-public class NTriplesInputTest extends AbstractTriplesInputFormatTests {
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new NTriplesInputFormat();
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nt";
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
deleted file mode 100644
index fcec570..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for the {@link NTriplesInputFormat}
- * 
- * 
- * 
- */
-public class WholeFileNTriplesInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new WholeFileNTriplesInputFormat();
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nt";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-    
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java b/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
deleted file mode 100644
index 4731832..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.rdfjson;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for RDF/JSON input
- * 
- * 
- * 
- */
-public class RdfJsonInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFJSON;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".rj";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new RdfJsonInputFormat();
-    }
-
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
deleted file mode 100644
index da70007..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/RdfXmlBlankNodeTest.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.rdfxml.RdfXmlInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.rdfxml.RdfXmlOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link RdfXmlInputFormat}
- */
-public class RdfXmlBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.RDFXML;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".rdf";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new RdfXmlInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new RdfXmlOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new RdfXmlInputFormat();
-    }
-
-    @Override
-    protected boolean respectsParserProfile() {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
deleted file mode 100644
index 146c836..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/bnodes/TurtleBlankNodeTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.bnodes;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.jena.hadoop.rdf.io.input.turtle.TurtleInputFormat;
-import org.apache.jena.hadoop.rdf.io.output.turtle.TurtleOutputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Tests blank node divergence when using the {@link TurtleInputFormat}
- */
-public class TurtleBlankNodeTest extends AbstractTripleBlankNodeTests {
-
-    @Override
-    protected Lang getLanguage() {
-        return Lang.TURTLE;
-    }
-
-    @Override
-    protected String getInitialInputExtension() {
-        return ".ttl";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createInitialInputFormat() {
-        return new TurtleInputFormat();
-    }
-
-    @Override
-    protected OutputFormat<LongWritable, TripleWritable> createIntermediateOutputFormat() {
-        return new TurtleOutputFormat<>();
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> createIntermediateInputFormat() {
-        return new TurtleInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
deleted file mode 100644
index 1f18a95..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedNodeTupleInputFormatTests.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-
-
-/**
- * 
- * 
- * @param <TValue>
- * @param <T>
- */
-public abstract class AbstractCompressedNodeTupleInputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        AbstractNodeTupleInputFormatTests<TValue, T> {
-    
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
-        return config;
-    }
-
-    @Override
-    protected OutputStream getOutputStream(File f) throws IOException {
-        CompressionCodec codec = this.getCompressionCodec();
-        if (codec instanceof Configurable) {
-            ((Configurable) codec).setConf(this.prepareConfiguration());
-        }
-        FileOutputStream fileOutput = new FileOutputStream(f, false);
-        return codec.createOutputStream(fileOutput);
-    }
-
-    /**
-     * Gets the compression codec to use
-     * 
-     * @return Compression codec
-     */
-    protected abstract CompressionCodec getCompressionCodec();
-
-    /**
-     * Indicates whether inputs can be split, defaults to false for compressed
-     * input tests
-     */
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
deleted file mode 100644
index 312aae7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedQuadsInputFormatTests.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for Quad input formats
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedQuadsInputFormatTests extends
-        AbstractCompressedNodeTupleInputFormatTests<Quad, QuadWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected void generateTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateBadTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write("<http://broken\n".getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
-        boolean bad = false;
-        for (int i = 0; i < num; i++, bad = !bad) {
-            if (bad) {
-                output.write("<http://broken\n".getBytes(utf8));
-            } else {
-                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" <http://graphs/" + i + "> .\n").getBytes(utf8));
-            }
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
deleted file mode 100644
index f0f0caf..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedTriplesInputFormatTests.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Abstract tests for Triple input formats
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedTriplesInputFormatTests extends
-        AbstractCompressedNodeTupleInputFormatTests<Triple, TripleWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected void generateTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateBadTuples(OutputStream output, int num) throws IOException {
-        for (int i = 0; i < num; i++) {
-            output.write("<http://broken\n".getBytes(utf8));
-        }
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected void generateMixedTuples(OutputStream output, int num) throws IOException {
-        boolean bad = false;
-        for (int i = 0; i < num; i++, bad = !bad) {
-            if (bad) {
-                output.write("<http://broken\n".getBytes(utf8));
-            } else {
-                output.write(("<http://subjects/" + i + "> <http://predicate> \"" + i + "\" .\n").getBytes(utf8));
-            }
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
deleted file mode 100644
index be2b1d7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileQuadInputFormatTests.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * 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;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
-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.query.Dataset;
-import com.hp.hpl.jena.query.DatasetFactory;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ModelFactory;
-import com.hp.hpl.jena.rdf.model.Property;
-import com.hp.hpl.jena.rdf.model.Resource;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Abstract tests for compressed whole file quad formats
- * 
- * 
- */
-public abstract class AbstractCompressedWholeFileQuadInputFormatTests extends
-        AbstractNodeTupleInputFormatTests<Quad, QuadWritable> {
-    
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
-        return config;
-    }
-
-    @Override
-    protected OutputStream getOutputStream(File f) throws IOException {
-        CompressionCodec codec = this.getCompressionCodec();
-        if (codec instanceof Configurable) {
-            ((Configurable) codec).setConf(this.prepareConfiguration());
-        }
-        FileOutputStream fileOutput = new FileOutputStream(f, false);
-        return codec.createOutputStream(fileOutput);
-    }
-
-    /**
-     * Gets the compression codec to use
-     * 
-     * @return Compression codec
-     */
-    protected abstract CompressionCodec getCompressionCodec();
-
-    /**
-     * Indicates whether inputs can be split, defaults to false for compressed
-     * input tests
-     */
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-
-    private void writeTuples(Dataset ds, OutputStream output) {
-        RDFDataMgr.write(output, ds, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
-    }
-
-    /**
-     * Gets the RDF language to write out generated tuples in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-    private void writeGoodTuples(OutputStream output, int num) throws IOException {
-        Dataset ds = DatasetFactory.createMem();
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num; i++) {
-            if (i % 100 == 0) {
-                ds.addNamedModel("http://example.org/graphs/" + (i / 100), m);
-                m = ModelFactory.createDefaultModel();
-            }
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        if (!m.isEmpty()) {
-            ds.addNamedModel("http://example.org/graphs/extra", m);
-        }
-        this.writeTuples(ds, output);
-    }
-
-    @Override
-    protected final void generateTuples(OutputStream output, int num) throws IOException {
-        this.writeGoodTuples(output, num);
-        output.close();
-    }
-
-    @Override
-    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
-        // Write good data
-        this.writeGoodTuples(output, num / 2);
-
-        // Write junk data
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num / 2; i++) {
-            output.write(junk);
-        }
-
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
deleted file mode 100644
index 56dd8ca..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/AbstractCompressedWholeFileTripleInputFormatTests.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.jena.hadoop.rdf.io.HadoopIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.AbstractNodeTupleInputFormatTests;
-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.Triple;
-import com.hp.hpl.jena.rdf.model.Model;
-import com.hp.hpl.jena.rdf.model.ModelFactory;
-import com.hp.hpl.jena.rdf.model.Property;
-import com.hp.hpl.jena.rdf.model.Resource;
-
-/**
- * Abstract tests for compressed whole file triple formats
- * 
- * 
- */
-public abstract class AbstractCompressedWholeFileTripleInputFormatTests extends
-        AbstractNodeTupleInputFormatTests<Triple, TripleWritable> {
-
-    private static final Charset utf8 = Charset.forName("utf-8");
-
-    @Override
-    protected Configuration prepareConfiguration() {
-        Configuration config = super.prepareConfiguration();
-        config.set(HadoopIOConstants.IO_COMPRESSION_CODECS, this.getCompressionCodec().getClass().getCanonicalName());
-        return config;
-    }
-
-    @Override
-    protected OutputStream getOutputStream(File f) throws IOException {
-        CompressionCodec codec = this.getCompressionCodec();
-        if (codec instanceof Configurable) {
-            ((Configurable) codec).setConf(this.prepareConfiguration());
-        }
-        FileOutputStream fileOutput = new FileOutputStream(f, false);
-        return codec.createOutputStream(fileOutput);
-    }
-
-    /**
-     * Gets the compression codec to use
-     * 
-     * @return Compression codec
-     */
-    protected abstract CompressionCodec getCompressionCodec();
-
-    /**
-     * Indicates whether inputs can be split, defaults to false for compressed
-     * input tests
-     */
-    @Override
-    protected boolean canSplitInputs() {
-        return false;
-    }
-
-    private void writeTuples(Model m, OutputStream output) {
-        RDFDataMgr.write(output, m, this.getRdfLanguage());
-    }
-
-    /**
-     * Gets the RDF language to write out generated tuples in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-    @Override
-    protected final void generateTuples(OutputStream output, int num) throws IOException {
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num; i++) {
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        this.writeTuples(m, output);
-        output.close();
-    }
-
-    @Override
-    protected final void generateMixedTuples(OutputStream output, int num) throws IOException {
-        // Write good data
-        Model m = ModelFactory.createDefaultModel();
-        Resource currSubj = m.createResource("http://example.org/subjects/0");
-        Property predicate = m.createProperty("http://example.org/predicate");
-        for (int i = 0; i < num / 2; i++) {
-            if (i % 10 == 0) {
-                currSubj = m.createResource("http://example.org/subjects/" + (i / 10));
-            }
-            m.add(currSubj, predicate, m.createTypedLiteral(i));
-        }
-        this.writeTuples(m, output);
-
-        // Write junk data
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num / 2; i++) {
-            output.write(junk);
-        }
-
-        output.flush();
-        output.close();
-    }
-
-    @Override
-    protected final void generateBadTuples(OutputStream output, int num) throws IOException {
-        byte[] junk = "junk data\n".getBytes(utf8);
-        for (int i = 0; i < num; i++) {
-            output.write(junk);
-        }
-        output.flush();
-        output.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
deleted file mode 100644
index d118f29..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDQuadInputFormatTests.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDQuadInputFormat;
-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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
deleted file mode 100644
index acb9e08..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/AbstractCompressedJsonLDTripleInputFormatTests.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.jsonld.JsonLDTripleInputFormat;
-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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java
deleted file mode 100644
index e5e7066..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java
deleted file mode 100644
index 8d2e122..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/BZippedJsonLDTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java
deleted file mode 100644
index 292b17f..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java
deleted file mode 100644
index e5edd6a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/DeflatedJsonLDTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java
deleted file mode 100644
index 0a4a240..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java
deleted file mode 100644
index 8a6324d..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/jsonld/GZippedJsonLDTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
deleted file mode 100644
index 2b8447b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedNQuadsInputFormatTests.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.nquads;
-
-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.compressed.AbstractCompressedQuadsInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.nquads.NQuadsInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Abstract compressed NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedNQuadsInputFormatTests extends AbstractCompressedQuadsInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedNQuadsInputFormatTests(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 InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new NQuadsInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
deleted file mode 100644
index 326258a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/AbstractCompressedWholeFileNQuadsInputFormatTests.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.nquads;
-
-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.nquads.WholeFileNQuadsInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed whole file NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedWholeFileNQuadsInputFormatTests extends
-        AbstractCompressedWholeFileQuadInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedWholeFileNQuadsInputFormatTests(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.NQUADS;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new WholeFileNQuadsInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
deleted file mode 100644
index 62dc9ce..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedNQuadsInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-
-/**
- * Tests for BZipped NQuads input
- * 
- * 
- * 
- */
-public class BZipppedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZipppedNQuadsInputTest() {
-        super(".nq.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
deleted file mode 100644
index e3d670c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/BZipppedWholeFileNQuadsInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped NQuads input
- * 
- * 
- * 
- */
-public class BZipppedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZipppedWholeFileNQuadsInputTest() {
-        super(".nq.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
deleted file mode 100644
index 5fec23e..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedNQuadsInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for deflated NQuads input
- * 
- * 
- * 
- */
-public class DeflatedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedNQuadsInputTest() {
-        super(".nq.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
deleted file mode 100644
index a31472d..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/DeflatedWholeFileNQuadsInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for deflated NQuads input
- * 
- * 
- * 
- */
-public class DeflatedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedWholeFileNQuadsInputTest() {
-        super(".nq.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
deleted file mode 100644
index 2d40dec..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedNQuadsInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped NQuads input
- * 
- * 
- * 
- */
-public class GZippedNQuadsInputTest extends AbstractCompressedNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedNQuadsInputTest() {
-        super(".nq.gz", new GzipCodec());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
deleted file mode 100644
index 0f73eb7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/nquads/GZippedWholeFileNQuadsInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped NQuads input
- * 
- * 
- * 
- */
-public class GZippedWholeFileNQuadsInputTest extends AbstractCompressedWholeFileNQuadsInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedWholeFileNQuadsInputTest() {
-        super(".nq.gz", new GzipCodec());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
deleted file mode 100644
index 611d862..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedBlockedNTriplesInputFormatTests.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.ntriples;
-
-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.ntriples.BlockedNTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Abstract compressed blocked NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedBlockedNTriplesInputFormatTests extends
-        AbstractCompressedWholeFileNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedBlockedNTriplesInputFormatTests(String ext, CompressionCodec codec) {
-        super(ext, codec);
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new BlockedNTriplesInputFormat();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
deleted file mode 100644
index 4d9aeba..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedNTriplesInputFormatTests.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.ntriples;
-
-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.compressed.AbstractCompressedTriplesInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.ntriples.NTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Abstract compressed NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedNTriplesInputFormatTests extends AbstractCompressedTriplesInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedNTriplesInputFormatTests(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 InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new NTriplesInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
deleted file mode 100644
index 7d4b510..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/AbstractCompressedWholeFileNTriplesInputFormatTests.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.ntriples;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.ntriples.WholeFileNTriplesInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed whole file NTriples input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedWholeFileNTriplesInputFormatTests extends
-        AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedWholeFileNTriplesInputFormatTests(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.NTRIPLES;
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new WholeFileNTriplesInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
deleted file mode 100644
index de45f17..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedBlockedNTriplesInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped blocked NTriples input
- * 
- * 
- * 
- */
-public class BZippedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedBlockedNTriplesInput() {
-        super(".nt.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
deleted file mode 100644
index fb1ab8b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-
-/**
- * Tests for BZipped NTriples input
- * 
- * 
- * 
- */
-public class BZippedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedNTriplesInputTest() {
-        super(".nt.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
deleted file mode 100644
index d50b6eb..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/BZippedWholeFileNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-
-/**
- * Tests for BZipped NTriples input
- * 
- * 
- * 
- */
-public class BZippedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedWholeFileNTriplesInputTest() {
-        super(".nt.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
deleted file mode 100644
index 9780707..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedBlockedNTriplesInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for deflated blocked NTriples input
- * 
- * 
- * 
- */
-public class DeflatedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedBlockedNTriplesInput() {
-        super(".nt.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
deleted file mode 100644
index b6dd1e9..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-
-/**
- * Tests for deflated NTriples input
- * 
- * 
- * 
- */
-public class DeflatedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedNTriplesInputTest() {
-        super(".nt.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
deleted file mode 100644
index e1e57f4..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/DeflatedWholeFileNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-
-/**
- * Tests for deflated NTriples input
- * 
- * 
- * 
- */
-public class DeflatedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedWholeFileNTriplesInputTest() {
-        super(".nt.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
deleted file mode 100644
index 012fd58..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedBlockedNTriplesInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped blocked NTriples input
- * 
- * 
- * 
- */
-public class GZippedBlockedNTriplesInput extends AbstractCompressedBlockedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedBlockedNTriplesInput() {
-        super(".nt.gz", new GzipCodec());
-    }
-}


[18/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java
deleted file mode 100644
index c67b3da..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trix/TriXOutputFormat.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java
deleted file mode 100644
index c7564ac..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputFormat.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java
deleted file mode 100644
index c0202d8..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleOutputFormat.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java
deleted file mode 100644
index c4eafd8..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedNodeTupleWriter.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java
deleted file mode 100644
index 13c2799..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedQuadWriter.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java
deleted file mode 100644
index 881cf15..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractBatchedTripleWriter.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java
deleted file mode 100644
index 89b8f4b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedNodeTupleWriter.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java
deleted file mode 100644
index 9ecef61..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedQuadWriter.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java
deleted file mode 100644
index 161c067..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractLineBasedTripleWriter.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
deleted file mode 100644
index 9d8eeb2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractNodeWriter.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java
deleted file mode 100644
index aa178b2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractStreamRdfNodeTupleWriter.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java
deleted file mode 100644
index d48546b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileNodeTupleWriter.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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);
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java
deleted file mode 100644
index 5fc0024..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileQuadWriter.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.Lang;
-import org.apache.jena.riot.RDFDataMgr;
-import org.apache.jena.riot.RDFWriterRegistry;
-
-import com.hp.hpl.jena.sparql.core.DatasetGraph;
-import com.hp.hpl.jena.sparql.core.DatasetGraphFactory;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract record writer for whole file triple formats
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public abstract class AbstractWholeFileQuadWriter<TKey> extends AbstractWholeFileNodeTupleWriter<TKey, Quad, QuadWritable> {
-
-    private DatasetGraph g = DatasetGraphFactory.createMem();
-
-    protected AbstractWholeFileQuadWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected final void add(QuadWritable value) {
-        this.g.add(value.get());
-    }
-
-    @SuppressWarnings("deprecation")
-    @Override
-    protected void writeOutput(Writer writer) {
-        RDFDataMgr.write(writer, this.g, RDFWriterRegistry.defaultSerialization(this.getRdfLanguage()));
-    }
-
-    /**
-     * Gets the RDF language to write the output in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
deleted file mode 100644
index bb26093..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/AbstractWholeFileTripleWriter.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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;
-
-/**
- * An abstract record writer for whole file triple formats
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public abstract class AbstractWholeFileTripleWriter<TKey> extends AbstractWholeFileNodeTupleWriter<TKey, Triple, TripleWritable> {
-
-    private Graph g = GraphFactory.createDefaultGraph();
-
-    protected AbstractWholeFileTripleWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected final void add(TripleWritable value) {
-        this.g.add(value.get());
-    }
-
-    @SuppressWarnings("deprecation")
-    @Override
-    protected final void writeOutput(Writer writer) {
-        RDFDataMgr.write(writer, this.g, this.getRdfLanguage());
-    }
-
-    /**
-     * Gets the RDF language to write the output in
-     * 
-     * @return RDF language
-     */
-    protected abstract Lang getRdfLanguage();
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
deleted file mode 100644
index e932e1f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/QuadsToTriplesWriter.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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 org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-/**
- * A record writer that converts quads into triples by stripping off the graph
- * field
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadsToTriplesWriter<TKey> extends RecordWriter<TKey, QuadWritable> {
-
-    private RecordWriter<TKey, TripleWritable> writer;
-
-    /**
-     * Creates a new writer
-     * 
-     * @param tripleWriter
-     *            Triple writer to use
-     */
-    public QuadsToTriplesWriter(RecordWriter<TKey, TripleWritable> tripleWriter) {
-        if (tripleWriter == null)
-            throw new NullPointerException("tripleWriter cannot be null");
-        this.writer = tripleWriter;
-    }
-
-    @Override
-    public void write(TKey key, QuadWritable value) throws IOException, InterruptedException {
-        this.writer.write(key, new TripleWritable(value.get().asTriple()));
-    }
-
-    @Override
-    public void close(TaskAttemptContext context) throws IOException, InterruptedException {
-        this.writer.close(context);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java
deleted file mode 100644
index e9156d2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfQuadWriter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.system.StreamRDF;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A writer for {@link StreamRDF} based quad writers
- * 
- * @param <TKey>
- *            Key type
- */
-public class StreamRdfQuadWriter<TKey> extends
-		AbstractStreamRdfNodeTupleWriter<TKey, Quad, QuadWritable> {
-
-	public StreamRdfQuadWriter(StreamRDF stream, Writer writer) {
-		super(stream, writer);
-	}
-
-	@Override
-	protected void sendOutput(TKey key, QuadWritable value, StreamRDF stream) {
-		stream.quad(value.get());
-	}
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java
deleted file mode 100644
index 174edf3..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/StreamRdfTripleWriter.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.system.StreamRDF;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A writer for {@link StreamRDF} based triple writers
- * 
- * @param <TKey>
- *            Key type
- */
-public class StreamRdfTripleWriter<TKey> extends AbstractStreamRdfNodeTupleWriter<TKey, Triple, TripleWritable> {
-
-	public StreamRdfTripleWriter(StreamRDF stream, Writer writer) {
-		super(stream, writer);
-	}
-
-	@Override
-	protected void sendOutput(TKey key, TripleWritable value, StreamRDF stream) {
-		stream.triple(value.get());
-	}
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
deleted file mode 100644
index 1b4b62f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDQuadWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.jsonld;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileQuadWriter;
-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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
deleted file mode 100644
index 8d2079d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/jsonld/JsonLDTripleWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.jsonld;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
-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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
deleted file mode 100644
index 8df606d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/nquads/NQuadsWriter.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.nquads;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractLineBasedQuadWriter;
-import org.apache.jena.riot.out.CharSpace;
-import org.apache.jena.riot.out.NodeFormatterNT;
-
-/**
- * A record writer for NQuads
- * 
- * 
- * 
- * @param <TKey>
- */
-public class NQuadsWriter<TKey> extends AbstractLineBasedQuadWriter<TKey> {
-
-    /**
-     * Creates a new writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public NQuadsWriter(Writer writer) {
-        super(writer, new NodeFormatterNT());
-    }
-
-    /**
-     * Creates a new writer using the given character space
-     * 
-     * @param writer
-     *            Writer
-     * @param charSpace
-     *            Character space
-     */
-    public NQuadsWriter(Writer writer, CharSpace charSpace) {
-        super(writer, new NodeFormatterNT(charSpace));
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
deleted file mode 100644
index 6413f21..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesNodeWriter.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.ntriples;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractNodeWriter;
-import org.apache.jena.riot.out.CharSpace;
-import org.apache.jena.riot.out.NodeFormatterNT;
-
-/**
- * A NTriples based node writer
- * 
- * 
- * 
- * @param <TValue>
- *            Value type
- */
-public class NTriplesNodeWriter<TValue> extends AbstractNodeWriter<TValue> {
-
-    /**
-     * Creates a new writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public NTriplesNodeWriter(Writer writer) {
-        super(writer);
-    }
-
-    /**
-     * Creates a new writer
-     * 
-     * @param writer
-     *            Writer
-     * @param charSpace
-     *            Character space to use
-     */
-    public NTriplesNodeWriter(Writer writer, CharSpace charSpace) {
-        super(writer, new NodeFormatterNT(charSpace));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java
deleted file mode 100644
index 2b3c0b4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/ntriples/NTriplesWriter.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.ntriples;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractLineBasedTripleWriter;
-import org.apache.jena.riot.out.CharSpace;
-import org.apache.jena.riot.out.NodeFormatterNT;
-
-/**
- * A record writer for NTriples
- * 
- * 
- * @param <TKey>
- *            Key type
- * 
- */
-public class NTriplesWriter<TKey> extends AbstractLineBasedTripleWriter<TKey> {
-
-    /**
-     * Creates a new writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public NTriplesWriter(Writer writer) {
-        super(writer, new NodeFormatterNT());
-    }
-
-    /**
-     * Creates a new writer using the given character space
-     * 
-     * @param writer
-     *            Writer
-     * @param charSpace
-     *            Character space
-     */
-    public NTriplesWriter(Writer writer, CharSpace charSpace) {
-        super(writer, new NodeFormatterNT(charSpace));
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java
deleted file mode 100644
index 30bd4fa..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfjson/RdfJsonWriter.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.rdfjson;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record writer for RDF/JSON
- * 
- * 
- * @param <TKey>
- *            Key type
- * 
- */
-public class RdfJsonWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
-
-    /**
-     * Creates a new record writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public RdfJsonWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFJSON;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java
deleted file mode 100644
index f202914..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/rdfxml/RdfXmlWriter.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.rdfxml;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record writer for RDF/XML
- * 
- * 
- * @param <TKey>
- *            Key type
- * 
- */
-public class RdfXmlWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
-
-    /**
-     * Creates a new record writer
-     * 
-     * @param writer
-     *            Writer
-     */
-    public RdfXmlWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.RDFXML;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
deleted file mode 100644
index 599dba9..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftQuadWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.thrift;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileQuadWriter;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-public class ThriftQuadWriter<TKey> extends AbstractWholeFileQuadWriter<TKey> {
-
-    public ThriftQuadWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
deleted file mode 100644
index 0089459..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/thrift/ThriftTripleWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.thrift;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractWholeFileTripleWriter;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-public class ThriftTripleWriter<TKey> extends AbstractWholeFileTripleWriter<TKey> {
-
-    public ThriftTripleWriter(Writer writer) {
-        super(writer);
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return RDFLanguages.THRIFT;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java
deleted file mode 100644
index 4c0c75f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/trig/BatchedTriGWriter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.trig;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedQuadWriter;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record writer for TriG that uses the batched approach, note that this
- * approach will produce invalid data when blank nodes span batches
- *  
- * @param <TKey>
- *            Key type
- */
-public class BatchedTriGWriter<TKey> extends AbstractBatchedQuadWriter<TKey> {
-
-	/**
-	 * Creates a new record writer
-	 * 
-	 * @param writer
-	 *            Writer
-	 * @param batchSize
-	 *            Batch size
-	 */
-	public BatchedTriGWriter(Writer writer, long batchSize) {
-		super(writer, batchSize);
-	}
-
-	@Override
-	protected Lang getRdfLanguage() {
-		return Lang.TRIG;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java
deleted file mode 100644
index 86ab2f9..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/writers/turtle/BatchedTurtleWriter.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.turtle;
-
-import java.io.Writer;
-
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedTripleWriter;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record writer for Turtle that uses the batched approach, note that this
- * approach will produce invalid data when blank nodes span batches
- * 
- * 
- * 
- * @param <TKey>
- */
-public class BatchedTurtleWriter<TKey> extends
-		AbstractBatchedTripleWriter<TKey> {
-
-	/**
-	 * Creates a new record writer
-	 * 
-	 * @param writer
-	 *            Writer
-	 * @param batchSize
-	 *            Batch size
-	 */
-	public BatchedTurtleWriter(Writer writer, long batchSize) {
-		super(writer, batchSize);
-	}
-
-	@Override
-	protected Lang getRdfLanguage() {
-		return Lang.TURTLE;
-	}
-
-}


[21/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java b/jena-hadoop-rdf/hadoop-rdf-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
deleted file mode 100644
index 7214b14..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/CharacteristicTests.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * 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.types;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
-import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
-import org.junit.Assert;
-import org.junit.Test;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * Tests for {@link CharacteristicWritable} and
- * {@link CharacteristicSetWritable}
- * 
- * 
- * 
- */
-public class CharacteristicTests {
-
-    /**
-     * Checks whether a writable round trips successfully
-     * 
-     * @param cw
-     *            Characteristic writable
-     * @throws IOException
-     */
-    private void checkRoundTrip(CharacteristicWritable cw) throws IOException {
-        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        DataOutputStream output = new DataOutputStream(outputStream);
-        cw.write(output);
-
-        ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray());
-        DataInputStream input = new DataInputStream(inputStream);
-        CharacteristicWritable actual = CharacteristicWritable.read(input);
-        Assert.assertEquals(cw, actual);
-    }
-
-    /**
-     * Tests characteristic round tripping
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_writable_01() throws IOException {
-        Node n = NodeFactory.createURI("http://example.org");
-        CharacteristicWritable expected = new CharacteristicWritable(n);
-        Assert.assertEquals(1, expected.getCount().get());
-
-        this.checkRoundTrip(expected);
-    }
-
-    /**
-     * Tests characteristic properties
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_writable_02() throws IOException {
-        Node n = NodeFactory.createURI("http://example.org");
-        CharacteristicWritable cw1 = new CharacteristicWritable(n);
-        CharacteristicWritable cw2 = new CharacteristicWritable(n, 100);
-        this.checkRoundTrip(cw1);
-        this.checkRoundTrip(cw2);
-
-        // Should still be equal since equality is only on the node not the
-        // count
-        Assert.assertEquals(cw1, cw2);
-    }
-
-    /**
-     * Tests characteristic properties
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_writable_03() throws IOException {
-        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
-        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
-        this.checkRoundTrip(cw1);
-        this.checkRoundTrip(cw2);
-
-        // Should not be equal as different nodes
-        Assert.assertNotEquals(cw1, cw2);
-    }
-
-    /**
-     * Checks that a writable round trips
-     * 
-     * @param set
-     *            Characteristic set
-     * @throws IOException
-     */
-    private void checkRoundTrip(CharacteristicSetWritable set) throws IOException {
-        // Test round trip
-        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-        DataOutputStream output = new DataOutputStream(outputStream);
-        set.write(output);
-
-        ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray());
-        DataInputStream input = new DataInputStream(inputStream);
-        CharacteristicSetWritable actual = CharacteristicSetWritable.read(input);
-        Assert.assertEquals(set, actual);
-    }
-
-    /**
-     * Checks a characteristic set
-     * 
-     * @param set
-     *            Set
-     * @param expectedItems
-     *            Expected number of characteristics
-     * @param expectedCounts
-     *            Expected counts for characteristics
-     */
-    protected final void checkCharacteristicSet(CharacteristicSetWritable set, int expectedItems, long[] expectedCounts) {
-        Assert.assertEquals(expectedItems, set.size());
-        Assert.assertEquals(expectedItems, expectedCounts.length);
-        Iterator<CharacteristicWritable> iter = set.getCharacteristics();
-        int i = 0;
-        while (iter.hasNext()) {
-            CharacteristicWritable cw = iter.next();
-            Assert.assertEquals(expectedCounts[i], cw.getCount().get());
-            i++;
-        }
-    }
-
-    /**
-     * Tests characteristic sets
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_writable_01() throws IOException {
-        CharacteristicSetWritable set = new CharacteristicSetWritable();
-
-        // Add some characteristics
-        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
-        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
-        set.add(cw1);
-        set.add(cw2);
-        this.checkCharacteristicSet(set, 2, new long[] { 1, 1 });
-        this.checkRoundTrip(set);
-    }
-
-    /**
-     * Tests characteristic sets
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_writable_02() throws IOException {
-        CharacteristicSetWritable set = new CharacteristicSetWritable();
-
-        // Add some characteristics
-        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
-        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"), 2);
-        set.add(cw1);
-        set.add(cw2);
-        this.checkCharacteristicSet(set, 1, new long[] { 3 });
-        this.checkRoundTrip(set);
-    }
-    
-    /**
-     * Tests characteristic sets
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_writable_03() throws IOException {
-        CharacteristicSetWritable set1 = new CharacteristicSetWritable();
-        CharacteristicSetWritable set2 = new CharacteristicSetWritable();
-
-        // Add some characteristics
-        CharacteristicWritable cw1 = new CharacteristicWritable(NodeFactory.createURI("http://example.org"));
-        CharacteristicWritable cw2 = new CharacteristicWritable(NodeFactory.createURI("http://example.org/other"));
-        set1.add(cw1);
-        set2.add(cw2);
-        this.checkCharacteristicSet(set1, 1, new long[] { 1 });
-        this.checkCharacteristicSet(set2, 1, new long[] { 1 });
-        this.checkRoundTrip(set1);
-        this.checkRoundTrip(set2);
-        
-        Assert.assertNotEquals(set1, set2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java b/jena-hadoop-rdf/hadoop-rdf-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
deleted file mode 100644
index a70dfb0..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-common/src/test/java/org/apache/jena/hadoop/rdf/io/types/RdfTypesTest.java
+++ /dev/null
@@ -1,406 +0,0 @@
-/*
- * 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.types;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.hadoop.io.WritableComparable;
-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.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Tests for the various RDF types defined by the
- * {@link org.apache.jena.hadoop.rdf.types} package
- * 
- * 
- * 
- */
-public class RdfTypesTest {
-
-    private static final Logger LOG = LoggerFactory.getLogger(RdfTypesTest.class);
-
-    private ByteArrayOutputStream outputStream;
-    private ByteArrayInputStream inputStream;
-
-    /**
-     * Prepare for output
-     * 
-     * @return Data output
-     */
-    private DataOutput prepareOutput() {
-        this.outputStream = new ByteArrayOutputStream();
-        return new DataOutputStream(this.outputStream);
-    }
-
-    /**
-     * Prepare for input from the previously written output
-     * 
-     * @return Data Input
-     */
-    private DataInput prepareInput() {
-        this.inputStream = new ByteArrayInputStream(this.outputStream.toByteArray());
-        return new DataInputStream(this.inputStream);
-    }
-
-    /**
-     * Prepare for input from the given data
-     * 
-     * @param data
-     *            Data
-     * @return Data Input
-     */
-    @SuppressWarnings("unused")
-    private DataInput prepareInput(byte[] data) {
-        this.inputStream = new ByteArrayInputStream(data);
-        return new DataInputStream(this.inputStream);
-    }
-
-    @SuppressWarnings({ "unchecked", "rawtypes" })
-    private <T extends WritableComparable> void testWriteRead(T writable, T expected) throws IOException, InstantiationException, IllegalAccessException,
-            ClassNotFoundException {
-        // Write out data
-        DataOutput output = this.prepareOutput();
-        writable.write(output);
-
-        // Read back in data
-        DataInput input = this.prepareInput();
-        T actual = (T) Class.forName(writable.getClass().getName()).newInstance();
-        actual.readFields(input);
-
-        LOG.info("Original = " + writable.toString());
-        LOG.info("Round Tripped = " + actual.toString());
-
-        // Check equivalent
-        Assert.assertEquals(0, expected.compareTo(actual));
-    }
-    
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_null() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = null;
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-    
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    @Ignore
-    public void node_writable_variable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createVariable("x");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-    
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    @Ignore
-    public void node_writable_variable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createVariable("really-log-variable-name-asddsfr4545egfdgdfgfdgdtgvdg-dfgfdgdfgdfgdfg4-dfvdfgdfgdfgfdgfdgdfgdfgfdg");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_uri_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createURI("http://example.org");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_uri_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createURI("http://user:password@example.org/some/path?key=value#id");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("simple");
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("language", "en", null);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_03() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("string", XSDDatatype.XSDstring);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_04() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("1234", XSDDatatype.XSDinteger);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_05() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("123.4", XSDDatatype.XSDdecimal);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_06() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("12.3e4", XSDDatatype.XSDdouble);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_literal_07() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createLiteral("true", XSDDatatype.XSDboolean);
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_bnode_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createAnon();
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-    }
-
-    /**
-     * Basic node writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void node_writable_bnode_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Node n = NodeFactory.createAnon();
-        NodeWritable nw = new NodeWritable(n);
-        testWriteRead(nw, nw);
-        NodeWritable nw2 = new NodeWritable(n);
-        testWriteRead(nw2, nw2);
-
-        Assert.assertEquals(0, nw.compareTo(nw2));
-    }
-
-    /**
-     * Basic triple writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void triple_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Triple t = new Triple(NodeFactory.createURI("http://example"), NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral("value"));
-        TripleWritable tw = new TripleWritable(t);
-        testWriteRead(tw, tw);
-    }
-    
-    /**
-     * Basic triple writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void triple_writable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Triple t = new Triple(NodeFactory.createAnon(), NodeFactory.createURI("http://predicate"), NodeFactory.createLiteral("value"));
-        TripleWritable tw = new TripleWritable(t);
-        testWriteRead(tw, tw);
-    }
-
-    /**
-     * Basic quad writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void quad_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Quad q = new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createURI("http://example"), NodeFactory.createURI("http://predicate"),
-                NodeFactory.createLiteral("value"));
-        QuadWritable qw = new QuadWritable(q);
-        testWriteRead(qw, qw);
-    }
-    
-    /**
-     * Basic quad writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void quad_writable_02() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Quad q = new Quad(Quad.defaultGraphNodeGenerated, NodeFactory.createAnon(), NodeFactory.createURI("http://predicate"),
-                NodeFactory.createLiteral("value"));
-        QuadWritable qw = new QuadWritable(q);
-        testWriteRead(qw, qw);
-    }
-
-    /**
-     * Basic tuple writable round tripping test
-     * 
-     * @throws IOException
-     * @throws InstantiationException
-     * @throws IllegalAccessException
-     * @throws ClassNotFoundException
-     */
-    @Test
-    public void tuple_writable_01() throws IOException, InstantiationException, IllegalAccessException, ClassNotFoundException {
-        Tuple<Node> t = Tuple.createTuple(NodeFactory.createURI("http://one"), NodeFactory.createURI("http://two"), NodeFactory.createLiteral("value"),
-                NodeFactory.createLiteral("foo"), NodeFactory.createURI("http://three"));
-        NodeTupleWritable tw = new NodeTupleWritable(t);
-        testWriteRead(tw, tw);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/pom.xml b/jena-hadoop-rdf/hadoop-rdf-io/pom.xml
deleted file mode 100644
index f7dbed1..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/pom.xml
+++ /dev/null
@@ -1,100 +0,0 @@
-<!--
-   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.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<modelVersion>4.0.0</modelVersion>
-	<parent>
-		<groupId>org.apache.jena</groupId>
-		<artifactId>jena-hadoop-rdf</artifactId>
-		<version>0.9.0-SNAPSHOT</version>
-	</parent>
-	<artifactId>jena-hadoop-rdf-io</artifactId>
-	<name>Apache Jena - RDF Tools for Hadoop - I/O</name>
-	<description>RDF Input/Output formats library for Hadoop</description>
-
-	<!-- Note that versions are managed by parent POMs -->
-	<dependencies>
-		<!-- Internal Project Dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-hadoop-rdf-common</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<!-- Hadoop Dependencies -->
-		<!-- Note these will be provided on the Hadoop cluster hence the provided 
-			scope -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-<!-- 			<exclusions>
-				hadoop-common apparently has all sorts of dependency convergence 
-					issues as of 2.4.0
-				Exclude this since it transitively has an outdated jackson-core-asl 
-					dependency
-				<exclusion>
-					<groupId>com.sun.jersey</groupId>
-					<artifactId>jersey-json</artifactId>
-				</exclusion>
-				Exclude these since they have outdated commons-logging dependencies
-				<exclusion>
-					<groupId>commons-httpclient</groupId>
-					<artifactId>commons-httpclient</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>commons-el</groupId>
-					<artifactId>commons-el</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>net.java.dev.jets3t</groupId>
-					<artifactId>jets3t</artifactId>
-				</exclusion>
-				Exclude these since it has an outdated dependency on servlet-api
-				<exclusion>
-					<groupId>tomcat</groupId>
-					<artifactId>jasper-runtime</artifactId>
-				</exclusion>
-				Exclude these since it has an outdated dependency on commons-codec
-				<exclusion>
-					<groupId>commons-codec</groupId>
-					<artifactId>commons-codec</artifactId>
-				</exclusion>
-			</exclusions> -->
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-mapreduce-client-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Jena dependencies -->
-		<dependency>
-			<groupId>org.apache.jena</groupId>
-			<artifactId>jena-arq</artifactId>
-		</dependency>
-
-		<!-- Test Dependencies -->
-		<dependency>
-			<groupId>junit</groupId>
-			<artifactId>junit</artifactId>
-			<scope>test</scope>
-		</dependency>
-	</dependencies>
-</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java
deleted file mode 100644
index 5c1b41c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/HadoopIOConstants.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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;
-
-/**
- * Hadoop IO related constants
- * 
- * 
- * 
- */
-public class HadoopIOConstants {
-
-    /**
-     * Private constructor prevents instantiation
-     */
-    private HadoopIOConstants() {
-    }
-
-    /**
-     * Map Reduce configuration setting for max line length
-     */
-    public static final String MAX_LINE_LENGTH = "mapreduce.input.linerecordreader.line.maxlength";
-
-    /**
-     * Run ID
-     */
-    public static final String RUN_ID = "runId";
-    
-    /**
-     * Compression codecs to use
-     */
-    public static final String IO_COMPRESSION_CODECS = "io.compression.codecs";
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
deleted file mode 100644
index 27c2bb2..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/RdfIOConstants.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-
-/**
- * RDF IO related constants
- * 
- * 
- * 
- */
-public class RdfIOConstants {
-
-    /**
-     * Private constructor prevents instantiation
-     */
-    private RdfIOConstants() {
-    }
-
-    /**
-     * Configuration key used to set whether bad tuples are ignored. This is the
-     * default behaviour, when explicitly set to {@code false} bad tuples will
-     * result in {@link IOException} being thrown by the relevant record
-     * readers.
-     */
-    public static final String INPUT_IGNORE_BAD_TUPLES = "rdf.io.input.ignore-bad-tuples";
-
-    /**
-     * Configuration key used to set the batch size used for RDF output formats
-     * that take a batched writing approach. Default value is given by the
-     * constant {@link #DEFAULT_OUTPUT_BATCH_SIZE}.
-     */
-    public static final String OUTPUT_BATCH_SIZE = "rdf.io.output.batch-size";
-
-    /**
-     * Default batch size for batched output formats
-     */
-    public static final long DEFAULT_OUTPUT_BATCH_SIZE = 10000;
-
-    /**
-     * Configuration key used to control behaviour with regards to how blank
-     * nodes are handled.
-     * <p>
-     * The default behaviour is that blank nodes are file scoped which is what
-     * the RDF specifications require.
-     * </p>
-     * <p>
-     * However in the case of a multi-stage pipeline this behaviour can cause
-     * blank nodes to diverge over several jobs and introduce spurious blank
-     * nodes over time. This is described in <a
-     * href="https://issues.apache.org/jira/browse/JENA-820">JENA-820</a> and
-     * enabling this flag for jobs in your pipeline allow you to work around
-     * this problem.
-     * </p>
-     * <h3>Warning</h3> You should only enable this flag for jobs that take in
-     * RDF output originating from previous jobs since our normal blank node
-     * allocation policy ensures that blank nodes will be file scoped and unique
-     * over all files (barring unfortunate hasing collisions). If you enable
-     * this for jobs that take in RDF originating from other sources you may
-     * incorrectly conflate blank nodes that are supposed to distinct and
-     * separate nodes.
-     */
-    public static final String GLOBAL_BNODE_IDENTITY = "rdf.io.input.bnodes.global-identity";
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
deleted file mode 100644
index 1fcb030..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractNLineFileInputFormat.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Abstract line based input format that reuses the machinery from
- * {@link NLineInputFormat} to calculate the splits
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Value type
- */
-public abstract class AbstractNLineFileInputFormat<TKey, TValue> extends FileInputFormat<TKey, TValue> {
-    
-    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractNLineFileInputFormat.class);
-
-    /**
-     * Logically splits the set of input files for the job, splits N lines of
-     * the input as one split.
-     * 
-     * @see FileInputFormat#getSplits(JobContext)
-     */
-    public final List<InputSplit> getSplits(JobContext job) throws IOException {
-        boolean debug = LOGGER.isDebugEnabled();
-        if (debug && FileInputFormat.getInputDirRecursive(job)) {
-            LOGGER.debug("Recursive searching for input data is enabled");
-        }
-        
-        List<InputSplit> splits = new ArrayList<InputSplit>();
-        int numLinesPerSplit = NLineInputFormat.getNumLinesPerSplit(job);
-        for (FileStatus status : listStatus(job)) {
-            if (debug) {
-                LOGGER.debug("Determining how to split input file/directory {}", status.getPath());
-            }
-            splits.addAll(NLineInputFormat.getSplitsForFile(status, job.getConfiguration(), numLinesPerSplit));
-        }
-        return splits;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
deleted file mode 100644
index e561cdb..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/AbstractWholeFileInputFormat.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.fs.Path;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-
-/**
- * Abstract implementation of a while file input format where each file is a
- * single split
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Value type
- */
-public abstract class AbstractWholeFileInputFormat<TKey, TValue> extends FileInputFormat<TKey, TValue> {
-
-    @Override
-    protected final boolean isSplitable(JobContext context, Path filename) {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
deleted file mode 100644
index b8fdbd5..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/QuadsInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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 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.QuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * RDF input format that can handle any RDF quads format that ARQ supports
- * selecting the format to use for each file based upon the file extension
- * 
- * 
- * 
- */
-public class QuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new QuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
deleted file mode 100644
index 03f394a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesInputFormat.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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 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.TriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-/**
- * RDF input format that can handle any RDF triples format that ARQ supports
- * selecting the format to use for each file based upon the file extension
- */
-public class TriplesInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TriplesReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
deleted file mode 100644
index bfd643e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/TriplesOrQuadsInputFormat.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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 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.TriplesOrQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * RDF input format that can handle any RDF triple/quads format that ARQ
- * supports selecting the format to use for each file based upon the file
- * extension. Triples are converted into quads in the default graph.
- * 
- * 
- * 
- */
-public class TriplesOrQuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TriplesOrQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
deleted file mode 100644
index 2464946..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputFormat.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.jsonld;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
deleted file mode 100644
index 0e08a4b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputFormat.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.jsonld;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.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/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
deleted file mode 100644
index 6829c4d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputFormat.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.nquads;
-
-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.AbstractNLineFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.nquads.BlockedNQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * NTriples input format where files are processed as blocks of lines rather
- * than in a line based manner as with the {@link NQuadsInputFormat} or as
- * whole files with the {@link WholeFileNQuadsInputFormat}
- * <p>
- * This provides a compromise between the higher parser setup of creating more
- * parsers and the benefit of being able to split input files over multiple
- * mappers.
- * </p>
- * 
- * 
- * 
- */
-public class BlockedNQuadsInputFormat extends AbstractNLineFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new BlockedNQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
deleted file mode 100644
index 802fbea..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.nquads;
-
-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.AbstractNLineFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.nquads.NQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * NQuads input format
- * 
- * 
- * 
- */
-public class NQuadsInputFormat extends AbstractNLineFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit arg0, TaskAttemptContext arg1)
-            throws IOException, InterruptedException {
-        return new NQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java
deleted file mode 100644
index 128d079..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputFormat.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.nquads;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.nquads.WholeFileNQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * NQuads input format where files are processed as complete files rather than
- * in a line based manner as with the {@link NQuadsInputFormat}
- * <p>
- * This has the advantage of less parser setup overhead but the disadvantage
- * that the input cannot be split over multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class WholeFileNQuadsInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new WholeFileNQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
deleted file mode 100644
index 292167b..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputFormat.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.ntriples;
-
-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.AbstractNLineFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.BlockedNTriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * NTriples input format where files are processed as blocks of lines rather
- * than in a line based manner as with the {@link NTriplesInputFormat} or as
- * whole files with the {@link WholeFileNTriplesInputFormat}
- * <p>
- * This provides a compromise between the higher parser setup of creating more
- * parsers and the benefit of being able to split input files over multiple
- * mappers.
- * </p>
- * 
- * 
- * 
- */
-public class BlockedNTriplesInputFormat extends AbstractNLineFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new BlockedNTriplesReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
deleted file mode 100644
index 1694c87..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.ntriples;
-
-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.AbstractNLineFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.NTriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * NTriples input format
- * 
- * 
- * 
- */
-public class NTriplesInputFormat extends AbstractNLineFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit inputSplit, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new NTriplesReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java
deleted file mode 100644
index 31c1252..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputFormat.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.ntriples;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.WholeFileNTriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * NTriples input format where files are processed as complete files rather than
- * in a line based manner as with the {@link NTriplesInputFormat}
- * <p>
- * This has the advantage of less parser setup overhead but the disadvantage
- * that the input cannot be split over multiple mappers.
- * </p>
- * 
- * 
- * 
- */
-public class WholeFileNTriplesInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new WholeFileNTriplesReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
deleted file mode 100644
index e5a7940..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.rdfjson;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.rdfjson.RdfJsonReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * RDF/JSON input format
- * 
- * 
- * 
- */
-public class RdfJsonInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new RdfJsonReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
deleted file mode 100644
index 4deb925..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.rdfxml;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.rdfxml.RdfXmlReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * RDF/XML input format
- * 
- * 
- * 
- */
-public class RdfXmlInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new RdfXmlReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
deleted file mode 100644
index 56d031e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
+++ /dev/null
@@ -1,344 +0,0 @@
-/*
- * 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.readers;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.input.util.BlockInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.RdfIOUtils;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
-import org.apache.jena.hadoop.rdf.io.input.util.TrackedPipedRDFStream;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFDataMgr;
-import org.apache.jena.riot.ReaderRIOT;
-import org.apache.jena.riot.lang.PipedRDFIterator;
-import org.apache.jena.riot.lang.PipedRDFStream;
-import org.apache.jena.riot.system.ParserProfile;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract implementation for a record reader that reads records from blocks
- * of files, this is a hybrid between {@link AbstractLineBasedNodeTupleReader}
- * and {@link AbstractWholeFileNodeTupleReader} in that it can only be used by
- * formats which can be split by lines but reduces the overhead by parsing the
- * split as a whole rather than as individual lines.
- * <p>
- * The keys produced are the approximate position in the file at which a tuple
- * was found and the values will be node tuples. Positions are approximate
- * because they are recorded after the point at which the most recent tuple was
- * parsed from the input thus they reflect the approximate position in the
- * stream immediately after which the triple was found.
- * </p>
- * 
- * 
- * 
- * @param <TValue>
- *            Value type
- * @param <T>
- *            Tuple type
- */
-public abstract class AbstractBlockBasedNodeTupleReader<TValue, T extends AbstractNodeTupleWritable<TValue>> extends RecordReader<LongWritable, T> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractBlockBasedNodeTupleReader.class);
-    private CompressionCodec compressionCodecs;
-    private TrackableInputStream input;
-    private LongWritable key;
-    private long start, length;
-    private T tuple;
-    private TrackedPipedRDFStream<TValue> stream;
-    private PipedRDFIterator<TValue> iter;
-    private Thread parserThread;
-    private boolean finished = false;
-    private boolean ignoreBadTuples = true;
-    private boolean parserFinished = false;
-    private Throwable parserError = null;
-
-    @Override
-    public void initialize(InputSplit genericSplit, TaskAttemptContext context) throws IOException, InterruptedException {
-        LOG.debug("initialize({}, {})", genericSplit, context);
-
-        // Assuming file split
-        if (!(genericSplit instanceof FileSplit))
-            throw new IOException("This record reader only supports FileSplit inputs");
-        FileSplit split = (FileSplit) genericSplit;
-
-        // Configuration
-        Configuration config = context.getConfiguration();
-        this.ignoreBadTuples = config.getBoolean(RdfIOConstants.INPUT_IGNORE_BAD_TUPLES, true);
-        if (this.ignoreBadTuples)
-            LOG.warn(
-                    "Configured to ignore bad tuples, parsing errors will be logged and further parsing aborted but no user visible errors will be thrown.  Consider setting {} to false to disable this behaviour",
-                    RdfIOConstants.INPUT_IGNORE_BAD_TUPLES);
-
-        // Figure out what portion of the file to read
-        start = split.getStart();
-        long end = start + split.getLength();
-        final Path file = split.getPath();
-        long totalLength = file.getFileSystem(context.getConfiguration()).getFileStatus(file).getLen();
-        boolean readToEnd = end == totalLength;
-        CompressionCodecFactory factory = new CompressionCodecFactory(config);
-        this.compressionCodecs = factory.getCodec(file);
-
-        LOG.info(String.format("Got split with start %d and length %d for file with total length of %d", new Object[] { start, split.getLength(), totalLength }));
-
-        // Open the file and prepare the input stream
-        FileSystem fs = file.getFileSystem(config);
-        FSDataInputStream fileIn = fs.open(file);
-        this.length = split.getLength();
-        if (start > 0)
-            fileIn.seek(start);
-
-        if (this.compressionCodecs != null) {
-            // Compressed input
-            // For compressed input NLineInputFormat will have failed to find
-            // any line breaks and will give us a split from 0 -> (length - 1)
-            // Add 1 and re-verify readToEnd so we can abort correctly if ever
-            // given a partial split of a compressed file
-            end++;
-            readToEnd = end == totalLength;
-            if (start > 0 || !readToEnd)
-                throw new IOException("This record reader can only be used with compressed input where the split is a whole file");
-            input = new TrackedInputStream(this.compressionCodecs.createInputStream(fileIn));
-        } else {
-            // Uncompressed input
-
-            if (readToEnd) {
-                input = new TrackedInputStream(fileIn);
-            } else {
-                // Need to limit the portion of the file we are reading
-                input = new BlockInputStream(fileIn, split.getLength());
-            }
-        }
-
-        // Set up background thread for parser
-        iter = this.getPipedIterator();
-        this.stream = this.getPipedStream(iter, this.input);
-        ParserProfile profile = RdfIOUtils.createParserProfile(context, file);
-        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage(), profile);
-        this.parserThread = new Thread(parserRunnable);
-        this.parserThread.setDaemon(true);
-        this.parserThread.start();
-    }
-
-    /**
-     * Gets the RDF iterator to use
-     * 
-     * @return Iterator
-     */
-    protected abstract PipedRDFIterator<TValue> getPipedIterator();
-
-    /**
-     * Gets the RDF stream to parse to
-     * 
-     * @param iterator
-     *            Iterator
-     * @return RDF stream
-     */
-    protected abstract TrackedPipedRDFStream<TValue> getPipedStream(PipedRDFIterator<TValue> iterator, TrackableInputStream input);
-
-    /**
-     * Gets the RDF language to use for parsing
-     * 
-     * @return
-     */
-    protected abstract Lang getRdfLanguage();
-
-    /**
-     * Creates the runnable upon which the parsing will run
-     * 
-     * @param input
-     *            Input
-     * @param stream
-     *            Stream
-     * @param lang
-     *            Language to use for parsing
-     * @return Parser runnable
-     */
-    private Runnable createRunnable(@SuppressWarnings("rawtypes") final AbstractBlockBasedNodeTupleReader reader, final InputStream input,
-            final PipedRDFStream<TValue> stream, final Lang lang, final ParserProfile profile) {
-        return new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    ReaderRIOT riotReader = RDFDataMgr.createReader(lang);
-                    riotReader.setParserProfile(profile);
-                    riotReader.read(input, null, lang.getContentType(), stream, null);
-                    //RDFDataMgr.parse(stream, input, null, lang);
-                    reader.setParserFinished(null);
-                } catch (Throwable e) {
-                    reader.setParserFinished(e);
-                }
-            }
-        };
-    }
-
-    /**
-     * Sets the parser thread finished state
-     * 
-     * @param e
-     *            Error (if any)
-     */
-    private void setParserFinished(Throwable e) {
-        synchronized (this.parserThread) {
-            this.parserError = e;
-            this.parserFinished = true;
-        }
-    }
-
-    /**
-     * Waits for the parser thread to have reported as finished
-     * 
-     * @throws InterruptedException
-     */
-    private void waitForParserFinished() throws InterruptedException {
-        do {
-            synchronized (this.parserThread) {
-                if (this.parserFinished)
-                    return;
-            }
-            Thread.sleep(50);
-        } while (true);
-    }
-
-    /**
-     * Creates an instance of a writable tuple from the given tuple value
-     * 
-     * @param tuple
-     *            Tuple value
-     * @return Writable tuple
-     */
-    protected abstract T createInstance(TValue tuple);
-
-    @Override
-    public boolean nextKeyValue() throws IOException, InterruptedException {
-        // Reuse key for efficiency
-        if (key == null) {
-            key = new LongWritable();
-        }
-
-        if (this.finished)
-            return false;
-
-        try {
-            if (this.iter.hasNext()) {
-                // Position will be relative to the start for the split we're
-                // processing
-                Long l = this.start + this.stream.getPosition();
-                if (l != null) {
-                    this.key.set(l);
-                    // For compressed input the actual length from which we
-                    // calculate progress is likely less than the actual
-                    // uncompressed length so we need to increment the
-                    // length as we go along
-                    // We always add 1 more than the current length because we
-                    // don't want to report 100% progress until we really have
-                    // finished
-                    if (this.compressionCodecs != null && l > this.length)
-                        this.length = l + 1;
-                }
-                this.tuple = this.createInstance(this.iter.next());
-                return true;
-            } else {
-                // Need to ensure that the parser thread has finished in order
-                // to determine whether we finished without error
-                this.waitForParserFinished();
-                if (this.parserError != null) {
-                    LOG.error("Error parsing block, aborting further parsing", this.parserError);
-                    if (!this.ignoreBadTuples)
-                        throw new IOException("Error parsing block at position " + (this.start + this.input.getBytesRead()) + ", aborting further parsing",
-                                this.parserError);
-                }
-
-                this.key = null;
-                this.tuple = null;
-                this.finished = true;
-                // This is necessary so that when compressed input is used we
-                // report 100% progress once we've reached the genuine end of
-                // the stream
-                if (this.compressionCodecs != null)
-                    this.length--;
-                return false;
-            }
-        } catch (IOException e) {
-            throw e;
-        } catch (Throwable e) {
-            // Failed to read the tuple on this line
-            LOG.error("Error parsing block, aborting further parsing", e);
-            if (!this.ignoreBadTuples) {
-                this.iter.close();
-                throw new IOException("Error parsing block at position " + (this.start + this.input.getBytesRead()) + ", aborting further parsing", e);
-            }
-            this.key = null;
-            this.tuple = null;
-            this.finished = true;
-            return false;
-        }
-    }
-
-    @Override
-    public LongWritable getCurrentKey() throws IOException, InterruptedException {
-        return this.key;
-    }
-
-    @Override
-    public T getCurrentValue() throws IOException, InterruptedException {
-        return this.tuple;
-    }
-
-    @Override
-    public float getProgress() throws IOException, InterruptedException {
-        float progress = 0.0f;
-        if (this.key == null) {
-            // We've either not started or we've finished
-            progress = (this.finished ? 1.0f : 0.0f);
-        } else if (this.key.get() == Long.MIN_VALUE) {
-            // We don't have a position so we've either in-progress or finished
-            progress = (this.finished ? 1.0f : 0.5f);
-        } else {
-            // We're some way through the file
-            progress = (this.key.get() - this.start) / (float) this.length;
-        }
-        LOG.debug("getProgress() --> {}", progress);
-        return progress;
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.iter.close();
-        this.input.close();
-        this.finished = true;
-    }
-
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java
deleted file mode 100644
index 2670cf4..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupByPredicateMapper.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A mapper which assists in grouping quads by predicate by reassigning their keys
- * to be their predicates
- * 
- * 
- * 
- * @param <TKey>
- */
-public class QuadGroupByPredicateMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
-
-    @Override
-    protected NodeWritable selectKey(Quad quad) {
-        return new NodeWritable(quad.getPredicate());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java
deleted file mode 100644
index 73809e8..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/QuadGroupBySubjectMapper.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A mapper which assists in grouping quads by subject by reassigning their keys
- * to be their subjects
- * 
- * 
- * 
- * @param <TKey>
- */
-public class QuadGroupBySubjectMapper<TKey> extends AbstractQuadGroupingMapper<TKey> {
-
-    @Override
-    protected NodeWritable selectKey(Quad quad) {
-        return new NodeWritable(quad.getSubject());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java
deleted file mode 100644
index 9fde939..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByObjectMapper.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A mapper which assists in grouping triples by object by reassigning their
- * keys to be their objects
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class TripleGroupByObjectMapper<TKey> extends AbstractTripleGroupingMapper<TKey> {
-
-    @Override
-    protected NodeWritable selectKey(Triple triple) {
-        return new NodeWritable(triple.getObject());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java
deleted file mode 100644
index dd15ef5..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupByPredicateMapper.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A mapper which assists in grouping triples by predicate by reassigning their
- * keys to be their predicates
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class TripleGroupByPredicateMapper<TKey> extends AbstractTripleGroupingMapper<TKey> {
-
-    @Override
-    protected NodeWritable selectKey(Triple triple) {
-        return new NodeWritable(triple.getPredicate());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java
deleted file mode 100644
index f1116c1..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/group/TripleGroupBySubjectMapper.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.group;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A mapper which assists in grouping triples by subject by reassigning their
- * keys to be their subjects
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class TripleGroupBySubjectMapper<TKey> extends AbstractTripleGroupingMapper<TKey> {
-
-    @Override
-    protected NodeWritable selectKey(Triple triple) {
-        return new NodeWritable(triple.getSubject());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java
deleted file mode 100644
index 840d78c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitToNodesMapper.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-
-/**
- * Abstract mapper implementation which splits the tuples into their constituent
- * nodes preserving the keys as-is
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleSplitToNodesMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        Mapper<TKey, T, TKey, NodeWritable> {
-
-    @Override
-    protected final void map(TKey key, T value, Context context) throws IOException, InterruptedException {
-        NodeWritable[] ns = this.split(value);
-        for (NodeWritable n : ns) {
-            context.write(key, n);
-        }
-    }
-
-    /**
-     * Splits the node tuple type into the individual nodes
-     * 
-     * @param tuple
-     *            Tuple
-     * @return Nodes
-     */
-    protected abstract NodeWritable[] split(T tuple);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java
deleted file mode 100644
index 7dc85fd..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/AbstractNodeTupleSplitWithNodesMapper.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-
-
-/**
- * Abstract mapper implementation which splits the tuples into their constituent
- * nodes using the tuples as the keys and the nodes as the values
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleSplitWithNodesMapper<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        Mapper<TKey, T, T, NodeWritable> {
-
-    @Override
-    protected void map(TKey key, T value, Context context) throws IOException, InterruptedException {
-        NodeWritable[] ns = this.split(value);
-        for (NodeWritable n : ns) {
-            context.write(value, n);
-        }
-    }
-
-    /**
-     * Splits the node tuple type into the individual nodes
-     * 
-     * @param tuple
-     *            Tuple
-     * @return Nodes
-     */
-    protected abstract NodeWritable[] split(T tuple);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java
deleted file mode 100644
index c993810..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitToNodesMapper.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A mapper which splits quads into their constituent nodes preserving the
- * existing keys as-is
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadSplitToNodesMapper<TKey> extends AbstractNodeTupleSplitToNodesMapper<TKey, Quad, QuadWritable> {
-
-    @Override
-    protected NodeWritable[] split(QuadWritable tuple) {
-        Quad q = tuple.get();
-        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
-                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java
deleted file mode 100644
index 09caef6..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/QuadSplitWithNodesMapper.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A mapper which splits quads into their constituent nodes using the quad as
- * the key and the nodes as the values
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadSplitWithNodesMapper<TKey> extends AbstractNodeTupleSplitWithNodesMapper<TKey, Quad, QuadWritable> {
-
-    @Override
-    protected NodeWritable[] split(QuadWritable tuple) {
-        Quad q = tuple.get();
-        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
-                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java
deleted file mode 100644
index 0ef02d9..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitToNodesMapper.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A mapper which splits triples into their constituent nodes
- * 
- * 
- * 
- * @param <TKey> Key type
- */
-public class TripleSplitToNodesMapper<TKey> extends AbstractNodeTupleSplitToNodesMapper<TKey, Triple, TripleWritable> {
-
-    @Override
-    protected NodeWritable[] split(TripleWritable tuple) {
-        Triple t = tuple.get();
-        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
-                new NodeWritable(t.getObject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java
deleted file mode 100644
index 7b18f55..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/split/TripleSplitWithNodesMapper.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.split;
-
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A mapper which splits triples into their constituent nodes
- * 
- * 
- * 
- * @param <TKey> Key type
- */
-public class TripleSplitWithNodesMapper<TKey> extends AbstractNodeTupleSplitWithNodesMapper<TKey, Triple, TripleWritable> {
-
-    @Override
-    protected NodeWritable[] split(TripleWritable tuple) {
-        Triple t = tuple.get();
-        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
-                new NodeWritable(t.getObject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java
deleted file mode 100644
index 76137fe..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/AbstractTriplesToQuadsMapper.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.transform;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * An abstract mapper which transforms triples into quads. Derived
- * implementations may choose how the graph to which triples are assigned is
- * decided.
- * <p>
- * Keys are left as is by this mapper.
- * </p>
- * 
- * 
- * 
- * @param <TKey>
- */
-public abstract class AbstractTriplesToQuadsMapper<TKey> extends Mapper<TKey, TripleWritable, TKey, QuadWritable> {
-
-    @Override
-    protected final void map(TKey key, TripleWritable value, Context context) throws IOException, InterruptedException {
-        Triple triple = value.get();
-        Node graphNode = this.selectGraph(triple);
-        context.write(key, new QuadWritable(new Quad(graphNode, triple)));
-    }
-
-    /**
-     * Selects the graph name to use for converting the given triple into a quad
-     * 
-     * @param triple
-     *            Triple
-     * @return Tuple
-     */
-    protected abstract Node selectGraph(Triple triple);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java
deleted file mode 100644
index 048e669..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/QuadsToTriplesMapper.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.transform;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * A mapper which transforms quads into triples
- * <p>
- * Keys are left as is by this mapper.
- * </p>
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class QuadsToTriplesMapper<TKey> extends Mapper<TKey, QuadWritable, TKey, TripleWritable> {
-
-    @Override
-    protected void map(TKey key, QuadWritable value, Context context) throws IOException, InterruptedException {
-        context.write(key, new TripleWritable(value.get().asTriple()));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java
deleted file mode 100644
index 394d5fd..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsBySubjectMapper.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.transform;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * A mapper which converts triples into quads using the subjects of the triples
- * as the graph nodes
- * 
- * 
- * @param <TKey>
- *            Key type
- * 
- */
-public class TriplesToQuadsBySubjectMapper<TKey> extends AbstractTriplesToQuadsMapper<TKey> {
-
-    @Override
-    protected final Node selectGraph(Triple triple) {
-        return triple.getSubject();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapper.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapper.java
deleted file mode 100644
index ef19edf..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/main/java/org/apache/jena/hadoop/rdf/mapreduce/transform/TriplesToQuadsConstantGraphMapper.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.transform;
-
-import java.io.IOException;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A mapper which converts triples to quads where all triples are placed in the
- * same graph
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- */
-public class TriplesToQuadsConstantGraphMapper<TKey> extends AbstractTriplesToQuadsMapper<TKey> {
-
-    private Node graphNode;
-
-    @Override
-    protected void setup(Context context) throws IOException, InterruptedException {
-        super.setup(context);
-        this.graphNode = this.getGraphNode();
-    }
-
-    /**
-     * Gets the graph node that will be used for all quads, this will be called
-     * once and only once during the
-     * {@link #setup(org.apache.hadoop.mapreduce.Mapper.Context)} method and the
-     * value returned cached for use throughout the lifetime of this mapper.
-     * <p>
-     * This implementation always used the default graph as the graph for
-     * generated quads. You can override this method in your own derived
-     * implementation to put triples into a different graph than the default
-     * graph.
-     * </p>
-     * <p>
-     * If instead you wanted to select different graphs for each triple you
-     * should extend {@link AbstractTriplesToQuadsMapper} instead and override
-     * the {@link #selectGraph(Triple)} method which is sealed in this
-     * implementation.
-     * </p>
-     * 
-     * @return
-     */
-    protected Node getGraphNode() {
-        return Quad.defaultGraphNodeGenerated;
-    }
-
-    @Override
-    protected final Node selectGraph(Triple triple) {
-        return this.graphNode;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapReduceTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapReduceTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapReduceTests.java
deleted file mode 100644
index 32c40f7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapReduceTests.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
-
-/**
- * Abstract tests for mappers
- * 
- * 
- * @param <TKey>
- *            Mapper input key type
- * @param <TValue>
- *            Mapper input value type
- * @param <TIntermediateKey>
- *            Mapper output/Reducer input key type
- * @param <TIntermediateValue>
- *            Mapper output/Reducer input value type
- * @param <TReducedKey>
- *            Reducer output key type
- * @param <TReducedValue>
- *            Reducer output value type
- * 
- * 
- */
-public abstract class AbstractMapReduceTests<TKey, TValue, TIntermediateKey, TIntermediateValue, TReducedKey, TReducedValue> {
-
-    /**
-     * Gets the mapper instance to test
-     * 
-     * @return Mapper instance
-     */
-    protected abstract Mapper<TKey, TValue, TIntermediateKey, TIntermediateValue> getMapperInstance();
-
-    /**
-     * Gets the reducer instance to test
-     * 
-     * @return Reducer instance
-     */
-    protected abstract Reducer<TIntermediateKey, TIntermediateValue, TReducedKey, TReducedValue> getReducerInstance();
-
-    /**
-     * Gets a map reduce driver that can be used to create a test case
-     * 
-     * @return Map reduce driver
-     */
-    protected MapReduceDriver<TKey, TValue, TIntermediateKey, TIntermediateValue, TReducedKey, TReducedValue> getMapReduceDriver() {
-        return new MapReduceDriver<TKey, TValue, TIntermediateKey, TIntermediateValue, TReducedKey, TReducedValue>(
-                this.getMapperInstance(), this.getReducerInstance());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapperTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapperTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapperTests.java
deleted file mode 100644
index ce6ab9d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/AbstractMapperTests.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-
-/**
- * Abstract tests for mappers
- * 
- * 
- * @param <TKeyIn>
- *            Input key type
- * @param <TValueIn>
- *            Input value type
- * @param <TKeyOut>
- *            Output key type
- * @param <TValueOut>
- *            Output value type
- * 
- */
-public abstract class AbstractMapperTests<TKeyIn, TValueIn, TKeyOut, TValueOut> {
-
-    /**
-     * Gets the mapper instance to test
-     * 
-     * @return Mapper instance
-     */
-    protected abstract Mapper<TKeyIn, TValueIn, TKeyOut, TValueOut> getInstance();
-
-    /**
-     * Gets a map driver that can be used to create a test case
-     * 
-     * @return Map driver
-     */
-    protected MapDriver<TKeyIn, TValueIn, TKeyOut, TValueOut> getMapDriver() {
-        MapDriver<TKeyIn, TValueIn, TKeyOut, TValueOut> driver = new MapDriver<TKeyIn, TValueIn, TKeyOut, TValueOut>(
-                this.getInstance());
-        this.configureDriver(driver);
-        return driver;
-    }
-
-    /**
-     * Method that may be overridden by test harnesses which need to configure
-     * the driver in more detail e.g. add configuration keys
-     * 
-     * @param driver
-     *            Driver
-     */
-    protected void configureDriver(MapDriver<TKeyIn, TValueIn, TKeyOut, TValueOut> driver) {
-        // Does nothing
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/TestDistinctTriples.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/TestDistinctTriples.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/TestDistinctTriples.java
deleted file mode 100644
index af32dac..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/TestDistinctTriples.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.junit.Assert;
-import org.junit.Test;
-
-import com.hp.hpl.jena.graph.Node;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-public class TestDistinctTriples
-        extends
-        AbstractMapReduceTests<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> {
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, TripleWritable, NullWritable> getMapperInstance() {
-        return new ValuePlusNullMapper<LongWritable, TripleWritable>();
-    }
-
-    @Override
-    protected Reducer<TripleWritable, NullWritable, NullWritable, TripleWritable> getReducerInstance() {
-        return new NullPlusKeyReducer<TripleWritable, NullWritable>();
-    }
-
-    @Test
-    public void distinct_triples_01() throws IOException {
-        MapReduceDriver<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> driver = this
-                .getMapReduceDriver();
-
-        Triple t = new Triple(NodeFactory.createURI("urn:s"), NodeFactory.createURI("urn:p"),
-                NodeFactory.createLiteral("1"));
-        TripleWritable tw = new TripleWritable(t);
-        driver.addInput(new LongWritable(1), tw);
-        driver.addOutput(NullWritable.get(), tw);
-
-        driver.runTest();
-    }
-
-    @Test
-    public void distinct_triples_02() throws IOException {
-        MapReduceDriver<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> driver = this
-                .getMapReduceDriver();
-
-        Triple t = new Triple(NodeFactory.createURI("urn:s"), NodeFactory.createURI("urn:p"),
-                NodeFactory.createLiteral("1"));
-        TripleWritable tw = new TripleWritable(t);
-        for (int i = 0; i < 100; i++) {
-            driver.addInput(new LongWritable(i), tw);
-        }
-        driver.addOutput(NullWritable.get(), tw);
-
-        driver.runTest();
-    }
-
-    @Test
-    public void distinct_triples_03() throws IOException {
-        MapReduceDriver<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> driver = this
-                .getMapReduceDriver();
-
-        Triple t = new Triple(NodeFactory.createURI("urn:s"), NodeFactory.createURI("urn:p"),
-                NodeFactory.createLiteral("1"));
-        Triple t2 = new Triple(t.getSubject(), t.getPredicate(), NodeFactory.createLiteral("2"));
-        Assert.assertNotEquals(t, t2);
-
-        TripleWritable tw = new TripleWritable(t);
-        TripleWritable tw2 = new TripleWritable(t2);
-        Assert.assertNotEquals(tw, tw2);
-
-        driver.addInput(new LongWritable(1), tw);
-        driver.addInput(new LongWritable(2), tw2);
-        driver.addOutput(NullWritable.get(), tw);
-        driver.addOutput(NullWritable.get(), tw2);
-
-        driver.runTest(false);
-    }
-
-    @Test
-    public void distinct_triples_04() throws IOException {
-        MapReduceDriver<LongWritable, TripleWritable, TripleWritable, NullWritable, NullWritable, TripleWritable> driver = this
-                .getMapReduceDriver();
-
-        Node s1 = NodeFactory.createURI("urn:nf#cbf2b2c7-109e-4097-bbea-f67f272c7fcc");
-        Node s2 = NodeFactory.createURI("urn:nf#bb08b75c-1ad2-47ef-acd2-eb2d92b94b89");
-        Node p = NodeFactory.createURI("urn:p");
-        Node o = NodeFactory.createURI("urn:66.230.159.118");
-        Assert.assertNotEquals(s1, s2);
-
-        Triple t1 = new Triple(s1, p, o);
-        Triple t2 = new Triple(s2, p, o);
-        Assert.assertNotEquals(t1, t2);
-
-        TripleWritable tw1 = new TripleWritable(t1);
-        TripleWritable tw2 = new TripleWritable(t2);
-        Assert.assertNotEquals(tw1, tw2);
-        Assert.assertNotEquals(0, tw1.compareTo(tw2));
-
-        driver.addInput(new LongWritable(1), tw1);
-        driver.addInput(new LongWritable(2), tw2);
-        driver.addOutput(NullWritable.get(), tw1);
-        driver.addOutput(NullWritable.get(), tw2);
-
-        driver.runTest(false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducerTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducerTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducerTests.java
deleted file mode 100644
index b2d0b92..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/AbstractCharacteristicSetGeneratingReducerTests.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapReduceTests;
-import org.apache.jena.hadoop.rdf.mapreduce.characteristics.AbstractCharacteristicSetGeneratingReducer;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
-import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.junit.Test;
-
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * Abstract tests for the {@link AbstractCharacteristicSetGeneratingReducer}
- * 
- * 
- * 
- * @param <TValue>
- * @param <T>
- */
-public abstract class AbstractCharacteristicSetGeneratingReducerTests<TValue, T extends AbstractNodeTupleWritable<TValue>>
-        extends AbstractMapReduceTests<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> {
-
-    /**
-     * Create a tuple
-     * 
-     * @param i
-     *            Key to use in creating the subject
-     * @param predicateUri
-     *            Predicate URI string
-     * @return Tuple
-     */
-    protected abstract T createTuple(int i, String predicateUri);
-
-    /**
-     * Creates a set consisting of the given predicates
-     * 
-     * @param predicates
-     *            Predicates
-     * @return Set
-     */
-    protected CharacteristicSetWritable createSet(MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver, int occurrences, String... predicates) {
-        CharacteristicSetWritable set = new CharacteristicSetWritable();
-        for (String predicateUri : predicates) {
-            set.add(new CharacteristicWritable(NodeFactory.createURI(predicateUri)));
-        }
-        for (int i = 1; i <= occurrences; i++) {
-            driver.addOutput(set, NullWritable.get());
-        }
-        return set;
-    }
-
-    /**
-     * Test basic characteristic set computation
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_generating_reducer_01() throws IOException {
-        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-        T tuple = this.createTuple(1, "http://predicate");
-        driver.addInput(new LongWritable(1), tuple);
-
-        this.createSet(driver, 1, "http://predicate");
-
-        driver.runTest(false);
-    }
-
-    /**
-     * Test basic characteristic set computation
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_generating_reducer_02() throws IOException {
-        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-        T tuple = this.createTuple(1, "http://predicate");
-        driver.addInput(new LongWritable(1), tuple);
-        driver.addInput(new LongWritable(1), tuple);
-
-        this.createSet(driver, 1, "http://predicate");
-
-        driver.runTest(false);
-    }
-
-    /**
-     * Test basic characteristic set computation
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_generating_reducer_03() throws IOException {
-        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-        T tuple = this.createTuple(1, "http://predicate");
-        driver.addInput(new LongWritable(1), tuple);
-        tuple = this.createTuple(2, "http://predicate");
-        driver.addInput(new LongWritable(2), tuple);
-
-        this.createSet(driver, 2, "http://predicate");
-
-        driver.runTest(false);
-    }
-
-    /**
-     * Test basic characteristic set computation
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_generating_reducer_04() throws IOException {
-        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-        T tuple = this.createTuple(1, "http://predicate");
-        driver.addInput(new LongWritable(1), tuple);
-        tuple = this.createTuple(1, "http://other");
-        driver.addInput(new LongWritable(1), tuple);
-
-        // Single entry sets
-        this.createSet(driver, 1, "http://predicate");
-        this.createSet(driver, 1, "http://other");
-        
-        // Two entry sets
-        this.createSet(driver, 1, "http://predicate", "http://other");
-
-        driver.runTest(false);
-    }
-
-    /**
-     * Test basic characteristic set computation
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_generating_reducer_05() throws IOException {
-        MapReduceDriver<LongWritable, T, NodeWritable, T, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-        T tuple = this.createTuple(1, "http://predicate");
-        driver.addInput(new LongWritable(1), tuple);
-        tuple = this.createTuple(1, "http://other");
-        driver.addInput(new LongWritable(2), tuple);
-        tuple = this.createTuple(1, "http://third");
-        driver.addInput(new LongWritable(3), tuple);
-
-        // Single entry sets
-        this.createSet(driver, 1, "http://predicate");
-        this.createSet(driver, 1, "http://other");
-        this.createSet(driver, 1, "http://third");
-
-        // Two entry sets
-        this.createSet(driver, 1, "http://predicate", "http://other");
-        this.createSet(driver, 1, "http://predicate", "http://third");
-        this.createSet(driver, 1, "http://other", "http://third");
-        
-        // Three entry sets
-        this.createSet(driver, 1, "http://predicate", "http://other", "http://third");
-
-        driver.runTest(false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducerTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducerTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducerTest.java
deleted file mode 100644
index 30da730..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/CharacteristicSetReducerTest.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
-import org.apache.hadoop.mrunit.types.Pair;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapReduceTests;
-import org.apache.jena.hadoop.rdf.mapreduce.characteristics.CharacteristicSetReducer;
-import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
-import org.apache.jena.hadoop.rdf.types.CharacteristicWritable;
-import org.junit.Assert;
-import org.junit.Test;
-
-import com.hp.hpl.jena.graph.NodeFactory;
-
-/**
- * Abstract tests for the {@link CharacteristicSetReducer}
- * 
- * 
- */
-public class CharacteristicSetReducerTest
-        extends
-        AbstractMapReduceTests<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> {
-
-    @Override
-    protected final Mapper<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable> getMapperInstance() {
-        // Identity mapper
-        return new Mapper<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable>();
-    }
-
-    @Override
-    protected final Reducer<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> getReducerInstance() {
-        return new CharacteristicSetReducer();
-    }
-
-    /**
-     * Creates a set consisting of the given predicates
-     * 
-     * @param predicates
-     *            Predicates
-     * @return Set
-     */
-    protected CharacteristicSetWritable createSet(
-            MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver,
-            int inputOccurrences, int outputOccurrences, String... predicates) {
-        CharacteristicSetWritable set = new CharacteristicSetWritable();
-        for (String predicateUri : predicates) {
-            set.add(new CharacteristicWritable(NodeFactory.createURI(predicateUri)));
-        }
-        for (int i = 1; i <= inputOccurrences; i++) {
-            driver.addInput(set, set);
-        }
-        for (int i = 1; i <= outputOccurrences; i++) {
-            driver.addOutput(set, NullWritable.get());
-        }
-        return set;
-    }
-
-    /**
-     * Test characteristic set reduction
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_reducer_01() throws IOException {
-        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-
-        this.createSet(driver, 1, 1, "http://predicate");
-
-        driver.runTest(false);
-    }
-
-    /**
-     * Test characteristic set reduction
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_reducer_02() throws IOException {
-        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-
-        this.createSet(driver, 2, 1, "http://predicate");
-
-        driver.runTest(false);
-
-        List<Pair<CharacteristicSetWritable, NullWritable>> results = driver.run();
-        CharacteristicSetWritable cw = results.get(0).getFirst();
-        Assert.assertEquals(2, cw.getCount().get());
-    }
-
-    /**
-     * Test characteristic set reduction
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_reducer_03() throws IOException {
-        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-
-        this.createSet(driver, 1, 1, "http://predicate");
-        this.createSet(driver, 1, 1, "http://other");
-
-        driver.runTest(false);
-    }
-
-    /**
-     * Test characteristic set reduction
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_reducer_04() throws IOException {
-        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-
-        this.createSet(driver, 2, 1, "http://predicate");
-        this.createSet(driver, 1, 1, "http://other");
-
-        driver.runTest(false);
-
-        List<Pair<CharacteristicSetWritable, NullWritable>> results = driver.run();
-        for (Pair<CharacteristicSetWritable, NullWritable> pair : results) {
-            CharacteristicSetWritable cw = pair.getFirst();
-            boolean expectTwo = cw.getCharacteristics().next().getNode().get().hasURI("http://predicate");
-            Assert.assertEquals(expectTwo ? 2 : 1, cw.getCount().get());
-        }
-    }
-
-    /**
-     * Test characteristic set reduction
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_reducer_05() throws IOException {
-        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-
-        this.createSet(driver, 1, 1, "http://predicate", "http://other");
-        this.createSet(driver, 1, 1, "http://other");
-
-        driver.runTest(false);
-    }
-
-    /**
-     * Test characteristic set reduction
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void characteristic_set_reducer_06() throws IOException {
-        MapReduceDriver<CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, CharacteristicSetWritable, NullWritable> driver = this
-                .getMapReduceDriver();
-
-        this.createSet(driver, 2, 1, "http://predicate", "http://other");
-        this.createSet(driver, 1, 1, "http://other");
-
-        driver.runTest(false);
-
-        List<Pair<CharacteristicSetWritable, NullWritable>> results = driver.run();
-        for (Pair<CharacteristicSetWritable, NullWritable> pair : results) {
-            CharacteristicSetWritable cw = pair.getFirst();
-            boolean expectTwo = cw.hasCharacteristic("http://predicate");
-            Assert.assertEquals(expectTwo ? 2 : 1, cw.getCount().get());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducerTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducerTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducerTest.java
deleted file mode 100644
index e647b68..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/characteristics/TripleCharacteristicSetGeneratingReducerTest.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.characteristics;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.jena.hadoop.rdf.mapreduce.characteristics.TripleCharacteristicSetGeneratingReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.group.TripleGroupBySubjectMapper;
-import org.apache.jena.hadoop.rdf.types.CharacteristicSetWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Tests for the {@link TripleCharacteristicSetGeneratingReducer}
- * 
- * 
- * 
- */
-public class TripleCharacteristicSetGeneratingReducerTest extends AbstractCharacteristicSetGeneratingReducerTests<Triple, TripleWritable> {
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, NodeWritable, TripleWritable> getMapperInstance() {
-        return new TripleGroupBySubjectMapper<LongWritable>();
-    }
-
-    @Override
-    protected Reducer<NodeWritable, TripleWritable, CharacteristicSetWritable, NullWritable> getReducerInstance() {
-        return new TripleCharacteristicSetGeneratingReducer();
-    }
-
-    @Override
-    protected TripleWritable createTuple(int i, String predicateUri) {
-        return new TripleWritable(new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI(predicateUri),
-                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountReducedTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountReducedTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountReducedTests.java
deleted file mode 100644
index ebdbcde..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountReducedTests.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapReduceTests;
-import org.apache.jena.hadoop.rdf.mapreduce.count.AbstractNodeTupleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.junit.Test;
-
-
-/**
- * Abstract tests for mappers derived from
- * {@link AbstractNodeTupleNodeCountMapper}
- * 
- * 
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleNodeCountReducedTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        AbstractMapReduceTests<LongWritable, T, NodeWritable, LongWritable, NodeWritable, LongWritable> {
-
-    /**
-     * Generates tuples for the tests
-     * 
-     * @param driver
-     *            Driver
-     * @param num
-     *            Number of tuples to generate
-     */
-    protected void generateData(MapReduceDriver<LongWritable, T, NodeWritable, LongWritable, NodeWritable, LongWritable> driver, int num) {
-        Map<NodeWritable, Long> counts = new HashMap<NodeWritable, Long>();
-        for (int i = 0; i < num; i++) {
-            LongWritable key = new LongWritable(i);
-            T value = this.createValue(i);
-            NodeWritable[] nodes = this.getNodes(value);
-
-            driver.addInput(key, value);
-            for (NodeWritable n : nodes) {
-                if (counts.containsKey(n)) {
-                    counts.put(n, counts.get(n) + 1);
-                } else {
-                    counts.put(n, 1l);
-                }
-            }
-        }
-        
-        for (Entry<NodeWritable, Long> kvp : counts.entrySet()) {
-            driver.addOutput(kvp.getKey(), new LongWritable(kvp.getValue()));
-        }
-    }
-
-    /**
-     * Creates a tuple value
-     * 
-     * @param i
-     *            Index
-     * @return Tuple value
-     */
-    protected abstract T createValue(int i);
-
-    /**
-     * Splits the tuple value into its constituent nodes
-     * 
-     * @param tuple
-     *            Tuple value
-     * @return Nodes
-     */
-    protected abstract NodeWritable[] getNodes(T tuple);
-
-    /**
-     * Runs a node count test
-     * 
-     * @param num
-     *            Number of tuples to generate
-     * @throws IOException
-     */
-    protected void testNodeCount(int num) throws IOException {
-        MapReduceDriver<LongWritable, T, NodeWritable, LongWritable, NodeWritable, LongWritable> driver = this.getMapReduceDriver();
-        this.generateData(driver, num);
-        driver.runTest(false);
-    }
-
-    /**
-     * Tests node counting
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void node_count_01() throws IOException {
-        this.testNodeCount(1);
-    }
-
-    /**
-     * Tests node counting
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void node_count_02() throws IOException {
-        this.testNodeCount(100);
-    }
-
-    /**
-     * Tests node counting
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void node_count_03() throws IOException {
-        this.testNodeCount(1000);
-    }
-
-    /**
-     * Tests node counting
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void node_count_04() throws IOException {
-        this.testNodeCount(2500);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountTests.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountTests.java
deleted file mode 100644
index e589b3a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/AbstractNodeTupleNodeCountTests.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mrunit.mapreduce.MapDriver;
-import org.apache.jena.hadoop.rdf.mapreduce.AbstractMapperTests;
-import org.apache.jena.hadoop.rdf.mapreduce.count.AbstractNodeTupleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.junit.Test;
-
-
-/**
- * Abstract tests for mappers derived from
- * {@link AbstractNodeTupleNodeCountMapper}
- * 
- * 
- * 
- * @param <TValue>
- *            Tuple type
- * @param <T>
- *            Writable tuple type
- */
-public abstract class AbstractNodeTupleNodeCountTests<TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        AbstractMapperTests<LongWritable, T, NodeWritable, LongWritable> {
-
-    /**
-     * Generates tuples for the tests
-     * 
-     * @param driver
-     *            Driver
-     * @param num
-     *            Number of tuples to generate
-     */
-    protected void generateData(MapDriver<LongWritable, T, NodeWritable, LongWritable> driver, int num) {
-        LongWritable expectedCount = new LongWritable(1);
-        for (int i = 0; i < num; i++) {
-            LongWritable key = new LongWritable(i);
-            T value = this.createValue(i);
-            NodeWritable[] nodes = this.getNodes(value);
-
-            driver.addInput(key, value);
-            for (NodeWritable n : nodes) {
-                driver.addOutput(n, expectedCount);
-            }
-        }
-    }
-
-    /**
-     * Creates a tuple value
-     * 
-     * @param i
-     *            Index
-     * @return Tuple value
-     */
-    protected abstract T createValue(int i);
-
-    /**
-     * Splits the tuple value into its constituent nodes
-     * 
-     * @param tuple
-     *            Tuple value
-     * @return Nodes
-     */
-    protected abstract NodeWritable[] getNodes(T tuple);
-
-    /**
-     * Runs a node count test
-     * 
-     * @param num
-     *            Number of tuples to generate
-     * @throws IOException
-     */
-    protected void testNodeCount(int num) throws IOException {
-        MapDriver<LongWritable, T, NodeWritable, LongWritable> driver = this.getMapDriver();
-        this.generateData(driver, num);
-        driver.runTest();
-    }
-
-    /**
-     * Tests node counting
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void node_count_01() throws IOException {
-        this.testNodeCount(1);
-    }
-
-    /**
-     * Tests node counting
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void node_count_02() throws IOException {
-        this.testNodeCount(100);
-    }
-
-    /**
-     * Tests node counting
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void node_count_03() throws IOException {
-        this.testNodeCount(1000);
-    }
-
-    /**
-     * Tests node counting
-     * 
-     * @throws IOException
-     */
-    @Test
-    public void node_count_04() throws IOException {
-        this.testNodeCount(2500);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapReduceTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapReduceTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapReduceTest.java
deleted file mode 100644
index b453bee..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapReduceTest.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Tests for the {@link TripleNodeCountMapper} used in conjunction with the
- * {@link NodeCountReducer}
- * 
- * 
- * 
- */
-public class QuadNodeCountMapReduceTest extends AbstractNodeTupleNodeCountReducedTests<Quad, QuadWritable> {
-
-    @Override
-    protected Mapper<LongWritable, QuadWritable, NodeWritable, LongWritable> getMapperInstance() {
-        return new QuadNodeCountMapper<LongWritable>();
-    }
-
-    @Override
-    protected Reducer<NodeWritable, LongWritable, NodeWritable, LongWritable> getReducerInstance() {
-        return new NodeCountReducer();
-    }
-
-    @Override
-    protected QuadWritable createValue(int i) {
-        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
-                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
-    }
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        Quad q = tuple.get();
-        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
-                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapperTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapperTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapperTest.java
deleted file mode 100644
index 869fc06..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/QuadNodeCountMapperTest.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.mapreduce.count.QuadNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * Tests for the {@link QuadNodeCountMapper}
- * 
- * 
- * 
- */
-public class QuadNodeCountMapperTest extends AbstractNodeTupleNodeCountTests<Quad, QuadWritable> {
-
-    @Override
-    protected Mapper<LongWritable, QuadWritable, NodeWritable, LongWritable> getInstance() {
-        return new QuadNodeCountMapper<LongWritable>();
-    }
-
-    @Override
-    protected QuadWritable createValue(int i) {
-        return new QuadWritable(new Quad(Quad.defaultGraphNodeGenerated, new Triple(
-                NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger))));
-    }
-
-    @Override
-    protected NodeWritable[] getNodes(QuadWritable tuple) {
-        Quad q = tuple.get();
-        return new NodeWritable[] { new NodeWritable(q.getGraph()), new NodeWritable(q.getSubject()),
-                new NodeWritable(q.getPredicate()), new NodeWritable(q.getObject()) };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapReduceTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapReduceTest.java b/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapReduceTest.java
deleted file mode 100644
index 660cfe7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-mapreduce/src/test/java/org/apache/jena/hadoop/rdf/mapreduce/count/TripleNodeCountMapReduceTest.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- *     
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.hadoop.rdf.mapreduce.count;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.NodeCountReducer;
-import org.apache.jena.hadoop.rdf.mapreduce.count.TripleNodeCountMapper;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
-import com.hp.hpl.jena.graph.NodeFactory;
-import com.hp.hpl.jena.graph.Triple;
-
-/**
- * Tests for the {@link TripleNodeCountMapper} used in conjunction with the
- * {@link NodeCountReducer}
- * 
- * 
- * 
- */
-public class TripleNodeCountMapReduceTest extends AbstractNodeTupleNodeCountReducedTests<Triple, TripleWritable> {
-
-    @Override
-    protected Mapper<LongWritable, TripleWritable, NodeWritable, LongWritable> getMapperInstance() {
-        return new TripleNodeCountMapper<LongWritable>();
-    }
-    
-
-    @Override
-    protected Reducer<NodeWritable, LongWritable, NodeWritable, LongWritable> getReducerInstance() {
-        return new NodeCountReducer();
-    }
-
-    @Override
-    protected TripleWritable createValue(int i) {
-        return new TripleWritable(
-                new Triple(NodeFactory.createURI("http://subjects/" + i), NodeFactory.createURI("http://predicate"),
-                        NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger)));
-    }
-
-    @Override
-    protected NodeWritable[] getNodes(TripleWritable tuple) {
-        Triple t = tuple.get();
-        return new NodeWritable[] { new NodeWritable(t.getSubject()), new NodeWritable(t.getPredicate()),
-                new NodeWritable(t.getObject()) };
-    }
-}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
deleted file mode 100644
index 7b04ef4..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/HadoopRdfIORegistry.java
+++ /dev/null
@@ -1,310 +0,0 @@
-/*
- * 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.registry;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.ServiceLoader;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * A registry which is used by various classes to dynamically select record
- * readers and writers based on a provided {@link Lang}
- * <p>
- * Readers and writers are dynamically discovered using the Java
- * {@link ServiceLoader} mechanism. This will look for files under
- * {@code META-INF/services} named
- * {@code org.apache.jena.hadoop.rdf.io.registry.ReaderFactory} and
- * {@code org.apache.jena.hadoop.rdf.io.registry.WriterFactory}. This follows
- * the standard {@linkplain ServiceLoader} format of provided one class name per
- * line which implements the relevant interface.
- * </p>
- * 
- */
-public class HadoopRdfIORegistry {
-
-    private static Map<Lang, ReaderFactory> readerFactories = new HashMap<>();
-    private static Map<Lang, WriterFactory> writerFactories = new HashMap<>();
-    private static boolean init = false;
-
-    static {
-        init();
-    }
-
-    private static synchronized void init() {
-        if (init)
-            return;
-
-        // Dynamically load and register reader factories
-        ServiceLoader<ReaderFactory> readerFactoryLoader = ServiceLoader.load(ReaderFactory.class);
-        Iterator<ReaderFactory> readerFactoryIterator = readerFactoryLoader.iterator();
-        while (readerFactoryIterator.hasNext()) {
-            ReaderFactory f = readerFactoryIterator.next();
-            addReaderFactory(f);
-        }
-
-        // Dynamically load and register writer factories
-        ServiceLoader<WriterFactory> writerFactoryLoader = ServiceLoader.load(WriterFactory.class);
-        Iterator<WriterFactory> writerFactoryIterator = writerFactoryLoader.iterator();
-        while (writerFactoryIterator.hasNext()) {
-            WriterFactory f = writerFactoryIterator.next();
-            addWriterFactory(f);
-        }
-
-        init = true;
-    }
-
-    /**
-     * Resets the registry to the default configuration
-     */
-    public static synchronized void reset() {
-        if (!init)
-            return;
-
-        init = false;
-        init();
-    }
-
-    /**
-     * Registers the reader factory for all the languages it declares itself as
-     * supporting
-     * 
-     * @param f
-     *            Reader factory
-     */
-    public static void addReaderFactory(ReaderFactory f) {
-        if (f == null)
-            throw new NullPointerException("Factory cannot be null");
-
-        readerFactories.put(f.getPrimaryLanguage(), f);
-        for (Lang altLang : f.getAlternativeLanguages()) {
-            readerFactories.put(altLang, f);
-        }
-    }
-
-    /**
-     * Registers the writer factory for all the languages it declares itself as
-     * supporting
-     * 
-     * @param f
-     *            Writer factory
-     */
-    public static void addWriterFactory(WriterFactory f) {
-        if (f == null)
-            throw new NullPointerException("Factory cannot be null");
-
-        writerFactories.put(f.getPrimaryLanguage(), f);
-        for (Lang altLang : f.getAlternativeLanguages()) {
-            writerFactories.put(altLang, f);
-        }
-    }
-
-    /**
-     * Gets whether there is a quad reader available for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return True if available, false otherwise
-     */
-    public static boolean hasQuadReader(Lang lang) {
-        if (lang == null)
-            return false;
-
-        ReaderFactory f = readerFactories.get(lang);
-        if (f == null)
-            return false;
-        return f.canReadQuads();
-    }
-
-    /**
-     * Gets whether there is a triple reader available for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return True if available, false otherwise
-     */
-    public static boolean hasTriplesReader(Lang lang) {
-        if (lang == null)
-            return false;
-
-        ReaderFactory f = readerFactories.get(lang);
-        if (f == null)
-            return false;
-        return f.canReadTriples();
-    }
-
-    /**
-     * Tries to create a quad reader for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return Quad reader if one is available
-     * @throws IOException
-     *             Thrown if a quad reader is not available or the given
-     *             language does not support quads
-     */
-    public static RecordReader<LongWritable, QuadWritable> createQuadReader(Lang lang) throws IOException {
-        if (lang == null)
-            throw new IOException("Cannot create a quad reader for an undefined language");
-
-        ReaderFactory f = readerFactories.get(lang);
-        if (f == null)
-            throw new IOException("No factory registered for language " + lang.getName());
-        if (!f.canReadQuads())
-            throw new IOException(lang.getName() + " does not support reading quads");
-
-        RecordReader<LongWritable, QuadWritable> reader = f.createQuadReader();
-        if (reader == null)
-            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples reader");
-        return reader;
-    }
-
-    /**
-     * Tries to create a triple reader for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return Triple reader if one is available
-     * @throws IOException
-     *             Thrown if a triple reader is not available or the given
-     *             language does not support triple
-     */
-    public static RecordReader<LongWritable, TripleWritable> createTripleReader(Lang lang) throws IOException {
-        if (lang == null)
-            throw new IOException("Cannot create a triple reader for an undefined language");
-
-        ReaderFactory f = readerFactories.get(lang);
-        if (f == null)
-            throw new IOException("No factory registered for language " + lang.getName());
-        if (!f.canReadTriples())
-            throw new IOException(lang.getName() + " does not support reading triples");
-
-        RecordReader<LongWritable, TripleWritable> reader = f.createTripleReader();
-        if (reader == null)
-            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples reader");
-        return reader;
-    }
-
-    /**
-     * Gets whether there is a quad writer available for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return True if available, false otherwise
-     */
-    public static boolean hasQuadWriter(Lang lang) {
-        if (lang == null)
-            return false;
-
-        WriterFactory f = writerFactories.get(lang);
-        if (f == null)
-            return false;
-        return f.canWriteQuads();
-    }
-
-    /**
-     * Gets whether there is a triple writer available for the given language
-     * 
-     * @param lang
-     *            Language
-     * @return True if available, false otherwise
-     */
-    public static boolean hasTriplesWriter(Lang lang) {
-        if (lang == null)
-            return false;
-
-        WriterFactory f = writerFactories.get(lang);
-        if (f == null)
-            return false;
-        return f.canWriteTriples();
-    }
-
-    /**
-     * Tries to create a quad writer for the given language
-     * 
-     * @param lang
-     *            Language
-     * @param writer
-     *            Writer
-     * @param config
-     *            Configuration
-     * 
-     * @return Quad writer if one is available
-     * @throws IOException
-     *             Thrown if a quad writer is not available or the given
-     *             language does not support quads
-     */
-    public static <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Lang lang, Writer writer,
-            Configuration config) throws IOException {
-        if (lang == null)
-            throw new IOException("Cannot create a quad writer for an undefined language");
-
-        WriterFactory f = writerFactories.get(lang);
-        if (f == null)
-            throw new IOException("No factory registered for language " + lang.getName());
-        if (!f.canWriteQuads())
-            throw new IOException(lang.getName() + " does not support writeing quads");
-
-        RecordWriter<TKey, QuadWritable> rwriter = f.<TKey> createQuadWriter(writer, config);
-        if (rwriter == null)
-            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples writer");
-        return rwriter;
-    }
-
-    /**
-     * Tries to create a triple writer for the given language
-     * 
-     * @param lang
-     *            Language
-     * @param writer
-     *            Writer
-     * @param config
-     *            Configuration
-     * @return Triple writer if one is available
-     * @throws IOException
-     *             Thrown if a triple writer is not available or the given
-     *             language does not support triple
-     */
-    public static <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Lang lang, Writer writer,
-            Configuration config) throws IOException {
-        if (lang == null)
-            throw new IOException("Cannot create a triple writer for an undefined language");
-
-        WriterFactory f = writerFactories.get(lang);
-        if (f == null)
-            throw new IOException("No factory registered for language " + lang.getName());
-        if (!f.canWriteTriples())
-            throw new IOException(lang.getName() + " does not support writing triples");
-
-        RecordWriter<TKey, TripleWritable> rwriter = f.<TKey> createTripleWriter(writer, config);
-        if (rwriter == null)
-            throw new IOException("Registered factory for " + lang.getName() + " produced a null triples writer");
-        return rwriter;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
deleted file mode 100644
index e1c98c7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/ReaderFactory.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.registry;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Interface for reader factories
- * 
- */
-public interface ReaderFactory {
-
-    /**
-     * Gets the primary language this factory produces readers for
-     * 
-     * @return Primary language
-     */
-    public abstract Lang getPrimaryLanguage();
-
-    /**
-     * Gets the alternative languages this factory can produce readers for
-     * 
-     * @return Alternative languages
-     */
-    public abstract Collection<Lang> getAlternativeLanguages();
-
-    /**
-     * Gets whether this factory can produce readers that are capable of reading
-     * quads
-     * 
-     * @return True if quads can be read, false if not
-     */
-    public abstract boolean canReadQuads();
-
-    /**
-     * Gets whether this factory can produce readers that are capable of reading
-     * triples
-     * 
-     * @return True if triples can be read, false if not
-     */
-    public abstract boolean canReadTriples();
-
-    /**
-     * Creates a quad reader
-     * 
-     * @return Quad reader
-     * @throws IOException
-     *             May be thrown if a quad reader cannot be created
-     */
-    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
-
-    /**
-     * Creates a triples reader
-     * 
-     * @return Triples reader
-     * @throws IOException
-     *             May be thrown if a triple reader cannot be created
-     */
-    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java
deleted file mode 100644
index db5635f..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/WriterFactory.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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.registry;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Interface for writer factories
- * 
- */
-public interface WriterFactory {
-
-    /**
-     * Gets the primary language this factory produces writers for
-     * 
-     * @return Primary language
-     */
-    public abstract Lang getPrimaryLanguage();
-
-    /**
-     * Gets the alternative languages this factory can produce writers for
-     * 
-     * @return Alternative languages
-     */
-    public abstract Collection<Lang> getAlternativeLanguages();
-
-    /**
-     * Gets whether this factory can produce writers that are capable of reading
-     * quads
-     * 
-     * @return True if quads can be read, false if not
-     */
-    public abstract boolean canWriteQuads();
-
-    /**
-     * Gets whether this factory can produce writers that are capable of reading
-     * triples
-     * 
-     * @return True if triples can be read, false if not
-     */
-    public abstract boolean canWriteTriples();
-
-    /**
-     * Creates a quad writer
-     * 
-     * @param writer
-     *            Writer
-     * @param config
-     *            Configuration
-     * 
-     * @return Quad writer
-     * @throws IOException
-     *             May be thrown if a quad writer cannot be created
-     */
-    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException;
-
-    /**
-     * Creates a triples writer
-     * 
-     * @param writer
-     *            Writer
-     * @param config
-     *            Configuration
-     * 
-     * @return Triples writer
-     * @throws IOException
-     *             May be thrown if a triple writer cannot be created
-     */
-    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
deleted file mode 100644
index 7fe15a9..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractQuadsOnlyReaderFactory.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract reader factory for languages that only support quads
- */
-public abstract class AbstractQuadsOnlyReaderFactory implements ReaderFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
-
-    public AbstractQuadsOnlyReaderFactory(Lang lang) {
-        this(lang, (Collection<Lang>)null);
-    }
-    
-    public AbstractQuadsOnlyReaderFactory(Lang lang, Lang...altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractQuadsOnlyReaderFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-    
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canReadQuads() {
-        return true;
-    }
-
-    @Override
-    public final boolean canReadTriples() {
-        return false;
-    }
-
-    @Override
-    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
-
-    @Override
-    public final RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        throw new IOException(this.lang.getName() + " does not support reading triples");
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java
deleted file mode 100644
index 60e45af..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractReaderFactory.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract reader factory for languages that support triples and quads
- */
-public abstract class AbstractReaderFactory implements ReaderFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
-
-    public AbstractReaderFactory(Lang lang) {
-        this(lang, (Collection<Lang>)null);
-    }
-    
-    public AbstractReaderFactory(Lang lang, Lang...altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractReaderFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-    
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canReadQuads() {
-        return true;
-    }
-
-    @Override
-    public final boolean canReadTriples() {
-        return true;
-    }
-
-    @Override
-    public abstract RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException;
-
-    @Override
-    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
deleted file mode 100644
index 7fb8131..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/AbstractTriplesOnlyReaderFactory.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.registry.ReaderFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract reader factory for languages that only support triples
- */
-public abstract class AbstractTriplesOnlyReaderFactory implements ReaderFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
-
-    public AbstractTriplesOnlyReaderFactory(Lang lang) {
-        this(lang, (Collection<Lang>)null);
-    }
-    
-    public AbstractTriplesOnlyReaderFactory(Lang lang, Lang...altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractTriplesOnlyReaderFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-    
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-    
-    @Override
-    public final boolean canReadQuads() {
-        return false;
-    }
-
-    @Override
-    public final boolean canReadTriples() {
-        return true;
-    }
-
-    @Override
-    public final RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        throw new IOException(this.lang.getName() + " does not support reading quads");
-    }
-
-    @Override
-    public abstract RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java
deleted file mode 100644
index 6b064a4..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/JsonLDReaderFactory.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.JsonLDQuadReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.jsonld.JsonLDTripleReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class JsonLDReaderFactory extends AbstractReaderFactory {
-    
-    public JsonLDReaderFactory() {
-        super(Lang.JSONLD);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new JsonLDQuadReader();
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new JsonLDTripleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
deleted file mode 100644
index 2296296..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NQuadsReaderFactory.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.nquads.WholeFileNQuadsReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class NQuadsReaderFactory extends AbstractQuadsOnlyReaderFactory {
-    
-    public NQuadsReaderFactory() {
-        super(Lang.NQUADS, Lang.NQ);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new WholeFileNQuadsReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
deleted file mode 100644
index a98a1ae..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/NTriplesReaderFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.ntriples.WholeFileNTriplesReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-public class NTriplesReaderFactory extends AbstractTriplesOnlyReaderFactory {
-
-    public NTriplesReaderFactory() {
-        super(Lang.NTRIPLES, Lang.NT);
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new WholeFileNTriplesReader();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
deleted file mode 100644
index ccf5feb..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfJsonReaderFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.rdfjson.RdfJsonReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class RdfJsonReaderFactory extends AbstractTriplesOnlyReaderFactory {
-
-    public RdfJsonReaderFactory() {
-        super(Lang.RDFJSON);
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new RdfJsonReader();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
deleted file mode 100644
index 1aa88d7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/RdfXmlReaderFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.rdfxml.RdfXmlReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-public class RdfXmlReaderFactory extends AbstractTriplesOnlyReaderFactory {
-
-    public RdfXmlReaderFactory() {
-        super(Lang.RDFXML);
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new RdfXmlReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java
deleted file mode 100644
index 25e8234..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/ThriftReaderFactory.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftQuadReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftTripleReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- *
- */
-public class ThriftReaderFactory extends AbstractReaderFactory {
-    
-    public ThriftReaderFactory() {
-        super(RDFLanguages.THRIFT);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new ThriftQuadReader();
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new ThriftTripleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
deleted file mode 100644
index 83ea818..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriGReaderFactory.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.trig.TriGReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class TriGReaderFactory extends AbstractQuadsOnlyReaderFactory {
-
-    public TriGReaderFactory() {
-        super(Lang.TRIG);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new TriGReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
deleted file mode 100644
index cb8795c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TriXReaderFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.trix.TriXReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class TriXReaderFactory extends AbstractQuadsOnlyReaderFactory {
-
-    public TriXReaderFactory() {
-        super(Lang.TRIX);
-    }
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createQuadReader() throws IOException {
-        return new TriXReader();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
deleted file mode 100644
index 7800376..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/readers/TurtleReaderFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.registry.readers;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.jena.hadoop.rdf.io.input.readers.turtle.TurtleReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-public class TurtleReaderFactory extends AbstractTriplesOnlyReaderFactory {
-    
-    public TurtleReaderFactory() {
-        super(Lang.TURTLE, Lang.TTL, Lang.N3);
-    }
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createTripleReader() throws IOException {
-        return new TurtleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
deleted file mode 100644
index 0cf137e..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractQuadsOnlyWriterFactory.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract writer factory for languages that only support quads
- */
-public abstract class AbstractQuadsOnlyWriterFactory implements WriterFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang> emptyList());
-
-    public AbstractQuadsOnlyWriterFactory(Lang lang) {
-        this(lang, (Collection<Lang>) null);
-    }
-
-    public AbstractQuadsOnlyWriterFactory(Lang lang, Lang... altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractQuadsOnlyWriterFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canWriteQuads() {
-        return true;
-    }
-
-    @Override
-    public final boolean canWriteTriples() {
-        return false;
-    }
-
-    @Override
-    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException;
-
-    @Override
-    public final <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        throw new IOException(this.lang.getName() + " does not support writing triples");
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
deleted file mode 100644
index e45c3da..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractTriplesOnlyWriterFactory.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract writer factory for languages that only support triples
- */
-public abstract class AbstractTriplesOnlyWriterFactory implements WriterFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang> emptyList());
-
-    public AbstractTriplesOnlyWriterFactory(Lang lang) {
-        this(lang, (Collection<Lang>) null);
-    }
-
-    public AbstractTriplesOnlyWriterFactory(Lang lang, Lang... altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractTriplesOnlyWriterFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canWriteQuads() {
-        return false;
-    }
-
-    @Override
-    public final boolean canWriteTriples() {
-        return true;
-    }
-
-    @Override
-    public final <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        throw new IOException(this.lang.getName() + " does not support writing quads");
-    }
-
-    @Override
-    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
deleted file mode 100644
index 669b9c4..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/AbstractWriterFactory.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-import java.io.Writer;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.jena.hadoop.rdf.io.registry.WriterFactory;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- * Abstract writer factory for languages that support triples and quads
- */
-public abstract class AbstractWriterFactory implements WriterFactory {
-
-    private Lang lang;
-    private Collection<Lang> alternateLangs = Collections.unmodifiableList(Collections.<Lang>emptyList());
-
-    public AbstractWriterFactory(Lang lang) {
-        this(lang, (Collection<Lang>)null);
-    }
-    
-    public AbstractWriterFactory(Lang lang, Lang...altLangs) {
-        this(lang, Arrays.asList(altLangs));
-    }
-
-    public AbstractWriterFactory(Lang lang, Collection<Lang> altLangs) {
-        this.lang = lang;
-        if (altLangs != null)
-            this.alternateLangs = Collections.unmodifiableCollection(altLangs);
-    }
-
-    @Override
-    public final Lang getPrimaryLanguage() {
-        return this.lang;
-    }
-    
-    @Override
-    public final Collection<Lang> getAlternativeLanguages() {
-        return this.alternateLangs;
-    }
-
-    @Override
-    public final boolean canWriteQuads() {
-        return true;
-    }
-
-    @Override
-    public final boolean canWriteTriples() {
-        return true;
-    }
-
-    @Override
-    public abstract <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config) throws IOException;
-
-    @Override
-    public abstract <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java
deleted file mode 100644
index 89e93ed..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/JsonLDWriterFactory.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.jsonld.JsonLDQuadWriter;
-import org.apache.jena.hadoop.rdf.io.output.writers.jsonld.JsonLDTripleWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class JsonLDWriterFactory extends AbstractWriterFactory {
-    
-    public JsonLDWriterFactory() {
-        super(Lang.JSONLD);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new JsonLDQuadWriter<>(writer);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new JsonLDTripleWriter<>(writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java
deleted file mode 100644
index abbbd0f..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NQuadsWriterFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.nquads.NQuadsWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class NQuadsWriterFactory extends AbstractQuadsOnlyWriterFactory {
-    
-    public NQuadsWriterFactory() {
-        super(Lang.NQUADS, Lang.NQ);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new NQuadsWriter<TKey>(writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java
deleted file mode 100644
index 88c9551..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/NTriplesWriterFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.ntriples.NTriplesWriter;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class NTriplesWriterFactory extends AbstractTriplesOnlyWriterFactory {
-    
-    public NTriplesWriterFactory() {
-        super(Lang.NTRIPLES, Lang.NT);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new NTriplesWriter<TKey>(writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
deleted file mode 100644
index 8252422..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfJsonWriterFactory.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.rdfjson.RdfJsonWriter;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class RdfJsonWriterFactory extends AbstractTriplesOnlyWriterFactory {
-
-    public RdfJsonWriterFactory() {
-        super(Lang.RDFJSON);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new RdfJsonWriter<TKey>(writer);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java
deleted file mode 100644
index b4ac8e3..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/RdfXmlWriterFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.rdfxml.RdfXmlWriter;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-/**
- *
- */
-public class RdfXmlWriterFactory extends AbstractTriplesOnlyWriterFactory {
-    
-    public RdfXmlWriterFactory() {
-        super(Lang.RDFXML);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new RdfXmlWriter<TKey>(writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java
deleted file mode 100644
index 757472c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/ThriftWriterFactory.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.writers.StreamRdfQuadWriter;
-import org.apache.jena.hadoop.rdf.io.output.writers.StreamRdfTripleWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.RDFLanguages;
-import org.apache.jena.riot.thrift.StreamRDF2Thrift;
-
-/**
- *
- */
-public class ThriftWriterFactory extends AbstractWriterFactory {
-
-    public ThriftWriterFactory() {
-        super(RDFLanguages.THRIFT);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfQuadWriter<TKey>(new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")),
-                false), writer);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfTripleWriter<TKey>(new StreamRDF2Thrift(new WriterOutputStream(writer, Charset.forName("utf-8")),
-                false), writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java
deleted file mode 100644
index 6d8b08a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriGWriterFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.StreamRdfQuadWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
-
-/**
- *
- */
-public class TriGWriterFactory extends AbstractQuadsOnlyWriterFactory {
-    
-    public TriGWriterFactory() {
-        super(Lang.TRIG);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfQuadWriter<TKey>(new WriterStreamRDFBlocks(writer), writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
deleted file mode 100644
index 0e1b7b2..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TriXWriterFactory.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.writers.StreamRdfQuadWriter;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.writer.StreamWriterTriX;
-
-/**
- *
- */
-public class TriXWriterFactory extends AbstractQuadsOnlyWriterFactory {
-    
-    public TriXWriterFactory() {
-        super(Lang.TRIX);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, QuadWritable> createQuadWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfQuadWriter<>(new StreamWriterTriX(new WriterOutputStream(writer, Charset.forName("utf-8"))), writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java b/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java
deleted file mode 100644
index c837f12..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/java/org/apache/jena/hadoop/rdf/io/registry/writers/TurtleWriterFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.registry.writers;
-
-import java.io.IOException;
-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.StreamRdfTripleWriter;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.writer.WriterStreamRDFBlocks;
-
-/**
- *
- */
-public class TurtleWriterFactory extends AbstractTriplesOnlyWriterFactory {
-    
-    public TurtleWriterFactory() {
-        super(Lang.TURTLE, Lang.TTL, Lang.N3);
-    }
-
-    @Override
-    public <TKey> RecordWriter<TKey, TripleWritable> createTripleWriter(Writer writer, Configuration config)
-            throws IOException {
-        return new StreamRdfTripleWriter<>(new WriterStreamRDFBlocks(writer), writer);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory b/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
deleted file mode 100644
index ec0e48a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.ReaderFactory
+++ /dev/null
@@ -1,10 +0,0 @@
-# Default Reader Factory implementations
-org.apache.jena.hadoop.rdf.io.registry.readers.JsonLDReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.NQuadsReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.NTriplesReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.RdfJsonReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.RdfXmlReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.ThriftReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.TriGReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.TriXReaderFactory
-org.apache.jena.hadoop.rdf.io.registry.readers.TurtleReaderFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory b/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
deleted file mode 100644
index 164880d..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/main/resources/META-INF/services/org.apache.jena.hadoop.rdf.io.registry.WriterFactory
+++ /dev/null
@@ -1,10 +0,0 @@
-# Default Writer Factory implementations
-org.apache.jena.hadoop.rdf.io.registry.writers.JsonLDWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.NQuadsWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.NTriplesWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.RdfJsonWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.RdfXmlWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.ThriftWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.TriGWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.TriXWriterFactory
-org.apache.jena.hadoop.rdf.io.registry.writers.TurtleWriterFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
deleted file mode 100644
index 5762fb7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/RdfTriplesInputTestMapper.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.log4j.Logger;
-
-
-/**
- * A test mapper which takes in line based RDF triple input and just produces triples
- * 
- *
- */
-public class RdfTriplesInputTestMapper extends Mapper<LongWritable, TripleWritable, NullWritable, TripleWritable> {
-    
-    private static final Logger LOG = Logger.getLogger(RdfTriplesInputTestMapper.class);
-
-    @Override
-    protected void map(LongWritable key, TripleWritable value, Context context)
-            throws IOException, InterruptedException {
-        LOG.info("Line " + key.toString() + " => " + value.toString());
-        context.write(NullWritable.get(), value);
-    }
-
-    
-}


[03/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
new file mode 100644
index 0000000..b2b3c33
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.trig;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped TriG input
+ * 
+ * 
+ * 
+ */
+public class BZippedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedTriGInputTest() {
+        super(".trig.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
new file mode 100644
index 0000000..c9579a9
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.trig;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated TriG input
+ * 
+ * 
+ * 
+ */
+public class DeflatedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedTriGInputTest() {
+        super(".trig.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
new file mode 100644
index 0000000..c3e4106
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.trig;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped TriG input
+ * 
+ * 
+ * 
+ */
+public class GZippedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedTriGInputTest() {
+        super(".trig.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
new file mode 100644
index 0000000..ad98e35
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
@@ -0,0 +1,72 @@
+/*
+ * 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.trix;
+
+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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.trix.TriXInputFormat;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed TriX input tests
+ */
+public abstract class AbstractCompressedTriXInputFormatTests extends
+        AbstractCompressedWholeFileQuadInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedTriXInputFormatTests(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.TRIX;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new TriXInputFormat();
+    }
+
+}
\ No newline at end of file

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

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

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

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
new file mode 100644
index 0000000..68d776a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
@@ -0,0 +1,75 @@
+/*
+ * 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.turtle;
+
+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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.io.input.turtle.TurtleInputFormat;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Abstract compressed Turtle input tests
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractCompressedTurtleInputFormatTests extends
+        AbstractCompressedWholeFileTripleInputFormatTests {
+
+    private String ext;
+    private CompressionCodec codec;
+
+    /**
+     * Creates new tests
+     * 
+     * @param ext
+     *            File extension
+     * @param codec
+     *            Compression codec
+     */
+    public AbstractCompressedTurtleInputFormatTests(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.TURTLE;
+    }
+
+    @Override
+    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new TurtleInputFormat();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
new file mode 100644
index 0000000..724b847
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.turtle;
+
+import org.apache.hadoop.io.compress.BZip2Codec;
+
+/**
+ * Tests for BZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class BZippedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public BZippedTurtleInputTest() {
+        super(".nt.bz2", new BZip2Codec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
new file mode 100644
index 0000000..eb5ee03
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.turtle;
+
+import org.apache.hadoop.io.compress.DefaultCodec;
+
+/**
+ * Tests for Deflated NTriples input
+ * 
+ * 
+ * 
+ */
+public class DeflatedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public DeflatedTurtleInputTest() {
+        super(".nt.deflate", new DefaultCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
new file mode 100644
index 0000000..817805c
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.turtle;
+
+import org.apache.hadoop.io.compress.GzipCodec;
+
+/**
+ * Tests for GZipped NTriples input
+ * 
+ * 
+ * 
+ */
+public class GZippedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
+
+    /**
+     * Creates new tests
+     */
+    public GZippedTurtleInputTest() {
+        super(".nt.gz", new GzipCodec());
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
new file mode 100644
index 0000000..92aac53
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.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.input.jsonld;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
new file mode 100644
index 0000000..63b6738
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.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.input.jsonld;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
new file mode 100644
index 0000000..6d1d02a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.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.input.nquads;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractBlockedQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for blocked NTriples input
+ * 
+ * 
+ * 
+ */
+public class BlockedNQuadsInputTest extends AbstractBlockedQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nq";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new BlockedNQuadsInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
new file mode 100644
index 0000000..3823728
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.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.input.nquads;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractQuadsInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+
+/**
+ * Tests for the NQuads input format
+ * 
+ *
+ */
+public class NQuadsInputTest extends AbstractQuadsInputFormatTests {
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new NQuadsInputFormat();
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nq";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
new file mode 100644
index 0000000..50b8bcf
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.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.input.nquads;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for NQuads input
+ * 
+ * 
+ * 
+ */
+public class WholeFileNQuadsInputTest extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nq";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new WholeFileNQuadsInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
new file mode 100644
index 0000000..ab92873
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.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.input.ntriples;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractBlockedTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for blocked NTriples input
+ * 
+ *
+ */
+public class BlockedNTriplesInputTest extends AbstractBlockedTripleInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nt";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new BlockedNTriplesInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
new file mode 100644
index 0000000..24fb731
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.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.input.ntriples;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractTriplesInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+
+/**
+ * Tests for the {@link NTriplesInputFormat}
+ * 
+ * 
+ * 
+ */
+public class NTriplesInputTest extends AbstractTriplesInputFormatTests {
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new NTriplesInputFormat();
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nt";
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
new file mode 100644
index 0000000..fcec570
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.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.input.ntriples;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for the {@link NTriplesInputFormat}
+ * 
+ * 
+ * 
+ */
+public class WholeFileNTriplesInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new WholeFileNTriplesInputFormat();
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".nt";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+    
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.java
new file mode 100644
index 0000000..4731832
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfjson/RdfJsonInputTest.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.input.rdfjson;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for RDF/JSON input
+ * 
+ * 
+ * 
+ */
+public class RdfJsonInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFJSON;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".rj";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new RdfJsonInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.java
new file mode 100644
index 0000000..f8edcc4
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/rdfxml/RdfXmlInputTest.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.input.rdfxml;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for RDF/XML input
+ * 
+ * 
+ * 
+ */
+public class RdfXmlInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFXML;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".rdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new RdfXmlInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.java
new file mode 100644
index 0000000..8d79295
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputTest.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.input.thrift;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+
+/**
+ * Tests for JSON-LD input
+ * 
+ *
+ */
+public class ThriftQuadInputTest extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new ThriftQuadInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.java
new file mode 100644
index 0000000..6b5e0b7
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputTest.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.input.thrift;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+
+
+/**
+ * Tests for JSON-LD input
+ * 
+ *
+ */
+public class ThriftTripleInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return RDFLanguages.THRIFT;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trdf";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new ThriftTripleInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.java
new file mode 100644
index 0000000..1fad0dc
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputTest.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.input.trig;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for TriG input
+ * 
+ *
+ */
+public class TriGInputTest extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIG;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trig";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new TriGInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.java
new file mode 100644
index 0000000..4a3a66a
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputTest.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.input.trix;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for TriX input
+ * 
+ *
+ */
+public class TriXInputTest extends AbstractWholeFileQuadInputFormatTests {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIX;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trix";
+    }
+
+    @Override
+    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
+        return new TriXInputFormat();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.java
new file mode 100644
index 0000000..e6211ba
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputTest.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.input.turtle;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for turtle input format
+ * 
+ * 
+ * 
+ */
+public class TurtleInputTest extends AbstractWholeFileTripleInputFormatTests {
+
+    @Override
+    protected final String getFileExtension() {
+        return ".ttl";
+    }
+
+    @Override
+    protected final Lang getRdfLanguage() {
+        return Lang.TURTLE;
+    }
+    
+    @Override
+    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
+        return new TurtleInputFormat();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
new file mode 100644
index 0000000..9532d56
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/AbstractTrackableInputStreamTests.java
@@ -0,0 +1,701 @@
+/*
+ * 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.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Abstract tests for {@link TrackableInputStream} implementations
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractTrackableInputStreamTests {
+
+    protected static final int KILO = 1024;
+    protected static final int BYTES_PER_KB = KILO;
+    protected static final int BYTES_PER_MB = BYTES_PER_KB * KILO;
+
+    /**
+     * Gets the instance to test using the given input as the stream to track
+     * 
+     * @param input
+     *            Input Stream
+     * @return Trackable Input Stream
+     */
+    protected abstract TrackableInputStream getInstance(InputStream input);
+
+    /**
+     * Generates an input stream containing the given number of bytes
+     * 
+     * @param length
+     *            Number of bytes
+     * @return Input stream
+     */
+    protected final InputStream generateData(int length) {
+        ByteArrayOutputStream output = new ByteArrayOutputStream(length);
+        byte b = (byte) 'b';
+        for (int i = 0; i < length; i++) {
+            output.write(b);
+        }
+        return new ByteArrayInputStream(output.toByteArray());
+    }
+
+    protected final void testSingleByteRead(int length) throws IOException {
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input);
+        long count = 0;
+        while (trackable.read() >= 0) {
+            count++;
+        }
+        Assert.assertEquals(length, count);
+        Assert.assertEquals(length, trackable.getBytesRead());
+        trackable.close();
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_single_01() throws IOException {
+        this.testSingleByteRead(0);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_single_02() throws IOException {
+        this.testSingleByteRead(100);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_single_03() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_single_04() throws IOException {
+        // 1 MB
+        this.testSingleByteRead(BYTES_PER_MB);
+    }
+
+    protected final void testMultiByteRead(int length, int bufferSize) throws IOException {
+        if (bufferSize < 1)
+            throw new IllegalArgumentException("bufferSize must be >= 1");
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input);
+        long count = 0;
+        byte[] buffer = new byte[bufferSize];
+        long read;
+        do {
+            read = trackable.read(buffer);
+            if (read > 0)
+                count += read;
+        } while (read >= 0);
+        Assert.assertEquals(length, count);
+        Assert.assertEquals(length, trackable.getBytesRead());
+        trackable.close();
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_01() throws IOException {
+        this.testMultiByteRead(0, 1);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_02() throws IOException {
+        this.testMultiByteRead(0, 16);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_03() throws IOException {
+        this.testMultiByteRead(0, BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_04() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_05() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_06() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_07() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_08() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes i.e. calling {@link InputStream#read(byte[])}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_09() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB);
+    }
+
+    protected final void testMultiByteRead(int length, int bufferSize, int readSize) throws IOException {
+        if (bufferSize < 1)
+            throw new IllegalArgumentException("bufferSize must be >= 1");
+        if (readSize < 1 || readSize > bufferSize)
+            throw new IllegalArgumentException("readSize must be >= 1 and <= bufferSize");
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input);
+        long count = 0;
+        byte[] buffer = new byte[bufferSize];
+        long read;
+        do {
+            read = trackable.read(buffer, 0, readSize);
+            if (read > 0)
+                count += read;
+        } while (read >= 0);
+        Assert.assertEquals(length, count);
+        Assert.assertEquals(length, trackable.getBytesRead());
+        trackable.close();
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_01() throws IOException {
+        this.testMultiByteRead(0, 1, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_02() throws IOException {
+        this.testMultiByteRead(0, 16, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_03() throws IOException {
+        this.testMultiByteRead(0, 16, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_04() throws IOException {
+        this.testMultiByteRead(0, BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_05() throws IOException {
+        this.testMultiByteRead(0, BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_06() throws IOException {
+        this.testMultiByteRead(0, BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_07() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 1, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_08() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 16, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_09() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, 16, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_10() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_11() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_12() throws IOException {
+        // 1KB
+        this.testMultiByteRead(BYTES_PER_KB, BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_13() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 1, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_14() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 16, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_15() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, 16, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_16() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_17() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test reading multiple bytes while reading less than the buffer size bytes
+     * i.e. calling {@link InputStream#read(byte[], int, int)}
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_read_multiple_partial_18() throws IOException {
+        // 1MB
+        this.testMultiByteRead(BYTES_PER_MB, BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    protected final void testSkip(int length, long skipSize) throws IOException {
+        if (skipSize < 1)
+            throw new IllegalArgumentException("skipSize must be >= 1");
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input);
+        long count = 0;
+        long skipped;
+        do {
+            skipped = trackable.skip(skipSize);
+            if (skipped > 0)
+                count += skipped;
+        } while (skipped > 0);
+        Assert.assertEquals(length, count);
+        Assert.assertEquals(length, trackable.getBytesRead());
+        trackable.close();
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_01() throws IOException {
+        this.testSkip(0, 1);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_02() throws IOException {
+        this.testSkip(100, 1);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_03() throws IOException {
+        this.testSkip(100, 16);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_04() throws IOException {
+        this.testSkip(100, BYTES_PER_KB);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_05() throws IOException {
+        // 1KB
+        this.testSkip(BYTES_PER_KB, 1);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_06() throws IOException {
+        // 1KB
+        this.testSkip(BYTES_PER_KB, 16);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_07() throws IOException {
+        // 1KB
+        this.testSkip(BYTES_PER_KB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_08() throws IOException {
+        // 1KB
+        this.testSkip(BYTES_PER_KB, BYTES_PER_MB);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_09() throws IOException {
+        // 1 MB
+        this.testSkip(BYTES_PER_MB, 1);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_10() throws IOException {
+        // 1 MB
+        this.testSkip(BYTES_PER_MB, 16);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_11() throws IOException {
+        // 1 MB
+        this.testSkip(BYTES_PER_MB, BYTES_PER_KB);
+    }
+
+    /**
+     * Test skipping
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_skip_single_12() throws IOException {
+        // 1 MB
+        this.testSkip(BYTES_PER_MB, BYTES_PER_MB);
+    }
+
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_01() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(-1, trackable.read());
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_02() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(0, trackable.read(new byte[0]));
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_03() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(-1, trackable.read(new byte[1]));
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_04() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(0, trackable.read(new byte[16], 0, 0));
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_05() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(-1, trackable.read(new byte[16], 0, 8));
+    }
+    
+    /**
+     * Tests behaviour after closing
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_06() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(0, trackable.skip(0));
+    }
+    
+    /**
+     * Tests exceptions are thrown trying to perform actions after closing the
+     * input
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void trackable_input_post_close_07() throws IOException {
+        InputStream input = this.generateData(0);
+        TrackableInputStream trackable = this.getInstance(input);
+        trackable.close();
+        Assert.assertEquals(0, trackable.skip(1));
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
new file mode 100644
index 0000000..f8819bc
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStreamTest.java
@@ -0,0 +1,240 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.jena.hadoop.rdf.io.input.util.BlockInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the {@link BlockInputStream}
+ * 
+ * 
+ * 
+ */
+public class BlockInputStreamTest extends AbstractTrackableInputStreamTests {
+
+    @Override
+    protected TrackableInputStream getInstance(InputStream input) {
+        return new BlockInputStream(input, Long.MAX_VALUE);
+    }
+
+    /**
+     * Gets an instance of a block input stream
+     * 
+     * @param input
+     *            Underlying input stream
+     * @param limit
+     *            Limit on bytes to read
+     * @return Block input stream
+     */
+    protected BlockInputStream getInstance(InputStream input, long limit) {
+        return new BlockInputStream(input, limit);
+    }
+    
+    protected final void testSingleByteRead(int length, long limit) throws IOException {
+        InputStream input = this.generateData(length);
+        TrackableInputStream trackable = this.getInstance(input, limit);
+        long count = 0;
+        while (trackable.read() >= 0) {
+            count++;
+        }
+        int expected = (int) Math.min(length, limit);
+        Assert.assertEquals(expected, count);
+        Assert.assertEquals(expected, trackable.getBytesRead());
+        trackable.close();
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_01() throws IOException {
+        this.testSingleByteRead(0, 0);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_02() throws IOException {
+        this.testSingleByteRead(100, 0);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_03() throws IOException {
+        this.testSingleByteRead(100, 50);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_04() throws IOException {
+        this.testSingleByteRead(100, 100);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_05() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, 1);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_06() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, 100);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_07() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_KB / 2);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_08() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_KB);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_09() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_MB);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_10() throws IOException {
+        // 1KB
+        this.testSingleByteRead(BYTES_PER_KB, BYTES_PER_MB * 10);
+    }
+
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_11() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, 1);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_12() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, 100);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_13() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_KB);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_14() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB / 2);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_15() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB);
+    }
+    
+    /**
+     * Test reading byte by byte
+     * 
+     * @throws IOException
+     */
+    @Test
+    public final void block_input_read_single_16() throws IOException {
+        // 1MB
+        this.testSingleByteRead(BYTES_PER_MB, BYTES_PER_MB * 10);
+    }
+
+}


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

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedNTriplesInputTest.java
deleted file mode 100644
index 3cb4ee0..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedNTriplesInputTest.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-
-
-
-/**
- * Tests for GZipped NTriples input
- * 
- * 
- * 
- */
-public class GZippedNTriplesInputTest extends AbstractCompressedNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedNTriplesInputTest() {
-        super(".nt.gz", new GzipCodec());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
deleted file mode 100644
index b7acc08..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/ntriples/GZippedWholeFileNTriplesInputTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped NTriples input
- * 
- * 
- * 
- */
-public class GZippedWholeFileNTriplesInputTest extends AbstractCompressedWholeFileNTriplesInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedWholeFileNTriplesInputTest() {
-        super(".nt.gz", new GzipCodec());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
deleted file mode 100644
index 3b2546d..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/AbstractCompressedRdfJsonInputFormatTests.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.rdfjson;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.rdfjson.RdfJsonInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed RDF/JSON input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedRdfJsonInputFormatTests extends AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedRdfJsonInputFormatTests(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.RDFJSON;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new RdfJsonInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
deleted file mode 100644
index 6e58d4b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/BZippedRdfJsonInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfjson;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped RDF/JSON input
- * 
- * 
- * 
- */
-public class BZippedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedRdfJsonInputTest() {
-        super(".rj.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
deleted file mode 100644
index 8b7b044..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/DeflatedRdfJsonInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfjson;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated RDF/JSON input
- * 
- * 
- * 
- */
-public class DeflatedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedRdfJsonInputTest() {
-        super(".rj.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
deleted file mode 100644
index 66996dd..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfjson/GZippedRdfJsonInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfjson;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped RDF/JSON input
- * 
- * 
- * 
- */
-public class GZippedRdfJsonInputTest extends AbstractCompressedRdfJsonInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedRdfJsonInputTest() {
-        super(".rj.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
deleted file mode 100644
index a6d1e24..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/AbstractCompressedRdfXmlInputFormatTests.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.rdfxml;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.rdfxml.RdfXmlInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed RDF/XML input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedRdfXmlInputFormatTests extends
- AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedRdfXmlInputFormatTests(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.RDFXML;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new RdfXmlInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
deleted file mode 100644
index 30a6c39..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/BZippedRdfXmlInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfxml;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped RDF/XML input
- * 
- * 
- * 
- */
-public class BZippedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedRdfXmlInputTest() {
-        super(".rdf.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
deleted file mode 100644
index a3d747c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/DeflatedRdfXmlInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfxml;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated RDF/XML input
- * 
- * 
- * 
- */
-public class DeflatedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedRdfXmlInputTest() {
-        super(".rdf.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
deleted file mode 100644
index 748785c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/rdfxml/GZippedRdfXmlInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.rdfxml;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped RDF/XML input
- * 
- * 
- * 
- */
-public class GZippedRdfXmlInputTest extends AbstractCompressedRdfXmlInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedRdfXmlInputTest() {
-        super(".rdf.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
deleted file mode 100644
index 78affb2..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftQuadInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.thrift;
-
-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftQuadInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Abstract compressed Thrift quad input tests
- */
-public abstract class AbstractCompressedThriftQuadInputFormatTests extends
-        AbstractCompressedWholeFileQuadInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedThriftQuadInputFormatTests(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 RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new ThriftQuadInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
deleted file mode 100644
index f837b89..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/AbstractCompressedThriftTripleInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.thrift;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.thrift.ThriftTripleInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-import org.apache.jena.riot.RDFLanguages;
-
-/**
- * Abstract compressed Thrift triple input tests
- */
-public abstract class AbstractCompressedThriftTripleInputFormatTests extends
-        AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedThriftTripleInputFormatTests(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 RDFLanguages.THRIFT;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new ThriftTripleInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
deleted file mode 100644
index 320d278..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped Thrift input
- */
-public class BZippedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedThriftQuadInputTest() {
-        super(".trdf.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
deleted file mode 100644
index bb2d65b..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/BZippedThriftTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped Thrift input
- */
-public class BZippedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedThriftTripleInputTest() {
-        super(".trdf.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
deleted file mode 100644
index 6872583..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated Thrift input
- */
-public class DeflatedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedThriftQuadInputTest() {
-        super(".trdf.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
deleted file mode 100644
index e76d2d7..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/DeflatedThriftTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated Thrift input
- */
-public class DeflatedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedThriftTripleInputTest() {
-        super(".trdf.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
deleted file mode 100644
index 6590f22..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftQuadInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped Thrift input
- */
-public class GZippedThriftQuadInputTest extends AbstractCompressedThriftQuadInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedThriftQuadInputTest() {
-        super(".trdf.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
deleted file mode 100644
index 1ce74f4..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/thrift/GZippedThriftTripleInputTest.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.thrift;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped Thrift input
- */
-public class GZippedThriftTripleInputTest extends AbstractCompressedThriftTripleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedThriftTripleInputTest() {
-        super(".trdf.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
deleted file mode 100644
index 2975f29..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/AbstractCompressedTriGInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.trig;
-
-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.trig.TriGInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed TriG input tests
- */
-public abstract class AbstractCompressedTriGInputFormatTests extends
-        AbstractCompressedWholeFileQuadInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedTriGInputFormatTests(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.TRIG;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new TriGInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
deleted file mode 100644
index b2b3c33..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/BZippedTriGInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.trig;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped TriG input
- * 
- * 
- * 
- */
-public class BZippedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedTriGInputTest() {
-        super(".trig.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
deleted file mode 100644
index c9579a9..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/DeflatedTriGInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.trig;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated TriG input
- * 
- * 
- * 
- */
-public class DeflatedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedTriGInputTest() {
-        super(".trig.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
deleted file mode 100644
index c3e4106..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trig/GZippedTriGInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.trig;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped TriG input
- * 
- * 
- * 
- */
-public class GZippedTriGInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedTriGInputTest() {
-        super(".trig.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
deleted file mode 100644
index ad98e35..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/AbstractCompressedTriXInputFormatTests.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.trix;
-
-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.compressed.AbstractCompressedWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.trix.TriXInputFormat;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed TriX input tests
- */
-public abstract class AbstractCompressedTriXInputFormatTests extends
-        AbstractCompressedWholeFileQuadInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedTriXInputFormatTests(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.TRIX;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new TriXInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/BZippedTriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/BZippedTriXInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/BZippedTriXInputTest.java
deleted file mode 100644
index fc51ec8..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/BZippedTriXInputTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.trix;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-import org.apache.jena.hadoop.rdf.io.input.compressed.trig.AbstractCompressedTriGInputFormatTests;
-
-/**
- * Tests for BZipped TriX input
- */
-public class BZippedTriXInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedTriXInputTest() {
-        super(".trix.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/DeflatedTriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/DeflatedTriXInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/DeflatedTriXInputTest.java
deleted file mode 100644
index a1a078d..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/DeflatedTriXInputTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.trix;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.jena.hadoop.rdf.io.input.compressed.trig.AbstractCompressedTriGInputFormatTests;
-
-/**
- * Tests for Deflated TriX input
- */
-public class DeflatedTriXInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedTriXInputTest() {
-        super(".trix.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/GZippedTriXInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/GZippedTriXInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/GZippedTriXInputTest.java
deleted file mode 100644
index 10c6980..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/trix/GZippedTriXInputTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.trix;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.jena.hadoop.rdf.io.input.compressed.trig.AbstractCompressedTriGInputFormatTests;
-
-/**
- * Tests for GZipped TriX input
- */
-public class GZippedTriXInputTest extends AbstractCompressedTriGInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedTriXInputTest() {
-        super(".trix.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
deleted file mode 100644
index 68d776a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/AbstractCompressedTurtleInputFormatTests.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.turtle;
-
-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.compressed.AbstractCompressedWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.io.input.turtle.TurtleInputFormat;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Abstract compressed Turtle input tests
- * 
- * 
- * 
- */
-public abstract class AbstractCompressedTurtleInputFormatTests extends
-        AbstractCompressedWholeFileTripleInputFormatTests {
-
-    private String ext;
-    private CompressionCodec codec;
-
-    /**
-     * Creates new tests
-     * 
-     * @param ext
-     *            File extension
-     * @param codec
-     *            Compression codec
-     */
-    public AbstractCompressedTurtleInputFormatTests(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.TURTLE;
-    }
-
-    @Override
-    protected final InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new TurtleInputFormat();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
deleted file mode 100644
index 724b847..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/BZippedTurtleInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.turtle;
-
-import org.apache.hadoop.io.compress.BZip2Codec;
-
-/**
- * Tests for BZipped NTriples input
- * 
- * 
- * 
- */
-public class BZippedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public BZippedTurtleInputTest() {
-        super(".nt.bz2", new BZip2Codec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
deleted file mode 100644
index eb5ee03..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/DeflatedTurtleInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.turtle;
-
-import org.apache.hadoop.io.compress.DefaultCodec;
-
-/**
- * Tests for Deflated NTriples input
- * 
- * 
- * 
- */
-public class DeflatedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public DeflatedTurtleInputTest() {
-        super(".nt.deflate", new DefaultCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
deleted file mode 100644
index 817805c..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/compressed/turtle/GZippedTurtleInputTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.turtle;
-
-import org.apache.hadoop.io.compress.GzipCodec;
-
-/**
- * Tests for GZipped NTriples input
- * 
- * 
- * 
- */
-public class GZippedTurtleInputTest extends AbstractCompressedTurtleInputFormatTests {
-
-    /**
-     * Creates new tests
-     */
-    public GZippedTurtleInputTest() {
-        super(".nt.gz", new GzipCodec());
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
deleted file mode 100644
index 92aac53..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDQuadInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.jsonld;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
deleted file mode 100644
index 63b6738..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/jsonld/JsonLDTripleInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.jsonld;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-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/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
deleted file mode 100644
index 6d1d02a..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/BlockedNQuadsInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractBlockedQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for blocked NTriples input
- * 
- * 
- * 
- */
-public class BlockedNQuadsInputTest extends AbstractBlockedQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nq";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new BlockedNQuadsInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
deleted file mode 100644
index 3823728..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/NQuadsInputTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractQuadsInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Tests for the NQuads input format
- * 
- *
- */
-public class NQuadsInputTest extends AbstractQuadsInputFormatTests {
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new NQuadsInputFormat();
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nq";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
deleted file mode 100644
index 50b8bcf..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/nquads/WholeFileNQuadsInputTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.nquads;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileQuadInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for NQuads input
- * 
- * 
- * 
- */
-public class WholeFileNQuadsInputTest extends AbstractWholeFileQuadInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NQUADS;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nq";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, QuadWritable> getInputFormat() {
-        return new WholeFileNQuadsInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
deleted file mode 100644
index ab92873..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/BlockedNTriplesInputTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractBlockedTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for blocked NTriples input
- * 
- *
- */
-public class BlockedNTriplesInputTest extends AbstractBlockedTripleInputFormatTests {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nt";
-    }
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new BlockedNTriplesInputFormat();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
deleted file mode 100644
index 24fb731..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/NTriplesInputTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractTriplesInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Tests for the {@link NTriplesInputFormat}
- * 
- * 
- * 
- */
-public class NTriplesInputTest extends AbstractTriplesInputFormatTests {
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new NTriplesInputFormat();
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nt";
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/49c4cffe/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
deleted file mode 100644
index fcec570..0000000
--- a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/ntriples/WholeFileNTriplesInputTest.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.ntriples;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.jena.hadoop.rdf.io.input.AbstractWholeFileTripleInputFormatTests;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-import org.apache.jena.riot.Lang;
-
-
-/**
- * Tests for the {@link NTriplesInputFormat}
- * 
- * 
- * 
- */
-public class WholeFileNTriplesInputTest extends AbstractWholeFileTripleInputFormatTests {
-
-    @Override
-    protected InputFormat<LongWritable, TripleWritable> getInputFormat() {
-        return new WholeFileNTriplesInputFormat();
-    }
-
-    @Override
-    protected String getFileExtension() {
-        return ".nt";
-    }
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.NTRIPLES;
-    }
-
-    
-}


[02/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
new file mode 100644
index 0000000..f453aa5
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStreamTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.input.util;
+
+import java.io.InputStream;
+
+import org.apache.jena.hadoop.rdf.io.input.util.TrackableInputStream;
+import org.apache.jena.hadoop.rdf.io.input.util.TrackedInputStream;
+
+/**
+ * Tests for the {@link TrackedInputStream}
+ * 
+ * 
+ * 
+ */
+public class TrackedInputStreamTest extends AbstractTrackableInputStreamTests {
+
+    @Override
+    protected TrackableInputStream getInstance(InputStream input) {
+        return new TrackedInputStream(input);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
new file mode 100644
index 0000000..b5ea2d8
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormatTests.java
@@ -0,0 +1,255 @@
+/*
+ * 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.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.lang.StreamRDFCounting;
+import org.apache.jena.riot.system.StreamRDFLib;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstract node tuple output format tests
+ * 
+ * 
+ * @param <TValue>
+ *            Tuple type
+ * @param <T>
+ *            Writable tuple type
+ * 
+ */
+public abstract class AbstractNodeTupleOutputFormatTests<TValue, T extends AbstractNodeTupleWritable<TValue>> {
+
+    @SuppressWarnings("unused")
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleOutputFormatTests.class);
+
+    protected static final int EMPTY_SIZE = 0, SMALL_SIZE = 100, LARGE_SIZE = 10000, VERY_LARGE_SIZE = 100000;
+
+    /**
+     * Temporary folder for the tests
+     */
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+
+    /**
+     * Prepares a fresh configuration
+     * 
+     * @return Configuration
+     */
+    protected Configuration prepareConfiguration() {
+        Configuration config = new Configuration(true);
+        // Nothing else to do
+        return config;
+    }
+
+    /**
+     * Gets the extra file extension to add to the filenames
+     * 
+     * @return File extension
+     */
+    protected abstract String getFileExtension();
+
+    /**
+     * Generates tuples to be output for testing
+     * 
+     * @param num
+     *            Number of tuples to generate
+     * @return Iterator of tuples
+     */
+    protected abstract Iterator<T> generateTuples(int num);
+
+    /**
+     * Counts tuples in the output file
+     * 
+     * @param f
+     *            Output file
+     * @return Tuple count
+     */
+    protected final long countTuples(File f) {
+        StreamRDFCounting counter = StreamRDFLib.count();
+        RDFDataMgr.parse(counter, f.getAbsolutePath(), this.getRdfLanguage(), null);
+        return counter.count();
+    }
+
+    /**
+     * Checks that tuples are as expected
+     * 
+     * @param f
+     *            File
+     * @param expected
+     *            Expected number of tuples
+     */
+    protected void checkTuples(File f, long expected) {
+        Assert.assertEquals(expected, this.countTuples(f));
+    }
+
+    /**
+     * Gets the RDF language of the produced output which is used to parse back
+     * in the output to validate the correct amount of output was produced
+     * 
+     * @return RDF language
+     */
+    protected abstract Lang getRdfLanguage();
+
+    /**
+     * Gets the output format to test
+     * 
+     * @return Output format
+     */
+    protected abstract OutputFormat<NullWritable, T> getOutputFormat();
+
+    /**
+     * Adds an output path to the job configuration
+     * 
+     * @param f
+     *            File
+     * @param config
+     *            Configuration
+     * @param job
+     *            Job
+     * @throws IOException
+     */
+    protected void addOutputPath(File f, Configuration config, Job job) throws IOException {
+        FileSystem fs = FileSystem.getLocal(config);
+        Path outputPath = fs.makeQualified(new Path(f.getAbsolutePath()));
+        FileOutputFormat.setOutputPath(job, outputPath);
+    }
+
+    protected File findOutputFile(File dir, JobContext context) throws FileNotFoundException, IOException {
+        Path outputPath = FileOutputFormat.getOutputPath(context);
+        RemoteIterator<LocatedFileStatus> files = outputPath.getFileSystem(context.getConfiguration()).listFiles(
+                outputPath, true);
+        while (files.hasNext()) {
+            LocatedFileStatus status = files.next();
+            if (status.isFile() && !status.getPath().getName().startsWith("_")) {
+                return new File(status.getPath().toUri());
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Tests output
+     * 
+     * @param f
+     *            File to output to
+     * @param num
+     *            Number of tuples to output
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    protected final void testOutput(File f, int num) throws IOException, InterruptedException {
+        // Prepare configuration
+        Configuration config = this.prepareConfiguration();
+
+        // Set up fake job
+        OutputFormat<NullWritable, T> outputFormat = this.getOutputFormat();
+        Job job = Job.getInstance(config);
+        job.setOutputFormatClass(outputFormat.getClass());
+        this.addOutputPath(f, job.getConfiguration(), job);
+        JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+        Assert.assertNotNull(FileOutputFormat.getOutputPath(context));
+
+        // Output the data
+        TaskAttemptID id = new TaskAttemptID("outputTest", 1, TaskType.MAP, 1, 1);
+        TaskAttemptContext taskContext = new TaskAttemptContextImpl(job.getConfiguration(), id);
+        RecordWriter<NullWritable, T> writer = outputFormat.getRecordWriter(taskContext);
+        Iterator<T> tuples = this.generateTuples(num);
+        while (tuples.hasNext()) {
+            writer.write(NullWritable.get(), tuples.next());
+        }
+        writer.close(taskContext);
+
+        // Check output
+        File outputFile = this.findOutputFile(this.folder.getRoot(), context);
+        Assert.assertNotNull(outputFile);
+        this.checkTuples(outputFile, num);
+    }
+
+    /**
+     * Basic output tests
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void output_01() throws IOException, InterruptedException {
+        this.testOutput(this.folder.getRoot(), EMPTY_SIZE);
+    }
+
+    /**
+     * Basic output tests
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void output_02() throws IOException, InterruptedException {
+        this.testOutput(this.folder.getRoot(), SMALL_SIZE);
+    }
+
+    /**
+     * Basic output tests
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    public void output_03() throws IOException, InterruptedException {
+        this.testOutput(this.folder.getRoot(), LARGE_SIZE);
+    }
+
+    /**
+     * Basic output tests
+     * 
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Test
+    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/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.java
new file mode 100644
index 0000000..f1822f6
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractQuadOutputFormatTests.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;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Abstract tests for quad output formats
+ * 
+ * 
+ * 
+ */
+public abstract class AbstractQuadOutputFormatTests extends AbstractNodeTupleOutputFormatTests<Quad, QuadWritable> {
+
+    @Override
+    protected Iterator<QuadWritable> generateTuples(int num) {
+        List<QuadWritable> qs = new ArrayList<QuadWritable>();
+        for (int i = 0; i < num; i++) {
+            Quad q = new Quad(NodeFactory.createURI("http://example.org/graphs/" + i),
+                    NodeFactory.createURI("http://example.org/subjects/" + i),
+                    NodeFactory.createURI("http://example.org/predicate"), NodeFactory.createLiteral(Integer.toString(i),
+                            XSDDatatype.XSDinteger));
+            qs.add(new QuadWritable(q));
+        }
+        return qs.iterator();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
new file mode 100644
index 0000000..90eb531
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/AbstractTripleOutputFormatTests.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+
+/**
+ * Abstract tests for triple output formats
+ * 
+ *
+ */
+public abstract class AbstractTripleOutputFormatTests extends AbstractNodeTupleOutputFormatTests<Triple, TripleWritable> {
+
+    @Override
+    protected Iterator<TripleWritable> generateTuples(int num) {
+        List<TripleWritable> ts = new ArrayList<TripleWritable>();
+        for (int i = 0; i < num; i++) {
+            Triple t = new Triple(NodeFactory.createURI("http://example.org/subjects/" + i), NodeFactory.createURI("http://example.org/predicate"), NodeFactory.createLiteral(Integer.toString(i), XSDDatatype.XSDinteger));
+            ts.add(new TripleWritable(t));
+        }
+        return ts.iterator();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
new file mode 100644
index 0000000..c6784a5
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdQuadOutputTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.jsonld;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * 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>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
new file mode 100644
index 0000000..d157409
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLdTripleOutputTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *     
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.hadoop.rdf.io.output.jsonld;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+/**
+ * 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>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.java
new file mode 100644
index 0000000..1a7ffa4
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputTest.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.nquads;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for NQuads output format
+ * 
+ * 
+ * 
+ */
+public class NQuadsOutputTest extends AbstractQuadOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".nq";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
+        return new NQuadsOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.java
new file mode 100644
index 0000000..ad9be56
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputTest.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.ntriples;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for NTriples output format
+ * 
+ * 
+ * 
+ */
+public class NTriplesOutputTest extends AbstractTripleOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".nt";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new NTriplesOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.java
new file mode 100644
index 0000000..833f89b
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputTest.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.rdfjson;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for RDF/JSON output
+ * 
+ * 
+ * 
+ */
+public class RdfJsonOutputTest extends AbstractTripleOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".rj";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFJSON;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new RdfJsonOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.java
new file mode 100644
index 0000000..40bc937
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputTest.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.rdfxml;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+
+
+/**
+ * Tests for RDF/XML output
+ * 
+ * 
+ * 
+ */
+public class RdfXmlOutputTest extends AbstractTripleOutputFormatTests {
+
+    @Override
+    protected String getFileExtension() {
+        return ".rdf";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFXML;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new RdfXmlOutputFormat<NullWritable>();
+    }
+
+}

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

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

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
new file mode 100644
index 0000000..fd886a3
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+
+/**
+ * Tests for TriG output
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class BatchedTriGOutputTest extends AbstractQuadOutputFormatTests {
+
+    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+    static long $bs2 = 1000;
+    static long $bs3 = 100;
+    static long $bs4 = 1;
+
+    /**
+     * @return Test parameters
+     */
+    @Parameters
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
+    }
+
+    private final long batchSize;
+
+    /**
+     * Creates new tests
+     * 
+     * @param batchSize
+     *            Batch size
+     */
+    public BatchedTriGOutputTest(long batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trig";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIG;
+    }
+
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
+        return config;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
+        return new BatchedTriGOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
new file mode 100644
index 0000000..9b2b669
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/StreamedTriGOutputTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.AbstractQuadOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+
+/**
+ * Tests for Turtle output
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class StreamedTriGOutputTest extends AbstractQuadOutputFormatTests {
+
+    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+    static long $bs2 = 1000;
+    static long $bs3 = 100;
+    static long $bs4 = 1;
+
+    /**
+     * @return Test parameters
+     */
+    @Parameters
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
+    }
+
+    private final long batchSize;
+
+    /**
+     * Creates new tests
+     * 
+     * @param batchSize
+     *            Batch size
+     */
+    public StreamedTriGOutputTest(long batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".trig";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIG;
+    }
+    
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
+        return config;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
+        return new TriGOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
new file mode 100644
index 0000000..c9b3a26
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGBlankNodeOutputTests.java
@@ -0,0 +1,120 @@
+/*
+ * 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.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.riot.RDFDataMgr;
+import org.junit.Assert;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ResIterator;
+import com.hp.hpl.jena.rdf.model.Resource;
+import com.hp.hpl.jena.sparql.core.Quad;
+
+/**
+ * Tests for TriG output with blank nodes
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class TriGBlankNodeOutputTests extends StreamedTriGOutputTest {
+
+	static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+	static long $bs2 = 1000;
+	static long $bs3 = 100;
+	static long $bs4 = 1;
+
+	/**
+	 * @return Test parameters
+	 */
+	@Parameters
+	public static Collection<Object[]> data() {
+		return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 },
+				{ $bs4 } });
+	}
+
+	/**
+	 * Creates new tests
+	 * 
+	 * @param batchSize
+	 *            Batch size
+	 */
+	public TriGBlankNodeOutputTests(long batchSize) {
+		super(batchSize);
+	}
+
+	@Override
+	protected Iterator<QuadWritable> generateTuples(int num) {
+		List<QuadWritable> qs = new ArrayList<QuadWritable>();
+		Node subject = NodeFactory.createAnon();
+		for (int i = 0; i < num; i++) {
+			Quad t = new Quad(
+					NodeFactory.createURI("http://example.org/graphs/" + i),
+					subject,
+					NodeFactory.createURI("http://example.org/predicate"),
+					NodeFactory.createLiteral(Integer.toString(i),
+							XSDDatatype.XSDinteger));
+			qs.add(new QuadWritable(t));
+		}
+		return qs.iterator();
+	}
+
+	@Override
+	protected void checkTuples(File f, long expected) {
+		super.checkTuples(f, expected);
+
+		Model m = RDFDataMgr.loadModel("file://" + f.getAbsolutePath(),
+				this.getRdfLanguage());
+		ResIterator iter = m.listSubjects();
+		Set<Node> subjects = new HashSet<Node>();
+		while (iter.hasNext()) {
+			Resource res = iter.next();
+			Assert.assertTrue(res.isAnon());
+			subjects.add(res.asNode());
+		}
+		// Should only be one subject unless the data was empty in which case
+		// there will be zero subjects
+		Assert.assertEquals(expected == 0 ? 0 : 1, subjects.size());
+	}
+
+	@Override
+	protected OutputFormat<NullWritable, QuadWritable> getOutputFormat() {
+		return new TriGOutputFormat<NullWritable>();
+	}
+
+}

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

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
new file mode 100644
index 0000000..a6c4d70
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/BatchedTurtleOutputTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+
+/**
+ * Tests for Turtle output
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class BatchedTurtleOutputTest extends AbstractTripleOutputFormatTests {
+
+    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+    static long $bs2 = 1000;
+    static long $bs3 = 100;
+    static long $bs4 = 1;
+
+    /**
+     * @return Test parameters
+     */
+    @Parameters
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
+    }
+
+    private final long batchSize;
+
+    /**
+     * Creates new tests
+     * 
+     * @param batchSize
+     *            Batch size
+     */
+    public BatchedTurtleOutputTest(long batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".ttl";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TURTLE;
+    }
+    
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
+        return config;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new BatchedTurtleOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
new file mode 100644
index 0000000..d8843d3
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/StreamedTurtleOutputTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.io.output.AbstractTripleOutputFormatTests;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+
+/**
+ * Tests for Turtle output
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class StreamedTurtleOutputTest extends AbstractTripleOutputFormatTests {
+
+    static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+    static long $bs2 = 1000;
+    static long $bs3 = 100;
+    static long $bs4 = 1;
+
+    /**
+     * @return Test parameters
+     */
+    @Parameters
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 }, { $bs4 } });
+    }
+
+    private final long batchSize;
+
+    /**
+     * Creates new tests
+     * 
+     * @param batchSize
+     *            Batch size
+     */
+    public StreamedTurtleOutputTest(long batchSize) {
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    protected String getFileExtension() {
+        return ".ttl";
+    }
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TURTLE;
+    }
+    
+    @Override
+    protected Configuration prepareConfiguration() {
+        Configuration config = super.prepareConfiguration();
+        config.setLong(RdfIOConstants.OUTPUT_BATCH_SIZE, this.batchSize);
+        return config;
+    }
+
+    @Override
+    protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+        return new TurtleOutputFormat<NullWritable>();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
new file mode 100644
index 0000000..8dcae4e
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/output/turtle/TurtleBlankNodeOutputTests.java
@@ -0,0 +1,118 @@
+/*
+ * 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.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.RDFDataMgr;
+import org.junit.Assert;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype;
+import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory;
+import com.hp.hpl.jena.graph.Triple;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ResIterator;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * Tests for Turtle output with blank nodes
+ * 
+ * 
+ * 
+ */
+@RunWith(Parameterized.class)
+public class TurtleBlankNodeOutputTests extends StreamedTurtleOutputTest {
+
+	static long $bs1 = RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE;
+	static long $bs2 = 1000;
+	static long $bs3 = 100;
+	static long $bs4 = 1;
+
+	/**
+	 * @return Test parameters
+	 */
+	@Parameters
+	public static Collection<Object[]> data() {
+		return Arrays.asList(new Object[][] { { $bs1 }, { $bs2 }, { $bs3 },
+				{ $bs4 } });
+	}
+
+	/**
+	 * Creates new tests
+	 * 
+	 * @param batchSize
+	 *            Batch size
+	 */
+	public TurtleBlankNodeOutputTests(long batchSize) {
+		super(batchSize);
+	}
+
+	@Override
+	protected Iterator<TripleWritable> generateTuples(int num) {
+		List<TripleWritable> ts = new ArrayList<TripleWritable>();
+		Node subject = NodeFactory.createAnon();
+		for (int i = 0; i < num; i++) {
+			Triple t = new Triple(subject,
+					NodeFactory.createURI("http://example.org/predicate"),
+					NodeFactory.createLiteral(Integer.toString(i),
+							XSDDatatype.XSDinteger));
+			ts.add(new TripleWritable(t));
+		}
+		return ts.iterator();
+	}
+
+	@Override
+	protected void checkTuples(File f, long expected) {
+		super.checkTuples(f, expected);
+
+		Model m = RDFDataMgr.loadModel("file://" + f.getAbsolutePath(),
+				this.getRdfLanguage());
+		ResIterator iter = m.listSubjects();
+		Set<Node> subjects = new HashSet<Node>();
+		while (iter.hasNext()) {
+			Resource res = iter.next();
+			Assert.assertTrue(res.isAnon());
+			subjects.add(res.asNode());
+		}
+		// Should only be one subject unless the data was empty in which case
+		// there will be zero subjects
+		Assert.assertEquals(expected == 0 ? 0 : 1, subjects.size());
+	}
+
+	@Override
+	protected OutputFormat<NullWritable, TripleWritable> getOutputFormat() {
+		return new TurtleOutputFormat<NullWritable>();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
new file mode 100644
index 0000000..2eae232
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-io/src/test/java/org/apache/jena/hadoop/rdf/io/registry/TestHadoopRdfIORegistry.java
@@ -0,0 +1,186 @@
+/*
+ * 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.registry;
+
+import java.io.IOException;
+import java.io.StringWriter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.jena.hadoop.rdf.types.QuadWritable;
+import org.apache.jena.hadoop.rdf.types.TripleWritable;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFLanguages;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for the {@link HadoopRdfIORegistry}
+ */
+public class TestHadoopRdfIORegistry {
+
+    private void testLang(Lang lang, boolean triples, boolean quads, boolean writesSupported) {
+        Assert.assertEquals(triples, HadoopRdfIORegistry.hasTriplesReader(lang));
+        Assert.assertEquals(quads, HadoopRdfIORegistry.hasQuadReader(lang));
+
+        // Some formats may be asymmetric
+        if (writesSupported) {
+            Assert.assertEquals(triples, HadoopRdfIORegistry.hasTriplesWriter(lang));
+            Assert.assertEquals(quads, HadoopRdfIORegistry.hasQuadWriter(lang));
+        } else {
+            Assert.assertFalse(HadoopRdfIORegistry.hasTriplesWriter(lang));
+            Assert.assertFalse(HadoopRdfIORegistry.hasQuadWriter(lang));
+        }
+
+        if (triples) {
+            // Check that triples are supported
+            RecordReader<LongWritable, TripleWritable> tripleReader;
+            try {
+                tripleReader = HadoopRdfIORegistry.createTripleReader(lang);
+                Assert.assertNotNull(tripleReader);
+            } catch (IOException e) {
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " can read triples but fails to produce a triple reader when asked: " + e.getMessage());
+            }
+
+            if (writesSupported) {
+                RecordWriter<NullWritable, TripleWritable> tripleWriter;
+                try {
+                    tripleWriter = HadoopRdfIORegistry.createTripleWriter(lang, new StringWriter(), new Configuration(
+                            false));
+                    Assert.assertNotNull(tripleWriter);
+                } catch (IOException e) {
+                    Assert.fail("Registry indicates that " + lang.getName()
+                            + " can write triples but fails to produce a triple writer when asked: " + e.getMessage());
+                }
+            }
+        } else {
+            // Check that triples are not supported
+            try {
+                HadoopRdfIORegistry.createTripleReader(lang);
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " cannot read triples but produced a triple reader when asked (error was expected)");
+            } catch (IOException e) {
+                // This is expected
+            }
+            try {
+                HadoopRdfIORegistry.createTripleWriter(lang, new StringWriter(), new Configuration(false));
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " cannot write triples but produced a triple write when asked (error was expected)");
+            } catch (IOException e) {
+                // This is expected
+            }
+        }
+
+        if (quads) {
+            // Check that quads are supported
+            RecordReader<LongWritable, QuadWritable> quadReader;
+            try {
+                quadReader = HadoopRdfIORegistry.createQuadReader(lang);
+                Assert.assertNotNull(quadReader);
+            } catch (IOException e) {
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " can read quads but fails to produce a quad reader when asked: " + e.getMessage());
+            }
+
+            if (writesSupported) {
+                RecordWriter<NullWritable, QuadWritable> quadWriter;
+                try {
+                    quadWriter = HadoopRdfIORegistry.createQuadWriter(lang, new StringWriter(),
+                            new Configuration(false));
+                    Assert.assertNotNull(quadWriter);
+                } catch (IOException e) {
+                    Assert.fail("Registry indicates that " + lang.getName()
+                            + " can write quads but fails to produce a triple writer when asked: " + e.getMessage());
+                }
+            }
+        } else {
+            try {
+                HadoopRdfIORegistry.createQuadReader(lang);
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " cannot read quads but produced a quad reader when asked (error was expected)");
+            } catch (IOException e) {
+                // This is expected
+            }
+            try {
+                HadoopRdfIORegistry.createQuadWriter(lang, new StringWriter(), new Configuration(false));
+                Assert.fail("Registry indicates that " + lang.getName()
+                        + " cannot write quads but produced a quad writer when asked (error was expected)");
+            } catch (IOException e) {
+                // This is expected
+            }
+        }
+    }
+
+    @Test
+    public void json_ld_registered() {
+        testLang(Lang.JSONLD, true, true, true);
+    }
+
+    @Test
+    public void nquads_registered() {
+        testLang(Lang.NQUADS, false, true, true);
+        testLang(Lang.NQ, false, true, true);
+    }
+
+    @Test
+    public void ntriples_registered() {
+        testLang(Lang.NTRIPLES, true, false, true);
+        testLang(Lang.NT, true, false, true);
+    }
+
+    @Test
+    public void rdf_json_registered() {
+        testLang(Lang.RDFJSON, true, false, true);
+    }
+
+    @Test
+    public void rdf_xml_registered() {
+        testLang(Lang.RDFXML, true, false, true);
+    }
+
+    @Test
+    public void rdf_thrift_registered() {
+        testLang(RDFLanguages.THRIFT, true, true, true);
+    }
+
+    @Test
+    public void trig_registered() {
+        testLang(Lang.TRIG, false, true, true);
+    }
+
+    @Test
+    public void trix_registered() {
+        testLang(Lang.TRIX, false, true, true);
+    }
+
+    @Test
+    public void turtle_registered() {
+        testLang(Lang.TURTLE, true, false, true);
+        testLang(Lang.TTL, true, false, true);
+        testLang(Lang.N3, true, false, true);
+    }
+
+    @Test
+    public void unregistered() {
+        testLang(Lang.RDFNULL, false, false, true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-stats/hadoop-job.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-stats/hadoop-job.xml b/jena-hadoop-rdf/jena-elephas-stats/hadoop-job.xml
new file mode 100644
index 0000000..de72645
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-stats/hadoop-job.xml
@@ -0,0 +1,46 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<assembly>
+  <id>hadoop-job</id>
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <dependencySets>
+    <dependencySet>
+      <unpack>false</unpack>
+      <scope>runtime</scope>
+      <outputDirectory>lib</outputDirectory>
+      <excludes>
+        <exclude>${groupId}:${artifactId}</exclude>
+      </excludes>
+    </dependencySet>
+    <dependencySet>
+      <unpack>true</unpack>
+      <includes>
+        <include>${groupId}:${artifactId}</include>
+      </includes>
+    </dependencySet>
+  </dependencySets>
+  <fileSets>
+    <fileSet>
+      <directory>${basedir}/target/test-classes</directory>
+      <outputDirectory>/</outputDirectory>
+    </fileSet>
+ </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/jena-elephas-stats/pom.xml
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/jena-elephas-stats/pom.xml b/jena-hadoop-rdf/jena-elephas-stats/pom.xml
new file mode 100644
index 0000000..899d612
--- /dev/null
+++ b/jena-hadoop-rdf/jena-elephas-stats/pom.xml
@@ -0,0 +1,103 @@
+<!--
+   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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<groupId>org.apache.jena</groupId>
+		<artifactId>jena-elephas</artifactId>
+		<version>0.9.0-SNAPSHOT</version>
+	</parent>
+	<artifactId>jena-elephas-stats</artifactId>
+	<name>Apache Jena - RDF Tools for Hadoop - Statistics Demo App</name>
+	<description>A demo application that can be run on Hadoop to produce a statistical analysis on arbitrary RDF inputs</description>
+
+	<dependencies>
+		<!-- Internal Project Dependencies -->
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-hadoop-rdf-io</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-hadoop-rdf-mapreduce</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<!-- CLI related Dependencies -->
+		<dependency>
+			<groupId>io.airlift</groupId>
+			<artifactId>airline</artifactId>
+			<version>0.6</version>
+		</dependency>
+
+		<!-- Hadoop Dependencies -->
+		<!-- Note these will be provided on the Hadoop cluster hence the provided 
+			scope -->
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test Dependencies -->
+		<dependency>
+			<groupId>org.apache.jena</groupId>
+			<artifactId>jena-hadoop-rdf-mapreduce</artifactId>
+			<version>${project.version}</version>
+			<classifier>tests</classifier>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.mrunit</groupId>
+			<artifactId>mrunit</artifactId>
+			<scope>test</scope>
+			<classifier>hadoop2</classifier>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<!-- Assembly plugin is used to produce the runnable Hadoop JAR with all 
+				dependencies contained therein -->
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<configuration>
+					<descriptors>
+						<descriptor>hadoop-job.xml</descriptor>
+					</descriptors>
+				</configuration>
+				<executions>
+					<execution>
+						<id>make-assembly</id>
+						<phase>package</phase>
+						<goals>
+							<goal>single</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>
\ No newline at end of file


[19/52] [abbrv] jena git commit: Rebrand to Jena Elephas per community vote

Posted by rv...@apache.org.
http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
deleted file mode 100644
index b3fb377..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/turtle/TurtleReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.readers.turtle;
-
-import org.apache.jena.hadoop.rdf.io.input.readers.AbstractWholeFileTripleReader;
-import org.apache.jena.riot.Lang;
-
-/**
- * A record reader for Turtle files
- * 
- * 
- * 
- */
-public class TurtleReader extends AbstractWholeFileTripleReader {
-
-    @Override
-    protected Lang getRdfLanguage() {
-        return Lang.TURTLE;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
deleted file mode 100644
index f75542a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftQuadInputFormat.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.thrift;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftQuadReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-public class ThriftQuadInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new ThriftQuadReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
deleted file mode 100644
index b60380d..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/thrift/ThriftTripleInputFormat.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.thrift;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.thrift.ThriftTripleReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-public class ThriftTripleInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new ThriftTripleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
deleted file mode 100644
index 0b36e93..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trig/TriGInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.trig;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.trig.TriGReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-
-/**
- * Input format for TriG
- * 
- * 
- * 
- */
-public class TriGInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TriGReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
deleted file mode 100644
index 723c5c3..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/trix/TriXInputFormat.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.trix;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.trix.TriXReader;
-import org.apache.jena.hadoop.rdf.types.QuadWritable;
-
-/**
- * Input format for TriX
- */
-public class TriXInputFormat extends AbstractWholeFileInputFormat<LongWritable, QuadWritable> {
-
-    @Override
-    public RecordReader<LongWritable, QuadWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TriXReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
deleted file mode 100644
index c7771b6..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/turtle/TurtleInputFormat.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.turtle;
-
-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.AbstractWholeFileInputFormat;
-import org.apache.jena.hadoop.rdf.io.input.readers.turtle.TurtleReader;
-import org.apache.jena.hadoop.rdf.types.TripleWritable;
-
-
-/**
- * Turtle input format
- * 
- * 
- * 
- */
-public class TurtleInputFormat extends AbstractWholeFileInputFormat<LongWritable, TripleWritable> {
-
-    @Override
-    public RecordReader<LongWritable, TripleWritable> createRecordReader(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        return new TurtleReader();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
deleted file mode 100644
index a9e692e..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/BlockInputStream.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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.util;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * A block input stream which can is a wrapper around another input stream which
- * restricts reading to a specific number of bytes and can report the number of
- * bytes read
- * <p>
- * The class assumes that the underlying input stream has already been seeked to
- * the appropriate start point
- * </p>
- * 
- * 
- * 
- */
-public final class BlockInputStream extends TrackedInputStream {
-
-    private long limit = Long.MAX_VALUE;
-
-    /**
-     * Creates a new tracked input stream
-     * 
-     * @param input
-     *            Input stream to track
-     * @param limit
-     *            Maximum number of bytes to read from the stream
-     */
-    public BlockInputStream(InputStream input, long limit) {
-        super(input);
-        if (limit < 0)
-            throw new IllegalArgumentException("limit must be >= 0");
-        this.limit = limit;
-    }
-
-    @Override
-    public int read() throws IOException {
-        if (this.bytesRead >= this.limit) {
-            return -1;
-        }
-        return super.read();
-    }
-
-    @Override
-    public int available() throws IOException {
-        if (this.bytesRead >= this.limit) {
-            return 0;
-        }
-        return super.available();
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        if (len == 0) {
-            return 0;
-        } else if (this.bytesRead >= this.limit) {
-            return -1;
-        } else if (len > this.limit - this.bytesRead) {
-            len = (int) (this.limit - this.bytesRead);
-        }
-        return super.read(b, off, len);
-    }
-
-    @Override
-    public long skip(long n) throws IOException {
-        if (n == 0) {
-            return 0;
-        } else if (this.bytesRead >= this.limit) {
-            return -1;
-        } else if (n > this.limit - this.bytesRead) {
-            n = this.limit - this.bytesRead;
-        }
-        return super.skip(n);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/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
deleted file mode 100644
index 372b22c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/RdfIOUtils.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.util;
-
-import java.util.UUID;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.jena.hadoop.rdf.io.RdfIOConstants;
-import org.apache.jena.riot.lang.LabelToNode;
-import org.apache.jena.riot.system.ErrorHandlerFactory;
-import org.apache.jena.riot.system.IRIResolver;
-import org.apache.jena.riot.system.ParserProfile;
-import org.apache.jena.riot.system.ParserProfileBase;
-import org.apache.jena.riot.system.Prologue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * RDF IO utility functions
- * 
- * 
- * 
- */
-public class RdfIOUtils {
-    private static final Logger LOGGER = LoggerFactory.getLogger(RdfIOUtils.class);
-
-    /**
-     * Private constructor prevents instantiation
-     */
-    private RdfIOUtils() {
-    }
-
-    /**
-     * Creates a parser profile for the given job context
-     * 
-     * @param context
-     *            Context
-     * @param path
-     *            File path
-     * @return Parser profile
-     */
-    public static ParserProfile createParserProfile(JobContext context, Path path) {
-        Prologue prologue = new Prologue(null, IRIResolver.createNoResolve());
-        UUID seed = RdfIOUtils.getSeed(context, path);
-        LabelToNode labelMapping = LabelToNode.createScopeByDocumentHash(seed);
-        return new ParserProfileBase(prologue, ErrorHandlerFactory.errorHandlerStd, labelMapping);
-    }
-
-    /**
-     * Selects a seed for use in generating blank node identifiers
-     * 
-     * @param context
-     *            Job Context
-     * @param path
-     *            File path
-     * @return Seed
-     */
-    public static UUID getSeed(JobContext context, Path path) {
-        // This is to ensure that blank node allocation policy is constant when
-        // subsequent MapReduce jobs need that
-        String jobId = context.getJobID().toString();
-        if (jobId == null) {
-            jobId = String.valueOf(System.currentTimeMillis());
-            LOGGER.warn(
-                    "Job ID was not set, using current milliseconds of {}. Sequence of MapReduce jobs must carefully handle blank nodes.",
-                    jobId);
-        }
-
-        if (!context.getConfiguration().getBoolean(RdfIOConstants.GLOBAL_BNODE_IDENTITY, false)) {
-            // Using normal file scoped blank node allocation
-            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());
-        } else {
-            // Using globally scoped blank node allocation
-            LOGGER.warn(
-                    "Using globally scoped blank node allocation policy from Job Details (ID={}) - this is unsafe if your RDF inputs did not originate from a previous job",
-                    jobId);
-            
-            return new UUID(jobId.hashCode(), 0);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
deleted file mode 100644
index 92e2df5..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackableInputStream.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.util;
-
-import java.io.InputStream;
-
-/**
- * An input stream that tracks the number of bytes read
- * 
- * 
- * 
- */
-public abstract class TrackableInputStream extends InputStream {
-
-    /**
-     * Gets the number of bytes read
-     * 
-     * @return Number of bytes read
-     */
-    public abstract long getBytesRead();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
deleted file mode 100644
index e51a866..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedInputStream.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.util;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * A tracked input stream which can is a wrapper around another input stream and
- * can report the number of bytes read
- * 
- * 
- * 
- */
-public class TrackedInputStream extends TrackableInputStream {
-
-    protected InputStream input;
-    protected long bytesRead = 0, lastMark;
-
-    /**
-     * Creates a new tracked input stream
-     * 
-     * @param input
-     *            Input stream to track
-     */
-    public TrackedInputStream(InputStream input) {
-        if (input == null)
-            throw new NullPointerException("Input cannot be null");
-        this.input = input;
-    }
-
-    @Override
-    public int read() throws IOException {
-        int read = this.input.read();
-        if (read >= 0)
-            this.bytesRead++;
-        return read;
-    }
-
-    @Override
-    public long getBytesRead() {
-        return this.bytesRead;
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.input.close();
-    }
-
-    @Override
-    public int available() throws IOException {
-        return this.input.available();
-    }
-
-    @Override
-    public synchronized void mark(int readlimit) {
-        this.input.mark(readlimit);
-        this.lastMark = this.bytesRead;
-    }
-
-    @Override
-    public boolean markSupported() {
-        return this.input.markSupported();
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        if (len == 0) return 0;
-        int read = this.input.read(b, off, len);
-        if (read > 0)
-            this.bytesRead += read;
-        return read;
-    }
-
-    @Override
-    public int read(byte[] b) throws IOException {
-        return this.read(b, 0, b.length);
-    }
-
-    @Override
-    public synchronized void reset() throws IOException {
-        this.input.reset();
-        this.bytesRead = this.lastMark;
-    }
-
-    @Override
-    public long skip(long n) throws IOException {
-        if (n == 0)
-            return 0;
-        long skipped = 0;
-        byte[] buffer = new byte[16];
-        int readSize = Math.min(buffer.length, n > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) n);
-        int read;
-        do {
-            if (n - skipped > readSize) {
-                read = this.input.read(buffer, 0, readSize);
-            } else {
-                read = this.input.read(buffer, 0, (int) (n - skipped));
-            }
-            if (read > 0) {
-                this.bytesRead += read;
-                skipped += read;
-            }
-        } while (skipped < n && read >= 0);
-
-        return skipped;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
deleted file mode 100644
index 845c709..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedQuadsStream.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A tracked piped quads stream
- * 
- * 
- * 
- */
-public class TrackedPipedQuadsStream extends TrackedPipedRDFStream<Quad> {
-
-    /**
-     * Creates a new stream
-     * 
-     * @param sink
-     *            Sink
-     * @param input
-     *            Input stream
-     */
-    public TrackedPipedQuadsStream(PipedRDFIterator<Quad> sink, TrackableInputStream input) {
-        super(sink, input);
-    }
-
-    @Override
-    public void triple(Triple triple) {
-        // Triples are discarded
-    }
-
-    @Override
-    public void quad(Quad quad) {
-        this.receive(quad);
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java
deleted file mode 100644
index 6e910be..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedRDFStream.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.util;
-
-import java.util.LinkedList;
-import java.util.Queue;
-
-import org.apache.jena.riot.lang.PipedRDFIterator;
-import org.apache.jena.riot.lang.PipedRDFStream;
-
-/**
- * A tracked piped RDF stream
- * 
- * 
- * 
- * @param <T>
- *            Type corresponding to a supported RDF primitive
- */
-public abstract class TrackedPipedRDFStream<T> extends PipedRDFStream<T> {
-
-    private TrackableInputStream input;
-    private Queue<Long> positions = new LinkedList<Long>();
-
-    protected TrackedPipedRDFStream(PipedRDFIterator<T> sink, TrackableInputStream input) {
-        super(sink);
-        this.input = input;
-    }
-
-    @Override
-    protected void receive(T t) {
-        // Track positions the input stream is at as we receive inputs
-        synchronized (this.positions) {
-            this.positions.add(this.input.getBytesRead());
-        }
-        super.receive(t);
-    }
-
-    /**
-     * Gets the next position
-     * 
-     * @return Position
-     */
-    public Long getPosition() {
-        synchronized (this.positions) {
-            return this.positions.poll();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java
deleted file mode 100644
index 2040c4f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/util/TrackedPipedTriplesStream.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.jena.riot.lang.PipedRDFIterator;
-
-import com.hp.hpl.jena.graph.Triple;
-import com.hp.hpl.jena.sparql.core.Quad;
-
-/**
- * A tracked piped triples stream
- * 
- * 
- * 
- */
-public class TrackedPipedTriplesStream extends TrackedPipedRDFStream<Triple> {
-
-    /**
-     * Creates a tracked triples stream
-     * 
-     * @param sink
-     *            Sink
-     * @param input
-     *            Input stream
-     */
-    public TrackedPipedTriplesStream(PipedRDFIterator<Triple> sink, TrackableInputStream input) {
-        super(sink, input);
-    }
-
-    @Override
-    public void triple(Triple triple) {
-        receive(triple);
-    }
-
-    @Override
-    public void quad(Quad quad) {
-        // Quads are discarded
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
deleted file mode 100644
index 02fbf9c..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractBatchedNodeTupleOutputFormat.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.io.RdfIOConstants;
-import org.apache.jena.hadoop.rdf.io.output.writers.AbstractBatchedNodeTupleWriter;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-
-
-/**
- * Abstract output format for formats that use a
- * {@link AbstractBatchedNodeTupleWriter} as their writer
- * 
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TTuple>
- *            Tuple type
- * @param <TValue>
- *            Writable tuple type i.e. the value type
- */
-public abstract class AbstractBatchedNodeTupleOutputFormat<TKey, TTuple, TValue extends AbstractNodeTupleWritable<TTuple>> extends
-        AbstractNodeTupleOutputFormat<TKey, TTuple, TValue> {
-
-    @Override
-    protected RecordWriter<TKey, TValue> getRecordWriter(Writer writer, Configuration config, Path outputPath) {
-        long batchSize = config.getLong(RdfIOConstants.OUTPUT_BATCH_SIZE, RdfIOConstants.DEFAULT_OUTPUT_BATCH_SIZE);
-        return this.getRecordWriter(writer, batchSize);
-    }
-    
-    protected abstract RecordWriter<TKey, TValue> getRecordWriter(Writer writer, long batchSize);
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
deleted file mode 100644
index cfc98bd..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeOutputFormat.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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.OutputStreamWriter;
-import java.io.Writer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.jena.hadoop.rdf.types.NodeWritable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Abstract output format which takes pairs with Node keys and arbitrary values
- * and writes them as a simple line based text file
- * 
- * 
- * 
- * @param <TValue> Value type
- */
-public abstract class AbstractNodeOutputFormat<TValue> extends FileOutputFormat<NodeWritable, TValue> {
-    
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeOutputFormat.class);
-
-    @Override
-    public RecordWriter<NodeWritable, TValue> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
-        Configuration config = context.getConfiguration();
-        boolean isCompressed = getCompressOutput(context);
-        CompressionCodec codec = null;
-        String extension = this.getFileExtension();
-        if (isCompressed) {
-            Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(context, GzipCodec.class);
-            codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, config);
-            extension += codec.getDefaultExtension();
-        }
-        Path file = getDefaultWorkFile(context, extension);
-        LOG.info("Writing output to file " + file);
-        FileSystem fs = file.getFileSystem(config);
-        if (!isCompressed) {
-            FSDataOutputStream fileOut = fs.create(file, false);
-            return this.getRecordWriter(new OutputStreamWriter(fileOut), config);
-        } else {
-            FSDataOutputStream fileOut = fs.create(file, false);
-            return this.getRecordWriter(new OutputStreamWriter(codec.createOutputStream(fileOut)), config);
-        }
-    }
-
-    /**
-     * Gets the file extension to use for output
-     * 
-     * @return File extension including the '.'
-     */
-    protected String getFileExtension() {
-        return ".nodes";
-    }
-
-    /**
-     * Gets the record writer to use
-     * 
-     * @param writer
-     *            Writer to write output to
-     * @param config
-     *            Configuration
-     * @return Record writer
-     */
-    protected abstract RecordWriter<NodeWritable, TValue> getRecordWriter(Writer writer, Configuration config);
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
deleted file mode 100644
index c4a34f5..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractNodeTupleOutputFormat.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.OutputStreamWriter;
-import java.io.Writer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.jena.hadoop.rdf.types.AbstractNodeTupleWritable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An abstract implementation of an output format for line based tuple formats
- * where the key is ignored and only the tuple values will be output
- * 
- * 
- * @param <TKey>
- *            Key type
- * @param <TValue>
- *            Tuple value type
- * @param <T>
- *            Writable node tuple type
- * 
- */
-public abstract class AbstractNodeTupleOutputFormat<TKey, TValue, T extends AbstractNodeTupleWritable<TValue>> extends
-        FileOutputFormat<TKey, T> {
-
-    private static final Logger LOG = LoggerFactory.getLogger(AbstractNodeTupleOutputFormat.class);
-
-    @Override
-    public RecordWriter<TKey, T> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
-        Configuration config = context.getConfiguration();
-        boolean isCompressed = getCompressOutput(context);
-        CompressionCodec codec = null;
-        
-        // Build the output file path
-        String extension = this.getFileExtension();
-        if (isCompressed) {
-            // Add compression extension if applicable
-            Class<? extends CompressionCodec> codecClass = getOutputCompressorClass(context, GzipCodec.class);
-            codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, config);
-            extension += codec.getDefaultExtension();
-        }
-        Path file = getDefaultWorkFile(context, extension);
-        LOG.info("Writing output to file " + file);
-        
-        // Open the file appropriately and create a record writer for it
-        FileSystem fs = file.getFileSystem(config);
-        if (!isCompressed) {
-            FSDataOutputStream fileOut = fs.create(file, false);
-            return this.getRecordWriter(new OutputStreamWriter(fileOut), config, file);
-        } else {
-            FSDataOutputStream fileOut = fs.create(file, false);
-            return this.getRecordWriter(new OutputStreamWriter(codec.createOutputStream(fileOut)), config, file);
-        }
-    }
-
-    /**
-     * Gets the file extension to use for output
-     * 
-     * @return File extension including the '.'
-     */
-    protected abstract String getFileExtension();
-
-    /**
-     * Gets the record writer to use
-     * 
-     * @param writer
-     *            Writer to write output to
-     * @param config
-     *            Configuration
-     * @param outputPath
-     *            Output path being written to
-     * @return Record writer
-     * @throws IOException
-     *             May be thrown if a record writer cannot be obtained for any
-     *             reason
-     */
-    protected abstract RecordWriter<TKey, T> getRecordWriter(Writer writer, Configuration config, Path outputPath)
-            throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a6c0fefc/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java
deleted file mode 100644
index 30999ae..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/AbstractStreamRdfNodeTupleOutputFormat.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java
deleted file mode 100644
index cc9fe2f..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/QuadsOutputFormat.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java
deleted file mode 100644
index 3eaf0d7..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOrQuadsOutputFormat.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java
deleted file mode 100644
index d9d4189..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/TriplesOutputFormat.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java
deleted file mode 100644
index 8f4797a..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDQuadOutputFormat.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java
deleted file mode 100644
index a8cbeac..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/jsonld/JsonLDTripleOutputFormat.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java
deleted file mode 100644
index a8ab017..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/nquads/NQuadsOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java
deleted file mode 100644
index 56935bb..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesNodeOutputFormat.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java
deleted file mode 100644
index 51b9b75..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/ntriples/NTriplesOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java
deleted file mode 100644
index e5fa114..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfjson/RdfJsonOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java
deleted file mode 100644
index 6c9a9ea..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/rdfxml/RdfXmlOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java
deleted file mode 100644
index bd07bff..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftQuadOutputFormat.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java
deleted file mode 100644
index 73e40bc..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/thrift/ThriftTripleOutputFormat.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java
deleted file mode 100644
index 6f33e29..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/BatchedTriGOutputFormat.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java
----------------------------------------------------------------------
diff --git a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java b/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java
deleted file mode 100644
index 0047095..0000000
--- a/jena-hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/output/trig/TriGOutputFormat.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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";
-    }
-
-}