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

svn commit: r1154204 - in /incubator/jena/Jena2/ARQ/trunk: src-dev/dev/ src/com/hp/hpl/jena/sparql/engine/binding/ src/org/openjena/atlas/json/io/parser/ src/org/openjena/riot/lang/ src/org/openjena/riot/tokens/

Author: andy
Date: Fri Aug  5 13:21:50 2011
New Revision: 1154204

URL: http://svn.apache.org/viewvc?rev=1154204&view=rev
Log:
Split Langbase into an engine and an implementation - enable inheriting the engine without public methods.

Added:
    incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangEngine.java   (with props)
Removed:
    incubator/jena/Jena2/ARQ/trunk/src-dev/dev/LangNTriples2.java
    incubator/jena/Jena2/ARQ/trunk/src-dev/dev/LangNTriples3.java
    incubator/jena/Jena2/ARQ/trunk/src-dev/dev/LangNTriples4.java
Modified:
    incubator/jena/Jena2/ARQ/trunk/src-dev/dev/RunARQ.java
    incubator/jena/Jena2/ARQ/trunk/src/com/hp/hpl/jena/sparql/engine/binding/BindingInputStream.java
    incubator/jena/Jena2/ARQ/trunk/src/org/openjena/atlas/json/io/parser/TokenizerJSON.java
    incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangBase.java
    incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangNTuple.java
    incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangTurtleBase.java
    incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/Tokenizer.java
    incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerBytes.java
    incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerText.java
    incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerWrapper.java

Modified: incubator/jena/Jena2/ARQ/trunk/src-dev/dev/RunARQ.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src-dev/dev/RunARQ.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src-dev/dev/RunARQ.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src-dev/dev/RunARQ.java Fri Aug  5 13:21:50 2011
@@ -12,6 +12,7 @@ import java.io.PrintWriter ;
 import java.io.Writer ;
 import java.util.Iterator ;
 import java.util.NoSuchElementException ;
+import java.util.PriorityQueue ;
 import java.util.concurrent.ArrayBlockingQueue ;
 import java.util.concurrent.BlockingQueue ;
 import java.util.concurrent.ExecutorService ;
