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 2013/06/23 18:26:33 UTC

svn commit: r1495837 - in /jena/trunk/jena-text/src/main/java/org/apache/jena/query/text: QueryPF.java TextIndex.java TextIndexLucene.java assembler/TextIndexLuceneAssembler.java

Author: andy
Date: Sun Jun 23 16:26:33 2013
New Revision: 1495837

URL: http://svn.apache.org/r1495837
Log:
Functionally correct but potentially slow implementation for checking existence of URI against a text query.  
This is not the common use case.

Modified:
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/QueryPF.java
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexLuceneAssembler.java

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/QueryPF.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/QueryPF.java?rev=1495837&r1=1495836&r2=1495837&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/QueryPF.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/QueryPF.java Sun Jun 23 16:26:33 2013
@@ -16,14 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.jena.query.text;
+package org.apache.jena.query.text ;
 
 import java.util.List ;
-import java.util.Map ;
 
 import org.apache.jena.atlas.iterator.Iter ;
 import org.apache.jena.atlas.lib.InternalErrorException ;
 import org.apache.jena.atlas.logging.Log ;
+import org.apache.lucene.queryparser.classic.QueryParser ;
+import org.slf4j.Logger ;
+import org.slf4j.LoggerFactory ;
 
 import com.hp.hpl.jena.datatypes.RDFDatatype ;
 import com.hp.hpl.jena.datatypes.xsd.XSDDatatype ;
@@ -39,82 +41,52 @@ import com.hp.hpl.jena.sparql.engine.ite
 import com.hp.hpl.jena.sparql.pfunction.PropFuncArg ;
 import com.hp.hpl.jena.sparql.pfunction.PropertyFunctionBase ;
 import com.hp.hpl.jena.sparql.util.IterLib ;
-import com.hp.hpl.jena.sparql.util.NodeFactoryExtra;
+import com.hp.hpl.jena.sparql.util.NodeFactoryExtra ;
 
