You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by co...@apache.org on 2018/12/06 22:39:57 UTC

[1/2] jena git commit: JENA-1645: Use uri predicate in concrete subject query.

Repository: jena
Updated Branches:
  refs/heads/master 31995c78a -> 501d9f68d


JENA-1645: Use uri predicate in concrete subject query.


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

Branch: refs/heads/master
Commit: 52d959c7a654b03e525fad214b027b6ac6aba2b2
Parents: 0efda76
Author: vdanyliuk <va...@outlook.com>
Authored: Wed Dec 5 11:10:49 2018 +0200
Committer: vdanyliuk <va...@outlook.com>
Committed: Wed Dec 5 11:10:49 2018 +0200

----------------------------------------------------------------------
 .../org/apache/jena/query/text/TextIndex.java   |  6 +-
 .../apache/jena/query/text/TextIndexLucene.java | 39 +++++++--
 .../org/apache/jena/query/text/TextQueryPF.java | 85 ++++++++++++++------
 3 files changed, 101 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/52d959c7/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java
----------------------------------------------------------------------
diff --git a/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java b/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java
index 7208db5..d3095b4 100644
--- a/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java
+++ b/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java
@@ -51,6 +51,10 @@ public interface TextIndex extends Closeable //, Transactional
     List<TextHit> query(Node property, String qs, String graphURI, String lang) ;
 
     List<TextHit> query(Node property, String qs, String graphURI, String lang, int limit, String highlight) ;
-    
+
+    default List<TextHit> query(String subjectUri, Node property, String qs, String graphURI, String lang, int limit, String highlight){
+        return query(property, qs, graphURI, lang, limit, highlight);
+    }
+
     EntityDefinition getDocDef() ;
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/52d959c7/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
----------------------------------------------------------------------
diff --git a/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java b/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
index 1ddce70..f449f85 100644
--- a/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
+++ b/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry ;
+import java.util.function.UnaryOperator;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.jena.datatypes.RDFDatatype ;
@@ -57,9 +58,12 @@ import org.apache.lucene.queryparser.classic.ParseException ;
 import org.apache.lucene.queryparser.classic.QueryParser ;
 import org.apache.lucene.queryparser.classic.QueryParserBase ;
 import org.apache.lucene.queryparser.complexPhrase.ComplexPhraseQueryParser ;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher ;
 import org.apache.lucene.search.Query ;
 import org.apache.lucene.search.ScoreDoc ;
+import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.highlight.Highlighter;
 import org.apache.lucene.search.highlight.InvalidTokenOffsetsException;
 import org.apache.lucene.search.highlight.QueryScorer;