@@ -128,118 +129,7 @@ public class RunARQ
     
     public static void main(String[] argv) throws Exception
     {
-        JsonObject result = new JsonObject();
-        result.put("progress", 100);
-        result.put("state", "Running");
-        JSON.write(System.out, result);
-        System.out.println() ;
-        System.out.flush();
-        exit(0) ;
-        
-        //arq.sparql.main("--data=D.ttl", "--query=Q.rq") ;
-        //arq.qparse.main("--query=Q.rq", "--print=opt") ;
-        String DIR = "/home/afs/W3C/SPARQL-docs/tests/data-sparql11/subquery/" ;
-        //sparql --strict --namedGraph data-sq.ttl --query graph-subquery-1.rq 
-        arq.sparql.main("--namedGraph="+DIR+"sq01.rdf",
-                        "--strict", 
-                        "--query="+DIR+"sq03.rq") ;
-        
-        exit(0) ;
-        
-        
-        {
-            Query query = QueryFactory.create("SELECT (count(*) AS ?C) { { SELECT ?s { ?s ?p ?o } LIMIT 100000000} }") ;
-            QueryEngineHTTP qExec = QueryExecutionFactory.createServiceRequest("http://dbpedia.org/sparql", query) ;
-            qExec.addParam("timeout", "10000") ;
-            ResultSetFormatter.out(qExec.execSelect()) ;
-            exit(0) ;
-            
-            
-        }
-        
-        
-        
-        riotcmd.infer.main("--rdfs=D.ttl", "D.ttl") ;
-        exit(0) ;
-        
-        PropertyFunctionRegistry.get().remove(RDFS.getURI()) ;
-        ParserProfile p = RiotLib.profile(Lang.NTRIPLES, null) ;
-        {
-            Writer w = new PrintWriter(System.out) ; 
-            OutputLangUtils.output(w, Node.ANY, null, null) ;
-            w.write("\n") ;
-            w.flush();
-            
-            
-            Tokenizer tokenizer = TokenizerFactory.makeTokenizerString("ANY <123> KW.") ;
-            for ( ; tokenizer.hasNext() ; )
-            {
-                Token t = tokenizer.next() ;
-                System.out.println(t) ;
-                if ( t.isNode() )
-                    System.out.println("==> "+t.asNode()) ;
-                Node n = p.create(null, t) ;
-                System.out.println("--> "+n) ;
-            }
-            exit(0) ;
-        }
-
-        Node node1 = Node_Blank.createAnon();
-        String str = NodeFmtLib.serialize(node1);
-        Tokenizer tokenizer = TokenizerFactory.makeTokenizerString(str);
-        Token token = tokenizer.next();
-        Node node2 = token.asNode();
-        System.out.println(node1) ;
-        System.out.println(node2) ;
-        exit(0) ;
-        
-        arq.qparse.main("--file=Q.arq") ; exit(0) ;
-        
-        String x = StrUtils.strjoinNL("(join",
-                                      //"  (conditional",
-                                      "  (leftjoin",
-                                      "    (bgp (triple ?s ?p1 ?o1))" ,
-                                      "    (bgp (triple ?s <foaf:knows> ?o2)))" ,
-                                      "  (table",
-                                      "    (row [?o2 :b])",
-                                      "  ))") ;
-        
-        Op op = SSE.parseOp(x) ;
-        System.out.print(op) ;
-        Op left = ((OpJoin)op).getLeft() ;
-        Op right = ((OpJoin)op).getRight() ;
-        
-        if ( false )
-        {
-            JoinClassifier.print = true ;
-            System.out.flush() ;
-            boolean b1 = JoinClassifier.isLinear(left, right) ;
-            System.out.println("Left/right: "+b1) ;
-           
-            
-            
-            System.out.println() ;
-            System.out.flush() ;
-            boolean b2 = JoinClassifier.isLinear(right, left) ;
-            System.out.println("Right/left: "+b2) ;
-            exit(0) ;
-            System.out.println() ;
-            System.out.flush() ;
-        }
-        Op op2 = Transformer.transform(new TransformJoinStrategy(ARQ.getContext()), op) ;
-        System.out.println(op2) ;
-        System.out.flush() ;
-        
-        exit(0) ;
-        
-        // -----
-        Query query = QueryFactory.read("Q.rq") ;
-        Model m = FileManager.get().loadModel("D.ttl") ;
-        //ARQ.setExecutionLogging(InfoLevel.ALL) ;
-        QueryExecution qExec = QueryExecutionFactory.create(query, m) ;
-        ResultSetFormatter.out(qExec.execSelect()) ;
-
-        exit(0) ;
+        PriorityQueue<Integer> pq = new PriorityQueue<Integer>() ;
     }
 
     public static void canoncialNodes()

Modified: incubator/jena/Jena2/ARQ/trunk/src/com/hp/hpl/jena/sparql/engine/binding/BindingInputStream.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/com/hp/hpl/jena/sparql/engine/binding/BindingInputStream.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/com/hp/hpl/jena/sparql/engine/binding/BindingInputStream.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/com/hp/hpl/jena/sparql/engine/binding/BindingInputStream.java Fri Aug  5 13:21:50 2011
@@ -29,7 +29,7 @@ import java.util.List ;
 
 import org.openjena.atlas.iterator.IteratorSlotted ;
 import org.openjena.riot.Lang ;
-import org.openjena.riot.lang.LangBase ;
+import org.openjena.riot.lang.LangEngine ;
 import org.openjena.riot.system.ParserProfile ;
 import org.openjena.riot.system.RiotLib ;
 import org.openjena.riot.tokens.Token ;
@@ -42,17 +42,27 @@ import com.hp.hpl.jena.iri.IRI ;
 import com.hp.hpl.jena.sparql.core.Var ;
 
 /** Language for reading in a steram of bindings.
- * https://cwiki.apache.org/confluence/display/JENA/BindingIO
- * 
- * VARS
- * PREFIX
+ * See <a href="https://cwiki.apache.org/confluence/display/JENA/BindingIO">BindingIO</a>
  * 
+ * <p>Summary:</p>
+ * <ul>
+ * <li>Directives:
+ *   <ul>
+ *     <li>VARS - list of variables.</li>
+ *     <li>PREFIX</li>
+ *   </ul>
+ *  </li> 
+ * <li>Lines of RDF terms (Turtle, no triple-quoted strings)</li>
+ * <li>Items on line align with last VARS declaration</li>
+ * <li>* for "same as last row"</li>
+ * <li>- for "undef"</li>
+ * </ul>
  */