-/** property function that accesses a Solr server */ 
-public class QueryPF extends PropertyFunctionBase
-{
-    private TextIndex server = null ;  
-    private boolean warningIssued = false ;
-    
-    public QueryPF() { } 
+/** property function that accesses a Solr server */
+public class QueryPF extends PropertyFunctionBase {
+    private static Logger log           = LoggerFactory.getLogger(QueryPF.class) ;
+    /*
+     * ?uri :queryPF (property? "string" limit? score?) score? not implemented
+     */
+
+    private TextIndex     server        = null ;
+    private boolean       warningIssued = false ;
+
+    public QueryPF() {}
 
     @Override
-    public void build(PropFuncArg argSubject, Node predicate, PropFuncArg argObject, ExecutionContext execCxt)
-    {
+    public void build(PropFuncArg argSubject, Node predicate, PropFuncArg argObject, ExecutionContext execCxt) {
         super.build(argSubject, predicate, argObject, execCxt) ;
-        
+
         DatasetGraph dsg = execCxt.getDataset() ;
         server = chooseTextIndex(dsg) ;
-        
-        if ( ! argSubject.isNode() )
-            throw new QueryBuildException("Subject is not a single node: "+argSubject) ;
-        
-        if ( argObject.isList() )
-        {
+
+        if (!argSubject.isNode())
+            throw new QueryBuildException("Subject is not a single node: " + argSubject) ;
+
+        if (argObject.isList()) {
             List<Node> list = argObject.getArgList() ;
-            if ( list.size() == 0 )
+            if (list.size() == 0)
                 throw new QueryBuildException("Zero-length argument list") ;
 
-            if ( list.size() > 4 )
-                throw new QueryBuildException("Too many arguments in list : "+list) ;
+            if (list.size() > 4)
+                throw new QueryBuildException("Too many arguments in list : " + list) ;
         }
     }
 
-    /*
-     * ?uri :queryPF (property? "string" limit? score?)
-     * score? not implemented
-     */
-    
-    // score limit - float : new IteratorTruncate<SolrDocument>(...., iter) ; 
-    
-    static class StrMatch
-    {
-        private final Node property ;
-        private final String queryString ;
-        private final int limit ;
-        private final float scoreLimit ;
-
-        public StrMatch(Node property, String queryString, int limit, float scoreLimit)
-        {
-            super() ;
-            this.property = property ;
-            this.queryString = queryString ;
-            this.limit = limit ;
-            this.scoreLimit = scoreLimit ;
-        }
-
-        public Node getProperty()           { return property ; }
-
-        public String getQueryString()      { return queryString ; }
-
-        public int getLimit()               { return limit ; }
-
-        public float getScoreLimit()        { return scoreLimit ; }
-    }
-    
-    private static TextIndex chooseTextIndex(DatasetGraph dsg)
-    {
+    private static TextIndex chooseTextIndex(DatasetGraph dsg) {
         Object obj = dsg.getContext().get(TextQuery.textIndex) ;
 
-        if ( obj != null )
-        {
-            try { return (TextIndex)obj ; } 
-            catch (ClassCastException ex) { Log.warn(QueryPF.class, "Context setting '"+TextQuery.textIndex+"'is not a TextIndex") ; }
+        if (obj != null) {
+            try {
+                return (TextIndex)obj ;
+            } catch (ClassCastException ex) {
+                Log.warn(QueryPF.class, "Context setting '" + TextQuery.textIndex + "'is not a TextIndex") ;
+            }
         }
 
-        if ( dsg instanceof DatasetGraphText )
-        {
+        if (dsg instanceof DatasetGraphText) {
             DatasetGraphText x = (DatasetGraphText)dsg ;
             return x.getTextIndex() ;
         }
@@ -123,134 +95,191 @@ public class QueryPF extends PropertyFun
     }
 
     @Override
-    public QueryIterator exec(Binding binding, PropFuncArg argSubject, Node predicate, PropFuncArg argObject, ExecutionContext execCxt)
-    {
-        if ( server == null )
-        {
-            if ( ! warningIssued )
-            {
+    public QueryIterator exec(Binding binding, PropFuncArg argSubject, Node predicate, PropFuncArg argObject,
+                              ExecutionContext execCxt) {
+        if (server == null) {
+            if (!warningIssued) {
                 Log.warn(getClass(), "No text index - no text search performed") ;
                 warningIssued = true ;
             }
             // Not a text dataset - no-op
             return IterLib.result(binding, execCxt) ;
         }
-     
+
         DatasetGraph dsg = execCxt.getDataset() ;
-        
-        if ( ! argSubject.isNode() )
+
+        if (!argSubject.isNode())
             throw new InternalErrorException("Subject is not a node (it was earlier!)") ;
-            
+
         Node s = argSubject.getArg() ;
-        
-        if ( s.isLiteral() )
+
+        if (s.isLiteral())
             // Does not match
             return IterLib.noResults(execCxt) ;
-        
+
         StrMatch match = objectToStruct(argObject) ;
+        if (match == null) {
+            // can't match
+            return IterLib.noResults(execCxt) ;
+        }
 
         // ----
-        
-        QueryIterator qIter =  ( Var.isVar(s) ) 
-            ? variableSubject(binding, s, match, execCxt)
-            : concreteSubject(binding, s, match, execCxt) ;
-        
-        if ( match.getLimit() >= 0 )
+
+        QueryIterator qIter = (Var.isVar(s)) ? variableSubject(binding, s, match, execCxt) : concreteSubject(binding,
+                                                                                                             s, match,
+                                                                                                             execCxt) ;
+
+        if (match.getLimit() >= 0)
             qIter = new QueryIterSlice(qIter, 0, match.getLimit(), execCxt) ;
         return qIter ;
     }
 
-    private QueryIterator variableSubject(Binding binding, Node s, StrMatch match, ExecutionContext execCxt )
-    {
+    private QueryIterator variableSubject(Binding binding, Node s, StrMatch match, ExecutionContext execCxt) {
         Var v = Var.alloc(s) ;
-        List<Node> r = server.query(match.getQueryString(), match.getLimit()) ;
-        // Make distinct.  Note interaction with limit is imperfect
+        List<Node> r = query(match.getQueryString(), match.getLimit()) ;
+        // Make distinct. Note interaction with limit is imperfect
         r = Iter.iter(r).distinct().toList() ;
         QueryIterator qIter = new QueryIterExtendByVar(binding, v, r.iterator(), execCxt) ;
         return qIter ;
     }
 
-    private QueryIterator concreteSubject(Binding binding, Node s, StrMatch match, ExecutionContext execCxt )
-    {
-        if ( ! s.isURI() )
-        {
-            Log.warn(this, "Subject not a URI: "+s) ;
-            return IterLib.noResults(execCxt) ; 
+    private QueryIterator concreteSubject(Binding binding, Node s, StrMatch match, ExecutionContext execCxt) {
+        if (!s.isURI()) {
+            log.warn("Subject not a URI: " + s) ;
+            return IterLib.noResults(execCxt) ;
         }
-        
+
         String uri = s.getURI() ;
-        Map<String, Node> x = server.get(uri) ;
-        if ( x == null || x.isEmpty() )
+        
+        // Restrict to matching and entity field be right.
+        String qs = match.getQueryString() ;
+        if ( false ) {
+            // This should work but it doesn't
+            String escaped = QueryParser.escape(uri) ;
+            String qs2 = server.getDocDef().getEntityField() + ":" + escaped ;
+            qs = qs2 + " AND " + qs ;
+            List<Node> x = query(qs, 1) ;
+            if (x == null || x.isEmpty())
+                return IterLib.noResults(execCxt) ;
+            else
+                return IterLib.result(binding, execCxt) ;
+        }
+        // Crude.
+        List<Node> x = query(qs, -1) ;
+        if ( x == null || ! x.contains(s) )
             return IterLib.noResults(execCxt) ;
         else
             return IterLib.result(binding, execCxt) ;
     }
 
-    /** Deconstruct the node or list object argument and make a StrMatch */ 
-    private StrMatch objectToStruct(PropFuncArg argObject)
-    {
-        
-        EntityDefinition docDef = server.getDocDef()  ;
-        if ( argObject.isNode() )
-        {
+    private List<Node> query(String queryString, int limit) {
+        // Explain
+        if ( log.isInfoEnabled())
+            log.info("Text query: {} ({})", queryString,limit) ;
+        return server.query(queryString, limit) ;
+    }
+    
+    /** Deconstruct the node or list object argument and make a StrMatch */
+    private StrMatch objectToStruct(PropFuncArg argObject) {
+        EntityDefinition docDef = server.getDocDef() ;
+        if (argObject.isNode()) {
             Node o = argObject.getArg() ;
-            
-            if ( ! o.isLiteral() )
-            { System.err.println("Bad/4") ; }
-            
+
+            if (!o.isLiteral()) {
+                log.warn("Object to text query is not a literal") ;
+                return null ;
+            }
+
             RDFDatatype dt = o.getLiteralDatatype() ;
-            if ( dt != null && dt != XSDDatatype.XSDstring )
-            { System.err.println("Bad") ; }
-                
+            if (dt != null && dt != XSDDatatype.XSDstring) {
+                log.warn("Object to text query is not a string") ;
+                return null ;
+            }
+
             String qs = o.getLiteralLexicalForm() ;
-            return new StrMatch(docDef.getPrimaryPredicate(), qs, -1, 0) ; 
+            return new StrMatch(docDef.getPrimaryPredicate(), qs, -1, 0) ;
         }
-         
+
         List<Node> list = argObject.getArgList() ;
-        if ( list.size() == 0 || list.size() > 3 )
-            throw new TextIndexException("Change in object list size") ; 
+        if (list.size() == 0 || list.size() > 3)
+            throw new TextIndexException("Change in object list size") ;
 
         Node p = docDef.getPrimaryPredicate() ;
         String field = docDef.getPrimaryField() ;
         int idx = 0 ;
         Node x = list.get(0) ;
         // Property?
-        if ( x.isURI() )
-        {
+        if (x.isURI()) {
             p = x ;
             idx++ ;
-            if ( idx >= list.size() )
-                throw new TextIndexException("Property specificied but no query string : "+list) ;
+            if (idx >= list.size())
+                throw new TextIndexException("Property specificied but no query string : " + list) ;
             x = list.get(idx) ;
-            field = docDef.getField(p) ; 
+            field = docDef.getField(p) ;
+            if (field == null) {
+                log.warn("Predicate not indexed: " + p) ;
+                return null ;
+            }
         }
-        
+
         // String!
-        if ( ! x.isLiteral() )
-            throw new TextIndexException("Query isn't a literal string : "+list) ;
-        if ( x.getLiteralDatatype() != null && ! x.getLiteralDatatype().equals(XSDDatatype.XSDstring) )
-            throw new TextIndexException("Query isn't a string : "+list) ;
-        String queryString = x.getLiteralLexicalForm() ;  
+        if (!x.isLiteral()) {
+            log.warn("Text query string is not a literal " + list) ;
+            return null ;
+        }
+        if (x.getLiteralDatatype() != null && !x.getLiteralDatatype().equals(XSDDatatype.XSDstring)) {
+            log.warn("Text query is not a string " + list) ;
+            return null ;
+        }
+        String queryString = x.getLiteralLexicalForm() ;
         idx++ ;
-        
+
         int limit = -1 ;
         float score = 0 ;
-        
-        if ( idx < list.size() )
-        {        
+
+        if (idx < list.size()) {
             // Limit?
             x = list.get(idx) ;
             idx++ ;
             int v = NodeFactoryExtra.nodeToInt(x) ;
-            limit = ( v < 0 ) ? -1 : v ; 
+            limit = (v < 0) ? -1 : v ;
         }
 
         String qs = queryString ;
-        if ( field != null )
-            qs = field+":"+qs ;
-        
+        if (field != null)
+            qs = field + ":" + qs ;
+
         return new StrMatch(p, qs, limit, score) ;
     }
-}
 
+    class StrMatch {
+        private final Node   property ;
+        private final String queryString ;
+        private final int    limit ;
+        private final float  scoreLimit ;
 
+        public StrMatch(Node property, String queryString, int limit, float scoreLimit) {
+            super() ;
+            this.property = property ;
+            this.queryString = queryString ;
+            this.limit = limit ;
+            this.scoreLimit = scoreLimit ;
+        }
+
+        public Node getProperty() {
+            return property ;
+        }
+
+        public String getQueryString() {
+            return queryString ;
+        }
+
+        public int getLimit() {
+            return limit ;
+        }
+
+        public float getScoreLimit() {
+            return scoreLimit ;
+        }
+    }
+}

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java?rev=1495837&r1=1495836&r2=1495837&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java Sun Jun 23 16:26:33 2013
@@ -25,13 +25,17 @@ import org.apache.jena.atlas.lib.Closeab
 
 import com.hp.hpl.jena.graph.Node ;
 
+/** TextIndex abstraction */ 
 public interface TextIndex extends Closeable //, Transactional 
 {
+    // Update operations
     public abstract void startIndexing() ;
     public abstract void addEntity(Entity entity) ;
     public abstract void finishIndexing() ;
     public abstract void abortIndexing() ;
     
+    // read operations
+    /** Get all entries for uri */
     public abstract Map<String, Node> get(String uri) ;
 
     /** Access the index - limit if -1 for as many as possible */ 

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java?rev=1495837&r1=1495836&r2=1495837&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java Sun Jun 23 16:26:33 2013
@@ -18,45 +18,52 @@
 
 package org.apache.jena.query.text;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.queryparser.classic.ParseException;
-import org.apache.lucene.queryparser.classic.QueryParser;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.Version;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException ;
+import java.util.* ;
+import java.util.Map.Entry ;
+
+import org.apache.lucene.analysis.Analyzer ;
+import org.apache.lucene.analysis.standard.StandardAnalyzer ;
+import org.apache.lucene.document.Document ;
+import org.apache.lucene.document.Field ;
+import org.apache.lucene.document.FieldType ;
+import org.apache.lucene.document.TextField ;
+import org.apache.lucene.index.DirectoryReader ;
+import org.apache.lucene.index.IndexReader ;
+import org.apache.lucene.index.IndexWriter ;
+import org.apache.lucene.index.IndexWriterConfig ;
+import org.apache.lucene.queryparser.classic.ParseException ;
+import org.apache.lucene.queryparser.classic.QueryParser ;
+import org.apache.lucene.search.IndexSearcher ;
+import org.apache.lucene.search.Query ;
+import org.apache.lucene.search.ScoreDoc ;
+import org.apache.lucene.store.Directory ;
+import org.apache.lucene.util.Version ;
+import org.slf4j.Logger ;
+import org.slf4j.LoggerFactory ;
 
-import com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.Node ;
 import com.hp.hpl.jena.graph.NodeFactory ;
-import com.hp.hpl.jena.sparql.util.NodeFactoryExtra;
+import com.hp.hpl.jena.sparql.util.NodeFactoryExtra ;
 
 public class TextIndexLucene implements TextIndex
 {
     private static Logger log = LoggerFactory.getLogger(TextIndexLucene.class) ;
     
     private static int MAX_N = 10000 ;
-    private static final Version VER = Version.LUCENE_41 ;
-    static FieldType ftIndexedStored = TextField.TYPE_STORED ; 
-    static FieldType ftIndexed = TextField.TYPE_NOT_STORED ;
+    public static final Version VER = Version.LUCENE_41 ;
+    
+    public static final FieldType ftIRI ;
+    static {
+        ftIRI = new FieldType() ;
+        ftIRI.setTokenized(false) ;
+        ftIRI.setStored(true) ;
+        ftIRI.setIndexed(true) ;
+        ftIRI.freeze() ;
+    }
+    //public static final FieldType ftText = TextField.TYPE_NOT_STORED ;
+    // Bigger index, easier to debug!
+    public static final FieldType ftText = TextField.TYPE_STORED ;
     
     private final EntityDefinition docDef ;
     private final Directory directory ;
@@ -70,11 +77,14 @@ public class TextIndexLucene implements 
         this.docDef = def ;
         
         // force creation of the index if it don't exist
-        // othewise if we get a search before data is written we get an exception
+        // otherwise if we get a search before data is written we get an exception
         startIndexing();
         finishIndexing();
     }
     
+    public Directory getDirectory()     { return directory ; }
+    public Analyzer getAnalyzer()       { return analyzer ; }
+    
     @Override
     public void startIndexing()
     { 
@@ -119,12 +129,12 @@ public class TextIndexLucene implements 
     private Document doc(Entity entity)
     {
         Document doc = new Document() ;
-        Field entField = new Field(docDef.getEntityField(), entity.getId(), ftIndexedStored) ;
+        Field entField = new Field(docDef.getEntityField(), entity.getId(), ftIRI) ;
         doc.add(entField) ;
         
         for ( Entry<String, Object> e : entity.getMap().entrySet() )
         {
-            Field field = new Field(e.getKey(), (String)e.getValue(), ftIndexed) ;
+            Field field = new Field(e.getKey(), (String)e.getValue(), ftText) ;
             doc.add(field) ;
         }
         return doc ;
@@ -144,13 +154,13 @@ public class TextIndexLucene implements 
         } catch (Exception ex) { exception(ex) ; return null ; } 
     }
     
-    private List<Map<String, Node>> get$(IndexReader indexReader , String uri)  throws ParseException, IOException {
+    private List<Map<String, Node>> get$(IndexReader indexReader, String uri)  throws ParseException, IOException {
         String escaped = QueryParser.escape(uri);
         String qs = docDef.getEntityField()+":"+escaped ;
         QueryParser queryParser = new QueryParser(VER, docDef.getPrimaryField(), analyzer);
         Query query = queryParser.parse(qs);
         IndexSearcher indexSearcher = new IndexSearcher(indexReader);
-        ScoreDoc[] sDocs = indexSearcher.search(query, 1).scoreDocs ;   // Only need one hit.    
+        ScoreDoc[] sDocs = indexSearcher.search(query, 1).scoreDocs ;    
         List<Map<String, Node>> records = new ArrayList<Map<String, Node>>() ;
 
         // Align and DRY with Solr.
@@ -195,7 +205,7 @@ public class TextIndexLucene implements 
         } catch (Exception ex) { exception(ex) ; return null ; } 
     }
         
-    public List<Node> query$(IndexReader indexReader , String qs, int limit) throws ParseException, IOException {
+    private List<Node> query$(IndexReader indexReader , String qs, int limit) throws ParseException, IOException {
         IndexSearcher indexSearcher = new IndexSearcher(indexReader);
         QueryParser queryParser = new QueryParser(VER, docDef.getPrimaryField(), analyzer);
         Query query = queryParser.parse(qs);

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexLuceneAssembler.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexLuceneAssembler.java?rev=1495837&r1=1495836&r2=1495837&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexLuceneAssembler.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexLuceneAssembler.java Sun Jun 23 16:26:33 2013
@@ -65,7 +65,7 @@ public class TextIndexLuceneAssembler ex
             if ( n.isLiteral() )
             {
                 if ( ! "mem".equals(n.asLiteral().getLexicalForm()) )
-                    throw new TextIndexException("No 'text:directory' property on "+root+ " is a liteal and not \"mem\"") ;
+                    throw new TextIndexException("No 'text:directory' property on "+root+ " is a literal and not \"mem\"") ;
                  directory = new RAMDirectory() ;
             }
             else