@@ -454,7 +458,7 @@ public class TextIndexLucene implements TextIndex {
     @Override
     public List<TextHit> query(Node property, String qs, String graphURI, String lang, int limit, String highlight) {
         try (IndexReader indexReader = DirectoryReader.open(directory)) {
-            return query$(indexReader, property, qs, graphURI, lang, limit, highlight) ;
+            return query$(indexReader, property, qs, UnaryOperator.identity(), graphURI, lang, limit, highlight) ;
         }
         catch (ParseException ex) {
             throw new TextIndexParseException(qs, ex.getMessage()) ;
@@ -464,6 +468,31 @@ public class TextIndexLucene implements TextIndex {
         }
     }
 
+    @Override
+    public List<TextHit> query(String subjectUri, Node property, String qs, String graphURI, String lang, int limit, String highlight) {
+        try (IndexReader indexReader = DirectoryReader.open(directory)) {
+            return query$(indexReader, property, qs, addUriPredicate(subjectUri), graphURI, lang, limit, highlight) ;
+        }
+        catch (ParseException ex) {
+            throw new TextIndexParseException(qs, ex.getMessage()) ;
+        }
+        catch (Exception ex) {
+            throw new TextIndexException("query", ex) ;
+        }
+    }
+
+    //In a case of making text search query for concrete subject
+    //adding uri predicate will make query much more efficient
+    private UnaryOperator<Query> addUriPredicate(String subjectUri) {
+        return (Query textQuery) -> {
+            String uriField = docDef.getEntityField();
+            return new BooleanQuery.Builder()
+                    .add(textQuery, BooleanClause.Occur.MUST)
+                    .add(new TermQuery(new Term(uriField, subjectUri)), BooleanClause.Occur.FILTER)
+                    .build();
+        };
+    }
+
     private List<TextHit> simpleResults(ScoreDoc[] sDocs, IndexSearcher indexSearcher, Query query, String field) 
             throws IOException {
         List<TextHit> results = new ArrayList<>() ;
@@ -613,8 +642,7 @@ public class TextIndexLucene implements TextIndex {
         }
     }
 
-    private List<TextHit> query$(IndexReader indexReader, Node property, String qs, String graphURI, String lang, int limit, String highlight)
-            throws ParseException, IOException, InvalidTokenOffsetsException {
+    private List<TextHit> query$(IndexReader indexReader, Node property, String qs, UnaryOperator<Query> textQueryExtender, String graphURI, String lang, int limit, String highlight) throws ParseException, IOException, InvalidTokenOffsetsException {
         String litField = docDef.getField(property) != null ?  docDef.getField(property) : docDef.getPrimaryField();
         String textField = litField;
         String textClause = "";               
@@ -658,8 +686,9 @@ public class TextIndexLucene implements TextIndex {
             queryString = "(" + queryString + ") AND " + graphClause ;
         
         Analyzer qa = getQueryAnalyzer(usingSearchFor, lang);
-        Query query = parseQuery(queryString, qa) ;
-        
+        Query textQuery = parseQuery(queryString, qa);
+        Query query = textQueryExtender.apply(textQuery);
+
         if ( limit <= 0 )
             limit = MAX_N ;
 

http://git-wip-us.apache.org/repos/asf/jena/blob/52d959c7/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java
----------------------------------------------------------------------
diff --git a/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java b/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java
index 3c61ac0..0b4e324 100644
--- a/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java
+++ b/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java
@@ -35,6 +35,7 @@ import org.apache.jena.graph.Node ;
 import org.apache.jena.ext.com.google.common.base.Strings;
 import org.apache.jena.ext.com.google.common.collect.LinkedListMultimap;
 import org.apache.jena.ext.com.google.common.collect.ListMultimap;
+import org.apache.jena.graph.Node_URI;
 import org.apache.jena.query.QueryBuildException ;
 import org.apache.jena.query.QueryExecException ;
 import org.apache.jena.sparql.core.* ;
@@ -250,8 +251,13 @@ public class TextQueryPF extends PropertyFunctionBase {
 
     private QueryIterator concreteSubject(Binding binding, Node s, Node score, Node literal, Node graph, StrMatch match, ExecutionContext execCxt) {
         log.trace("concreteSubject: {}", match) ;
-        ListMultimap<String,TextHit> x = query(match.getProperty(), match.getQueryString(), match.getLang(), -1, match.getHighlight(), execCxt) ;
-        
+        ListMultimap<String,TextHit> x;
+
+        if (s instanceof Node_URI) {
+            x = query(s.getURI(), match.getProperty(), match.getQueryString(), match.getLang(), -1, match.getHighlight(), execCxt);
+        } else {
+            x = query(match.getProperty(), match.getQueryString(), match.getLang(), -1, match.getHighlight(), execCxt);
+        }
         if ( x == null ) // null return value - empty result
             return IterLib.noResults(execCxt) ;
         
@@ -262,38 +268,62 @@ public class TextQueryPF extends PropertyFunctionBase {
 
     private ListMultimap<String,TextHit> query(Node property, String queryString, String lang, int limit, String highlight, ExecutionContext execCxt) {
         String graphURI = chooseGraphURI(execCxt);
-        
-        if ( graphURI == null ) {
-            Explain.explain(execCxt.getContext(), "Text query: "+queryString) ;
-            log.debug("Text query: {} ({})", queryString, limit) ;
+
+        explainQuery(queryString, limit, execCxt, graphURI);
+
+        if (textIndex.getDocDef().areQueriesCached()) {
+            // Cache-key does not matter if lang or graphURI are null
+            String cacheKey = limit + " " + property + " " + queryString + " " + lang + " " + graphURI ;
+            Cache<String, ListMultimap<String, TextHit>> queryCache = prepareCache(execCxt);
+
+            log.trace("Caching Text query: {} with key: >>{}<< in cache: {}", queryString, cacheKey, queryCache) ;
+
+            return queryCache.getOrFill(cacheKey, ()->performQuery(property, queryString, graphURI, lang, limit, highlight));
         } else {
-            Explain.explain(execCxt.getContext(), "Text query <"+graphURI+">: "+queryString) ;
-            log.debug("Text query: {} <{}> ({})", queryString, graphURI, limit) ;
+            log.trace("Executing w/o cache Text query: {}", queryString) ;
+            return performQuery(property, queryString, graphURI, lang, limit, highlight);
         }
+    }
+
+    private ListMultimap<String,TextHit> query(String uri, Node property, String queryString, String lang, int limit, String highlight, ExecutionContext execCxt) {
+        String graphURI = chooseGraphURI(execCxt);
+
+        explainQuery(queryString, limit, execCxt, graphURI);
 
-        ListMultimap<String,TextHit> results;
-        
         if (textIndex.getDocDef().areQueriesCached()) {
             // Cache-key does not matter if lang or graphURI are null
-            String cacheKey = limit + " " + property + " " + queryString + " " + lang + " " + graphURI ;
-            @SuppressWarnings("unchecked")
-            Cache<String,ListMultimap<String,TextHit>> queryCache = 
-                (Cache<String,ListMultimap<String,TextHit>>) execCxt.getContext().get(cacheSymbol);
-            if (queryCache == null) {
-                /* doesn't yet exist, need to create it */
-                queryCache = CacheFactory.createCache(CACHE_SIZE);
-                execCxt.getContext().put(cacheSymbol, queryCache);
-            }
+            String cacheKey = uri + " " + limit + " " + property + " " + queryString + " " + lang + " " + graphURI ;
+            Cache<String, ListMultimap<String, TextHit>> queryCache = prepareCache(execCxt);
 
             log.trace("Caching Text query: {} with key: >>{}<< in cache: {}", queryString, cacheKey, queryCache) ;
 
-            results = queryCache.getOrFill(cacheKey, ()->performQuery(property, queryString, graphURI, lang, limit, highlight));
+            return queryCache.getOrFill(cacheKey, ()->performQuery(uri, property, queryString, graphURI, lang, limit, highlight));
         } else {
             log.trace("Executing w/o cache Text query: {}", queryString) ;
-            results = performQuery(property, queryString, graphURI, lang, limit, highlight);
+            return performQuery(uri, property, queryString, graphURI, lang, limit, highlight);
         }
+    }
 
-        return results;
+    private Cache<String, ListMultimap<String, TextHit>> prepareCache(ExecutionContext execCxt) {
+        @SuppressWarnings("unchecked")
+        Cache<String, ListMultimap<String, TextHit>> queryCache =
+                (Cache<String, ListMultimap<String, TextHit>>) execCxt.getContext().get(cacheSymbol);
+        if (queryCache == null) {
+            /* doesn't yet exist, need to create it */
+            queryCache = CacheFactory.createCache(CACHE_SIZE);
+            execCxt.getContext().put(cacheSymbol, queryCache);
+        }
+        return queryCache;
+    }
+
+    private void explainQuery(String queryString, int limit, ExecutionContext execCxt, String graphURI) {
+        if ( graphURI == null ) {
+            Explain.explain(execCxt.getContext(), "Text query: "+queryString) ;
+            log.debug("Text query: {} ({})", queryString, limit) ;
+        } else {
+            Explain.explain(execCxt.getContext(), "Text query <"+graphURI+">: "+queryString) ;
+            log.debug("Text query: {} <{}> ({})", queryString, graphURI, limit) ;
+        }
     }
 
     private String chooseGraphURI(ExecutionContext execCxt) {
@@ -314,13 +344,22 @@ public class TextQueryPF extends PropertyFunctionBase {
     
     private ListMultimap<String,TextHit> performQuery(Node property, String queryString, String graphURI, String lang, int limit, String highlight) {
         List<TextHit> resultList = textIndex.query(property, queryString, graphURI, lang, limit, highlight) ;
+        return mapResult(resultList);
+    }
+
+    private ListMultimap<String,TextHit> performQuery(String uri, Node property, String queryString, String graphURI, String lang, int limit, String highlight) {
+        List<TextHit> resultList = textIndex.query(uri, property, queryString, graphURI, lang, limit, highlight) ;
+        return mapResult(resultList);
+    }
+
+    private ListMultimap<String, TextHit> mapResult(List<TextHit> resultList) {
         ListMultimap<String,TextHit> results = LinkedListMultimap.create();
         for (TextHit result : resultList) {
             results.put(TextQueryFuncs.subjectToString(result.getNode()), result);
         }
         return results;
     }
-    
+
     /** Deconstruct the node or list object argument and make a StrMatch 
      * The 'executionTime' flag indicates whether this is for a build time
      * static check, or for runtime execution.


[2/2] jena git commit: JEAN-1645 Merge commit 'refs/pull/503/head' of https://github.com/apache/jena. This closes #503

Posted by co...@apache.org.
JEAN-1645 Merge commit 'refs/pull/503/head' of https://github.com/apache/jena. This closes #503


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

Branch: refs/heads/master
Commit: 501d9f68d46e30ce5289b04dfba13f07bbf7d2d9
Parents: 31995c7 52d959c
Author: Code Ferret <co...@apache.org>
Authored: Thu Dec 6 13:15:08 2018 -0600
Committer: Code Ferret <co...@apache.org>
Committed: Thu Dec 6 13:15:08 2018 -0600

----------------------------------------------------------------------
 .../org/apache/jena/query/text/TextIndex.java   |  6 +-
 .../apache/jena/query/text/TextIndexLucene.java | 39 +++++++--
 .../org/apache/jena/query/text/TextQueryPF.java | 85 ++++++++++++++------
 3 files changed, 101 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/501d9f68/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
----------------------------------------------------------------------
diff --cc jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
index b217c8a,f449f85..00a9a88
--- a/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
+++ b/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
@@@ -653,14 -677,18 +681,15 @@@ public class TextIndexLucene implement
          
          String queryString = textClause ;
  
 -        if (graphClause != null)
 -            queryString = "(" + queryString + ") AND " + graphClause ;
 +        if (graphURI != null) {
 +            String escaped = QueryParserBase.escape(graphURI) ;
 +            queryString = "(" + queryString + ") AND " + getDocDef().getGraphField() + ":" + escaped ;
 +        }
          
          Analyzer qa = getQueryAnalyzer(usingSearchFor, lang);
-         Query query = parseQuery(queryString, qa) ;
-         
+         Query textQuery = parseQuery(queryString, qa);
+         Query query = textQueryExtender.apply(textQuery);
+ 
          if ( limit <= 0 )
              limit = MAX_N ;