-public class BindingInputStream extends LangBase<Binding>
-    implements Iterator<Binding>
+public class BindingInputStream extends LangEngine implements Iterator<Binding>
 {
-    // ?? Rework so it is not exposing all of LangBase
-
+    // In effect, multiple Inheritance.
+    // We implementation-inherit from LangEngine(no public methods) 
+    // and also IteratorTuples (redirecting calls to be object) 
     private final IteratorTuples iter ;
     
     public BindingInputStream(InputStream in)
@@ -72,7 +82,7 @@ public class BindingInputStream extends 
     
     private BindingInputStream(Tokenizer tokenizer, ParserProfile profile)
     {
-        super(tokenizer, null, profile) ;
+        super(tokenizer, profile) ;
         iter = new IteratorTuples() ;
         
         // Fixes to TokenizerText
@@ -87,12 +97,6 @@ public class BindingInputStream extends 
     }
 
     //@Override
-    public Lang getLang()
-    {
-        return null ;
-    }
-
-    //@Override
     public boolean hasNext()
     {
         return iter.hasNext() ;
@@ -108,11 +112,6 @@ public class BindingInputStream extends 
     public void remove()
     { iter.remove() ; }
 
-    @Override
-    protected void runParser()
-    {}
-    
-    // Multiple Inheritance.
     class IteratorTuples extends IteratorSlotted<Binding>
     {
         private Binding lastLine ;
@@ -139,7 +138,7 @@ public class BindingInputStream extends 
 
             int i = 0 ;
             
-            while( ! eof() && ! lookingAt(TokenType.DOT) )
+            while( ! lookingAt(TokenType.DOT) )
             {
                 if ( i >= vars.size() )
                     exception(peekToken(), "Too many items in a line.  Expected "+vars.size()) ;

Modified: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/atlas/json/io/parser/TokenizerJSON.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/org/openjena/atlas/json/io/parser/TokenizerJSON.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/org/openjena/atlas/json/io/parser/TokenizerJSON.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/org/openjena/atlas/json/io/parser/TokenizerJSON.java Fri Aug  5 13:21:50 2011
@@ -51,6 +51,11 @@ public class TokenizerJSON implements To
         return token != null ;
     }
     
+    public final boolean eof()
+    {
+        return hasNext() ;
+    }
+
     /** Move to next token */
     public final Token next()
     {
@@ -60,6 +65,13 @@ public class TokenizerJSON implements To
         token = null ;
         return t ;
     }
+
+    
+    public final Token peek()
+    {
+        if ( ! hasNext() ) return null ;
+        return token ; 
+    }
     
     //@Override
     public void remove()

Modified: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangBase.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangBase.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangBase.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangBase.java Fri Aug  5 13:21:50 2011
@@ -1,231 +1,61 @@
-/*
- * (c) Copyright 2009 Hewlett-Packard Development Company, LP
- * All rights reserved.
- * [See end of file]
- */
-
-package org.openjena.riot.lang;
-
-import static org.openjena.riot.tokens.TokenType.EOF ;
-import static org.openjena.riot.tokens.TokenType.NODE ;
-import org.openjena.atlas.AtlasException ;
-import org.openjena.atlas.event.Event ;
-import org.openjena.atlas.event.EventManager ;
-import org.openjena.atlas.iterator.PeekIterator ;
-import org.openjena.atlas.lib.Sink ;
-import org.openjena.riot.ErrorHandler ;
-import org.openjena.riot.RiotParseException ;
-import org.openjena.riot.SysRIOT ;
-import org.openjena.riot.system.ParserProfile ;
-import org.openjena.riot.tokens.Token ;
-import org.openjena.riot.tokens.TokenType ;
-import org.openjena.riot.tokens.Tokenizer ;
-
-import com.hp.hpl.jena.graph.Node ;
-
-/** Common operations for RIOT parsers */
-public abstract class LangBase<X> implements LangRIOT
-{
-    protected ParserProfile profile ;
-    protected final Tokenizer tokens ;
-    private final PeekIterator<Token> peekIter ;
-
-    protected final Sink<X> sink ; 
-    
-    protected LangBase(Tokenizer tokens,
-                       Sink<X> sink,
-                       ParserProfile profile)
-
-    {
-        //setChecker(checker) ;
-        setProfile(profile) ;
-        this.sink = sink ;
-        this.tokens = tokens ;
-        this.peekIter = new PeekIterator<Token>(tokens) ;
-    }
-     
-    //@Override
-    public ParserProfile getProfile()                     { return profile ; }
-    //@Override
-    public void setProfile(ParserProfile profile)
-    {
-        this.profile = profile ;
-    }
-    
-//    //@Override
-//    public Checker getChecker()                 { return checker ; }
-//    //@Override
-//    // Bad separation of responsibilitied :-(
-//    public void    setChecker(Checker checker)
-//    { 
-//        this.checker = checker ;
-//        if ( checker != null)
-//            this.errorHandler = checker.getHandler() ;
-//        else
-//            this.errorHandler = ErrorHandlerLib.errorHandlerNoLogging ;
-//    }
-    
-    public void parse()
-    {
-        EventManager.send(sink, new Event(SysRIOT.startRead, null)) ;
-        runParser() ;
-        sink.flush() ;
-        EventManager.send(sink, new Event(SysRIOT.finishRead, null)) ;
-        tokens.close();
-    }
-    
-    // ---- Managing tokens.
-    
-    /** Run the parser - events have been handled. */
-    protected abstract void runParser() ;
-
-    protected final Token peekToken()
-    {
-        // Avoid repeating.
-        if ( eof() ) return tokenEOF ;
-        return peekIter.peek() ;
-    }
-    
-    // Set when we get to EOF to record line/col of the EOF.
-    private Token tokenEOF = null ;
-
-    protected final boolean eof()
-    {
-        if ( tokenEOF != null )
-            return true ;
-        
-        if ( ! moreTokens() )
-        {
-            tokenEOF = new Token(tokens.getLine(), tokens.getColumn()) ;
-            tokenEOF.setType(EOF) ;
-            return true ;
-        }
-        return false ;
-    }
-
-    protected final boolean moreTokens() 
-    {
-        return peekIter.hasNext() ;
-    }
-    
-    protected final boolean lookingAt(TokenType tokenType)
-    {
-        if ( eof() )
-            return tokenType == EOF ;
-        if ( tokenType == NODE )
-            return peekToken().isNode() ;
-//        if ( tokenType == KEYWORD )
-//        {
-//            String image = tokenRaw().getImage() ;
-//            if ( image.equals(KW_TRUE) )
-//                return true ;
-//            if ( image.equals(KW_FALSE) )
-//                return true ;
-//            return false ; 
-//        }
-        // NB IRIs and PREFIXED_NAMEs
-        return peekToken().hasType(tokenType) ;
-    }
-    
-    // Remember line/col of last token for messages 
-    protected long currLine = -1 ;
-    protected long currCol = -1 ;
-    
-    protected final Token nextToken()
-    {
-        if ( eof() )
-            return tokenEOF ;
-        
-        // Tokenizer errors appear here!
-        try {
-            Token t = peekIter.next() ;
-            currLine = t.getLine() ;
-            currCol = t.getColumn() ;
-            return t ;
-        } catch (RiotParseException ex)
-        {
-            // Intercept to log it.
-            raiseException(ex) ;
-            throw ex ;
-        }
-        catch (AtlasException ex)
-        {
-            // Bad I/O
-            RiotParseException ex2 = new RiotParseException(ex.getMessage(), -1, -1) ;
-            raiseException(ex2) ;
-            throw ex2 ;
-        }
-    }
-
-    protected final Node scopedBNode(Node scopeNode, String label)
-    {
-        return profile.getLabelToNode().get(scopeNode, label) ;
-    }
-    
-    protected final void expectOrEOF(String msg, TokenType tokenType)
-    {
-        // DOT or EOF
-        if ( eof() )
-            return ;
-        expect(msg, tokenType) ;
-    }
-    
-    protected final void expect(String msg, TokenType ttype)
-    {
-        
-        if ( ! lookingAt(ttype) )
-        {
-            Token location = peekToken() ;
-            exception(location, msg) ;
-        }
-        nextToken() ;
-    }
-
-    protected final void exception(Token token, String msg, Object... args)
-    { 
-        if ( token != null )
-            exceptionDirect(String.format(msg, args), token.getLine(), token.getColumn()) ;
-        else
-            exceptionDirect(String.format(msg, args), -1, -1) ;
-    }
-
-    protected final void exceptionDirect(String msg, long line, long col)
-    { 
-        raiseException(new RiotParseException(msg, line, col)) ;
-    }
-    
-    protected final void raiseException(RiotParseException ex)
-    { 
-        ErrorHandler errorHandler = profile.getHandler() ; 
-        if ( errorHandler != null )
-            errorHandler.fatal(ex.getOriginalMessage(), ex.getLine(), ex.getCol()) ;
-        throw ex ;
-    }
-}
-
-/*
- * (c) Copyright 2009 Hewlett-Packard Development Company, LP
- * All rights reserved.
- *
- * Redistribution and use in source and binary forms, with or without
- * modification, are permitted provided that the following conditions
- * are met:
- * 1. Redistributions of source code must retain the above copyright
- *    notice, this list of conditions and the following disclaimer.
- * 2. Redistributions in binary form must reproduce the above copyright
- *    notice, this list of conditions and the following disclaimer in the
- *    documentation and/or other materials provided with the distribution.
- * 3. The name of the author may not be used to endorse or promote products
- *    derived from this software without specific prior written permission.
- *
- * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
- * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
- * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
- * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
- * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
- * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
- * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
- * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
- * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
- */
\ No newline at end of file
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.openjena.riot.lang;
+
+import org.openjena.atlas.event.Event ;
+import org.openjena.atlas.event.EventManager ;
+import org.openjena.atlas.lib.Sink ;
+import org.openjena.riot.SysRIOT ;
+import org.openjena.riot.system.ParserProfile ;
+import org.openjena.riot.tokens.Tokenizer ;
+
+
+public abstract class LangBase<X> extends LangEngine implements LangRIOT
+{
+    protected final Sink<X> sink ; 
+
+    protected LangBase(Tokenizer tokens, ParserProfile profile, Sink<X> sink)
+    {
+        super(tokens, profile) ;
+        this.sink = sink ;
+    }
+
+    public void parse()
+    {
+        EventManager.send(sink, new Event(SysRIOT.startRead, null)) ;
+        runParser() ;
+        sink.flush() ;
+        EventManager.send(sink, new Event(SysRIOT.finishRead, null)) ;
+        tokens.close();
+    }
+    
+    /** Run the parser - events have been handled. */
+    protected abstract void runParser() ;
+
+    public ParserProfile getProfile()
+    {
+        return profile ;
+    }
+
+    public void setProfile(ParserProfile profile)
+    {
+        super.profile = profile ; 
+    }
+}
+

Added: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangEngine.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangEngine.java?rev=1154204&view=auto
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangEngine.java (added)
+++ incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangEngine.java Fri Aug  5 13:21:50 2011
@@ -0,0 +1,187 @@
+/*
+ * (c) Copyright 2009 Hewlett-Packard Development Company, LP
+ * All rights reserved.
+ * [See end of file]
+ */
+
+package org.openjena.riot.lang;
+
+import static org.openjena.riot.tokens.TokenType.EOF ;
+import static org.openjena.riot.tokens.TokenType.NODE ;
+import org.openjena.atlas.AtlasException ;
+import org.openjena.atlas.iterator.PeekIterator ;
+import org.openjena.riot.ErrorHandler ;
+import org.openjena.riot.RiotParseException ;
+import org.openjena.riot.system.ParserProfile ;
+import org.openjena.riot.tokens.Token ;
+import org.openjena.riot.tokens.TokenType ;
+import org.openjena.riot.tokens.Tokenizer ;
+
+import com.hp.hpl.jena.graph.Node ;
+
+/** Common operations for RIOT parsers - not the implementation LangRIOT  */
+public class LangEngine
+{
+    protected ParserProfile profile ;
+    protected final Tokenizer tokens ;
+    private final PeekIterator<Token> peekIter ;
+
+    protected LangEngine(Tokenizer tokens, ParserProfile profile)
+    {
+        this.tokens = tokens ;
+        this.profile = profile ;
+        this.peekIter = new PeekIterator<Token>(tokens) ;
+    }
+     
+    // ---- Managing tokens.
+    
+    protected final Token peekToken()
+    {
+        // Avoid repeating.
+        if ( eof() ) return tokenEOF ;
+        return peekIter.peek() ;
+    }
+    
+    // Set when we get to EOF to record line/col of the EOF.
+    private Token tokenEOF = null ;
+
+    protected final boolean eof()
+    {
+        if ( tokenEOF != null )
+            return true ;
+        
+        if ( ! moreTokens() )
+        {
+            tokenEOF = new Token(tokens.getLine(), tokens.getColumn()) ;
+            tokenEOF.setType(EOF) ;
+            return true ;
+        }
+        return false ;
+    }
+
+    protected final boolean moreTokens() 
+    {
+        return peekIter.hasNext() ;
+    }
+    
+    protected final boolean lookingAt(TokenType tokenType)
+    {
+        if ( eof() )
+            return tokenType == EOF ;
+        if ( tokenType == NODE )
+            return peekToken().isNode() ;
+//        if ( tokenType == KEYWORD )
+//        {
+//            String image = tokenRaw().getImage() ;
+//            if ( image.equals(KW_TRUE) )
+//                return true ;
+//            if ( image.equals(KW_FALSE) )
+//                return true ;
+//            return false ; 
+//        }
+        // NB IRIs and PREFIXED_NAMEs
+        return peekToken().hasType(tokenType) ;
+    }
+    
+    // Remember line/col of last token for messages 
+    protected long currLine = -1 ;
+    protected long currCol = -1 ;
+    
+    protected final Token nextToken()
+    {
+        if ( eof() )
+            return tokenEOF ;
+        
+        // Tokenizer errors appear here!
+        try {
+            Token t = peekIter.next() ;
+            currLine = t.getLine() ;
+            currCol = t.getColumn() ;
+            return t ;
+        } catch (RiotParseException ex)
+        {
+            // Intercept to log it.
+            raiseException(ex) ;
+            throw ex ;
+        }
+        catch (AtlasException ex)
+        {
+            // Bad I/O
+            RiotParseException ex2 = new RiotParseException(ex.getMessage(), -1, -1) ;
+            raiseException(ex2) ;
+            throw ex2 ;
+        }
+    }
+
+    protected final Node scopedBNode(Node scopeNode, String label)
+    {
+        return profile.getLabelToNode().get(scopeNode, label) ;
+    }
+    
+    protected final void expectOrEOF(String msg, TokenType tokenType)
+    {
+        // DOT or EOF
+        if ( eof() )
+            return ;
+        expect(msg, tokenType) ;
+    }
+    
+    protected final void expect(String msg, TokenType ttype)
+    {
+        
+        if ( ! lookingAt(ttype) )
+        {
+            Token location = peekToken() ;
+            exception(location, msg) ;
+        }
+        nextToken() ;
+    }
+
+    protected final void exception(Token token, String msg, Object... args)
+    { 
+        if ( token != null )
+            exceptionDirect(String.format(msg, args), token.getLine(), token.getColumn()) ;
+        else
+            exceptionDirect(String.format(msg, args), -1, -1) ;
+    }
+
+    protected final void exceptionDirect(String msg, long line, long col)
+    { 
+        raiseException(new RiotParseException(msg, line, col)) ;
+    }
+    
+    protected final void raiseException(RiotParseException ex)
+    { 
+        ErrorHandler errorHandler = profile.getHandler() ; 
+        if ( errorHandler != null )
+            errorHandler.fatal(ex.getOriginalMessage(), ex.getLine(), ex.getCol()) ;
+        throw ex ;
+    }
+}
+
+/*
+ * (c) Copyright 2009 Hewlett-Packard Development Company, LP
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. The name of the author may not be used to endorse or promote products
+ *    derived from this software without specific prior written permission.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
\ No newline at end of file

Propchange: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangEngine.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangNTuple.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangNTuple.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangNTuple.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangNTuple.java Fri Aug  5 13:21:50 2011
@@ -46,7 +46,7 @@ public abstract class LangNTuple<X> exte
                          ParserProfile profile,
                          Sink<X> sink)
     { 
-        super(tokens, sink, profile) ;
+        super(tokens, profile, sink) ;
     }
 
     /** Method to parse the whole stream of triples, sending each to the sink */ 

Modified: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangTurtleBase.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangTurtleBase.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangTurtleBase.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/lang/LangTurtleBase.java Fri Aug  5 13:21:50 2011
@@ -114,7 +114,7 @@ public abstract class LangTurtleBase<X> 
     protected LangTurtleBase(String baseURI, Tokenizer tokens,
                              ParserProfile profile, Sink<X> sink)
     { 
-        super(tokens, sink, profile) ;
+        super(tokens, profile, sink) ;
     }
     
     @Override

Modified: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/Tokenizer.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/Tokenizer.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/Tokenizer.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/Tokenizer.java Fri Aug  5 13:21:50 2011
@@ -19,6 +19,12 @@ public interface Tokenizer extends Itera
     /** Move to next token */
     public Token next() ;
     
+    /** Peek next token : null for no token. */
+    public Token peek() ;
+    
+    /** End of tokens? */
+    public boolean eof() ;
+
     public long getLine() ;
     public long getColumn() ;
 }

Modified: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerBytes.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerBytes.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerBytes.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerBytes.java Fri Aug  5 13:21:50 2011
@@ -113,6 +113,18 @@ public final class TokenizerBytes implem
         return t ;
     }
     
+    
+    public final Token peek()
+    {
+        if ( ! hasNext() ) return null ;
+        return token ;
+    }
+    
+    public final boolean eof()
+    {
+        return hasNext() ;
+    }
+    
     //@Override
     public void remove()
     { throw new UnsupportedOperationException() ; }

Modified: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerText.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerText.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerText.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerText.java Fri Aug  5 13:21:50 2011
@@ -83,6 +83,11 @@ public final class TokenizerText impleme
     }    
     
     
