You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by rv...@apache.org on 2014/04/08 15:42:21 UTC

svn commit: r1585723 [2/4] - in /jena/Experimental/hadoop-rdf: hadoop-rdf-common/src/test/java/com/ hadoop-rdf-common/src/test/java/org/ hadoop-rdf-common/src/test/java/org/apache/ hadoop-rdf-common/src/test/java/org/apache/jena/ hadoop-rdf-common/src/...

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedNodeTupleReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,335 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.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.lang.PipedRDFIterator;
+import org.apache.jena.riot.lang.PipedRDFStream;
+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>
+ * 
+ * @author rvesse
+ * 
+ * @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(AbstractLineBasedNodeTupleReader.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);
+
+        // 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);
+        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage());
+        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) {
+        return new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    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)
+                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;
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedQuadReader.java Tue Apr  8 13:42:18 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ */
+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);
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractBlockBasedTripleReader.java Tue Apr  8 13:42:18 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ */
+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);
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedNodeTupleReader.java Tue Apr  8 13:42:18 2014
@@ -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>
+ * 
+ * @author rvesse
+ * 
+ * @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);
+
+        // 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 " + newSize + " at position " + (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("Bad tuple at position " + (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

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedQuadReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,50 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package org.apache.jena.hadoop.rdf.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
+ * 
+ * @author rvesse
+ * 
+ */
+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);
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractLineBasedTripleReader.java Tue Apr  8 13:42:18 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ */
+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);
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractRdfReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,110 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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
+ * 
+ * @author rvesse
+ * 
+ * @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();
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileNodeTupleReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,318 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.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.lang.PipedRDFIterator;
+import org.apache.jena.riot.lang.PipedRDFStream;
+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>
+ * 
+ * @author rvesse
+ * 
+ * @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);
+
+        // 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);
+        Runnable parserRunnable = this.createRunnable(this, this.input, stream, this.getRdfLanguage());
+        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) {
+        return new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    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()) {
+                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)
+                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;
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileQuadReader.java Tue Apr  8 13:42:18 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ */
+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);
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/AbstractWholeFileTripleReader.java Tue Apr  8 13:42:18 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ */
+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);
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/BlockedNQuadsReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/BlockedNQuadsReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/BlockedNQuadsReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/BlockedNQuadsReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,44 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package org.apache.jena.hadoop.rdf.io.input.readers;
+
+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>
+ * 
+ * @author rvesse
+ * 
+ */
+public class BlockedNQuadsReader extends AbstractBlockBasedQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NQUADS;
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/BlockedNTriplesReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/BlockedNTriplesReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/BlockedNTriplesReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/BlockedNTriplesReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,44 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package org.apache.jena.hadoop.rdf.io.input.readers;
+
+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>
+ * 
+ * @author rvesse
+ * 
+ */
+public class BlockedNTriplesReader extends AbstractBlockBasedTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.NTRIPLES;
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/NQuadsReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/NQuadsReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/NQuadsReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/NQuadsReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,48 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package org.apache.jena.hadoop.rdf.io.input.readers;
+
+import java.util.Iterator;
+
+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
+ * 
+ * @author rvesse
+ * 
+ */
+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);
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/NTriplesReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/NTriplesReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/NTriplesReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/NTriplesReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,47 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package org.apache.jena.hadoop.rdf.io.input.readers;
+
+import java.util.Iterator;
+
+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
+ * 
+ * @author rvesse
+ * 
+ */
+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);
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/QuadsReader.java Tue Apr  8 13:42:18 2014
@@ -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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+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
+ * 
+ * @author rvesse
+ * 
+ */
+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?");
+
+        if (lang.equals(Lang.NQ) || lang.equals(Lang.NQUADS)) {
+            return new WholeFileNQuadsReader();
+        } else if (lang.equals(Lang.TRIG)) {
+            return new TriGReader();
+        }
+        throw new IOException(lang.getLabel() + " has no associated RecordReader implementation");
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/RdfJsonReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/RdfJsonReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/RdfJsonReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/RdfJsonReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,36 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.riot.Lang;
+
+/**
+ * A record reader for RDF/JSON files
+ * 
+ * @author rvesse
+ * 
+ */
+public class RdfJsonReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFJSON;
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/RdfXmlReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/RdfXmlReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/RdfXmlReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/RdfXmlReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,36 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.riot.Lang;
+
+/**
+ * A record reader for RDF/XML files
+ * 
+ * @author rvesse
+ * 
+ */
+public class RdfXmlReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.RDFXML;
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriGReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriGReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriGReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriGReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,36 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.riot.Lang;
+
+/**
+ * A record reader for TriG files
+ * 
+ * @author rvesse
+ * 
+ */
+public class TriGReader extends AbstractWholeFileQuadReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TRIG;
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesOrQuadsReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,77 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.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
+ * 
+ * @author rvesse
+ * 
+ */
+@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 (lang.equals(Lang.NQ) || lang.equals(Lang.NQUADS)) {
+            return new WholeFileNQuadsReader();
+        } else if (lang.equals(Lang.TRIG)) {
+            return new TriGReader();
+        } else if (lang.equals(Lang.NTRIPLES) || lang.equals(Lang.NT)) {
+            return new TriplesToQuadsReader(new WholeFileNTriplesReader());
+        } else if (lang.equals(Lang.TTL) || lang.equals(Lang.TURTLE) || lang.equals(Lang.N3)) {
+            return new TriplesToQuadsReader(new TurtleReader());
+        } else if (lang.equals(Lang.RDFXML)) {
+            return new TriplesToQuadsReader(new RdfXmlReader());
+        } else if (lang.equals(Lang.RDFJSON)) {
+            return new TriplesToQuadsReader(new RdfJsonReader());
+        }
+        throw new IOException(lang.getLabel() + " has no associated RecordReader implementation");
+    }
+
+    /**
+     * 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;
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesReader.java Tue Apr  8 13:42:18 2014
@@ -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.readers;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.RecordReader;
+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
+ * @author rvesse
+ *
+ */
+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?");
+
+        if (lang.equals(Lang.NTRIPLES) || lang.equals(Lang.NT)) {
+            return new WholeFileNTriplesReader();
+        } else if (lang.equals(Lang.TTL) || lang.equals(Lang.TURTLE) || lang.equals(Lang.N3)) {
+            return new TurtleReader();
+        } else if (lang.equals(Lang.RDFXML)) {
+            return new RdfXmlReader();
+        } else if (lang.equals(Lang.RDFJSON)) {
+            return new RdfJsonReader();
+        }
+        throw new IOException(lang.getLabel() + " has no associated RecordReader implementation");
+    }
+
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TriplesToQuadsReader.java Tue Apr  8 13:42:18 2014
@@ -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
+ * 
+ * @author rvesse
+ * 
+ */
+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();
+    }
+}

Added: jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TurtleReader.java
URL: http://svn.apache.org/viewvc/jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TurtleReader.java?rev=1585723&view=auto
==============================================================================
--- jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TurtleReader.java (added)
+++ jena/Experimental/hadoop-rdf/hadoop-rdf-io/src/main/java/org/apache/jena/hadoop/rdf/io/input/readers/TurtleReader.java Tue Apr  8 13:42:18 2014
@@ -0,0 +1,36 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+    
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express 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.riot.Lang;
+
+/**
+ * A record reader for Turtle files
+ * 
+ * @author rvesse
+ * 
+ */
+public class TurtleReader extends AbstractWholeFileTripleReader {
+
+    @Override
+    protected Lang getRdfLanguage() {
+        return Lang.TURTLE;
+    }
+
+}