+    public final boolean eof()
+    {
+        return hasNext() ;
+    }
+
     //@Override
     public final Token next()
     {
@@ -96,8 +101,7 @@ public final class TokenizerText impleme
     //@Override
     public final Token peek()
     {
-        if ( ! hasNext() ) 
-            throw new NoSuchElementException() ;
+        if ( ! hasNext() ) return null ;
         return token ; 
     }
     

Modified: incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerWrapper.java
URL: http://svn.apache.org/viewvc/incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerWrapper.java?rev=1154204&r1=1154203&r2=1154204&view=diff
==============================================================================
--- incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerWrapper.java (original)
+++ incubator/jena/Jena2/ARQ/trunk/src/org/openjena/riot/tokens/TokenizerWrapper.java Fri Aug  5 13:21:50 2011
@@ -1,5 +1,6 @@
 /*
  * (c) Copyright 2009 Hewlett-Packard Development Company, LP
+ * (c) Copyright 2011 Epimorphics Ltd.
  * All rights reserved.
  * [See end of file]
  */
@@ -34,12 +35,22 @@ public class TokenizerWrapper implements
         return other.hasNext() ;
     }
 
+    public boolean eof()
+    {
+        return other.eof() ;
+    }
+
     //@Override
     public Token next()
     {
         return other.next();
     }
 
+    public Token peek()
+    {
+        return other.peek() ;
+    }
+
     // @Override
     public void remove()
     {
@@ -55,6 +66,7 @@ public class TokenizerWrapper implements
 
 /*
  * (c) Copyright 2009 Hewlett-Packard Development Company, LP
+ * (c) Copyright 2011 Epimorphics Ltd.
  * All rights reserved.
  *
  * Redistribution and use in source and binary forms, with or without