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/12/09 12:59:52 UTC

svn commit: r1549557 - in /jena/trunk/jena-text/src: main/java/org/apache/jena/query/text/ main/java/org/apache/jena/query/text/assembler/ test/java/org/apache/jena/query/text/

Author: andy
Date: Mon Dec  9 11:59:51 2013
New Revision: 1549557

URL: http://svn.apache.org/r1549557
Log:
Patch applied from JENA-605

jena-text-graph-with-tests.diff / 2013-12-09

Added:
    jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithGraphTextIndex.java
    jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithLuceneGraphTextIndex.java
    jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithLuceneGraphTextIndex.java
Modified:
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/Entity.java
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/EntityDefinition.java
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerTriples.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/TextIndexSolr.java
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/EntityMapAssembler.java
    jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndex.java
    jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/Entity.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/Entity.java?rev=1549557&r1=1549556&r2=1549557&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/Entity.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/Entity.java Mon Dec  9 11:59:51 2013
@@ -24,12 +24,20 @@ import java.util.Map ;
 public class Entity
 {
     private final String id ;
+    private final String graph ;
     private final Map<String, Object> map = new HashMap<String, Object>() ;
 
-    public Entity(String entityId)          { this.id = entityId ; }
+    public Entity(String entityId, String entityGraph) {
+        this.id = entityId ;
+        this.graph = entityGraph;
+    }
+
+    public Entity(String entityId)          { this(entityId, null) ; }
     
     public String getId()                   { return id ; }
 
+    public String getGraph()                { return graph ; }
+
     public void put(String key, Object value)
     { map.put(key, value) ; }
     

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/EntityDefinition.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/EntityDefinition.java?rev=1549557&r1=1549556&r2=1549557&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/EntityDefinition.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/EntityDefinition.java Mon Dec  9 11:59:51 2013
@@ -39,6 +39,7 @@ public class EntityDefinition {
     // Collections.unmodifiableCollection(fieldToPredicate.keySet()) ;
     private final String                 entityField ;
     private final String                 primaryField ;
+    private final String                 graphField ;
     //private final Node                   primaryPredicate ;
 
     /**
@@ -48,8 +49,21 @@ public class EntityDefinition {
      *            The primary/default field to search
      */
     public EntityDefinition(String entityField, String primaryField) {
+        this(entityField, primaryField, (String)null) ;
+    }
+
+    /**
+     * @param entityField
+     *            The entity being indexed (e.g. it's URI).
+     * @param primaryField
+     *            The primary/default field to search
+     * @param graphField
+     *            The field that stores graph URI, or null
+     */
+    public EntityDefinition(String entityField, String primaryField, String graphField) {
         this.entityField = entityField ;
         this.primaryField = primaryField ;
+        this.graphField = graphField ;
     }
 
     /**
@@ -61,7 +75,7 @@ public class EntityDefinition {
      *            The property associated with the primary/default field
      */
     public EntityDefinition(String entityField, String primaryField, Resource primaryPredicate) {
-        this(entityField, primaryField, primaryPredicate.asNode()) ;
+        this(entityField, primaryField, null, primaryPredicate.asNode()) ;
     }
 
     /**
@@ -73,7 +87,21 @@ public class EntityDefinition {
      *            The property associated with the primary/default field
      */
     public EntityDefinition(String entityField, String primaryField, Node primaryPredicate) {
-        this(entityField, primaryField) ;
+        this(entityField, primaryField, null, primaryPredicate) ;
+    }
+
+    /**
+     * @param entityField
+     *            The entity being indexed (e.g. it's URI).
+     * @param primaryField
+     *            The primary/default field to search
+     * @param graphField
+     *            The field that stores graph URI, or null
+     * @param primaryPredicate
+     *            The property associated with the primary/default field
+     */
+    public EntityDefinition(String entityField, String primaryField, String graphField, Node primaryPredicate) {
+        this(entityField, primaryField, graphField) ;
         set(primaryField, primaryPredicate) ;
     }
 
@@ -107,6 +135,10 @@ public class EntityDefinition {
         return getOne(c) ;
     }
 
+    public String getGraphField() {
+        return graphField ;
+    }
+
     public Collection<String> fields() {
         return fields ;
     }

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerTriples.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerTriples.java?rev=1549557&r1=1549556&r2=1549557&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerTriples.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerTriples.java Mon Dec  9 11:59:51 2013
@@ -65,7 +65,8 @@ public class TextDocProducerTriples impl
             return ;
 
         String x = (s.isURI() ) ? s.getURI() : s.getBlankNodeLabel() ;
-        Entity entity = new Entity(x) ;
+        String graph = (g.isURI() ) ? g.getURI() : "_:" + g.getBlankNodeLabel() ;
+        Entity entity = new Entity(x, graph) ;
 
         if ( ! o.isLiteral() )
         {

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=1549557&r1=1549556&r2=1549557&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 Mon Dec  9 11:59:51 2013
@@ -23,10 +23,13 @@ import java.util.* ;
 import java.util.Map.Entry ;
 
 import org.apache.lucene.analysis.Analyzer ;
+import org.apache.lucene.analysis.core.KeywordAnalyzer ;
+import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper ;
 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.StringField ;
 import org.apache.lucene.document.TextField ;
 import org.apache.lucene.index.DirectoryReader ;
 import org.apache.lucene.index.IndexReader ;
@@ -61,6 +64,7 @@ public class TextIndexLucene implements 
         ftIRI.setIndexed(true) ;
         ftIRI.freeze() ;
     }
+    public static final FieldType ftString = StringField.TYPE_NOT_STORED ;
     public static final FieldType ftText = TextField.TYPE_NOT_STORED ;
     // Bigger index, easier to debug!
     // public static final FieldType ftText = TextField.TYPE_STORED ;
@@ -68,13 +72,21 @@ public class TextIndexLucene implements 
     private final EntityDefinition docDef ;
     private final Directory directory ;
     private IndexWriter indexWriter ;
-    private Analyzer analyzer = new StandardAnalyzer(VER);
+    private Analyzer analyzer ;
     
     public TextIndexLucene(Directory directory, EntityDefinition def)
     {
         this.directory = directory ;
         this.docDef = def ;
         
+        // create the analyzer as a wrapper that uses KeywordAnalyzer for
+        // entity and graph fields and StandardAnalyzer for all other
+        Map<String,Analyzer> analyzerPerField = new HashMap<String,Analyzer>() ;
+        analyzerPerField.put(def.getEntityField(), new KeywordAnalyzer()) ;
+        if (def.getGraphField() != null)
+            analyzerPerField.put(def.getGraphField(), new KeywordAnalyzer()) ;
+        this.analyzer = new PerFieldAnalyzerWrapper(new StandardAnalyzer(VER), analyzerPerField) ;
+        
         // force creation of the index if it don't exist
         // otherwise if we get a search before data is written we get an exception
         startIndexing();
@@ -136,6 +148,13 @@ public class TextIndexLucene implements 
         Document doc = new Document() ;
         Field entField = new Field(docDef.getEntityField(), entity.getId(), ftIRI) ;
         doc.add(entField) ;
+
+        String graphField = docDef.getGraphField() ;
+        if ( graphField != null )
+        {
+            Field gField = new Field(graphField, entity.getGraph(), ftString) ;
+            doc.add(gField) ;
+        }
         
         for ( Entry<String, Object> e : entity.getMap().entrySet() )
         {

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndexSolr.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndexSolr.java?rev=1549557&r1=1549556&r2=1549557&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndexSolr.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextIndexSolr.java Mon Dec  9 11:59:51 2013
@@ -87,6 +87,13 @@ public class TextIndexSolr implements Te
     {
         SolrInputDocument doc = new SolrInputDocument() ;
         doc.addField(docDef.getEntityField(), entity.getId()) ;
+        
+        String graphField = docDef.getGraphField() ;
+        if ( graphField != null )
+        {
+            doc.addField(graphField, entity.getGraph()) ;
+        }
+        
         // the addition needs to be done as a partial update
         // otherwise, if we have multiple fields, each successive
         // addition will replace the previous one and we are left

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java?rev=1549557&r1=1549556&r2=1549557&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/TextQueryPF.java Mon Dec  9 11:59:51 2013
@@ -32,6 +32,8 @@ import com.hp.hpl.jena.datatypes.xsd.XSD
 import com.hp.hpl.jena.graph.Node ;
 import com.hp.hpl.jena.query.QueryBuildException ;
 import com.hp.hpl.jena.sparql.core.DatasetGraph ;
+import com.hp.hpl.jena.sparql.core.GraphView ;
+import com.hp.hpl.jena.sparql.core.Quad ;
 import com.hp.hpl.jena.sparql.core.Substitute ;
 import com.hp.hpl.jena.sparql.core.Var ;
 import com.hp.hpl.jena.sparql.engine.ExecutionContext ;
@@ -181,6 +183,18 @@ public class TextQueryPF extends Propert
     }
 
     private List<Node> query(String queryString, int limit, ExecutionContext execCxt) {
+        // use the graph information in the text index if possible
+        if (server.getDocDef().getGraphField() != null
+            && execCxt.getActiveGraph() instanceof GraphView) {
+            GraphView activeGraph = (GraphView)execCxt.getActiveGraph() ;
+            if (!Quad.isUnionGraph(activeGraph.getGraphName())) {
+                String uri = activeGraph.getGraphName() != null ? activeGraph.getGraphName().getURI() : Quad.defaultGraphNodeGenerated.getURI() ;
+                String escaped = QueryParser.escape(uri) ;
+                String qs2 = server.getDocDef().getGraphField() + ":" + escaped ;
+                queryString = "(" + queryString + ") AND " + qs2 ;
+            }
+        } 
+    
         Explain.explain(execCxt.getContext(), "Text query: "+queryString) ;
         if ( log.isDebugEnabled())
             log.debug("Text query: {} ({})", queryString,limit) ;

Modified: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/EntityMapAssembler.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/EntityMapAssembler.java?rev=1549557&r1=1549556&r2=1549557&view=diff
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/EntityMapAssembler.java (original)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/EntityMapAssembler.java Mon Dec  9 11:59:51 2013
@@ -62,7 +62,10 @@ public class EntityMapAssembler extends 
                                         "SELECT * {" ,
                                         "  ?eMap  :entityField  ?entityField ;" ,
                                         "         :map ?map ;",
-                                        "         :defaultField ?dftField" , 
+                                        "         :defaultField ?dftField ." ,
+                                        "  OPTIONAL {" ,
+                                        "    ?eMap :graphField ?graphField" ,
+                                        "  }",
                                         "}") ;
         ParameterizedSparqlString pss = new ParameterizedSparqlString(qs1) ;
         pss.setIri("eMap", root.getURI()) ;
@@ -83,7 +86,7 @@ public class EntityMapAssembler extends 
         
         QuerySolution qsol1 = results.get(0) ;
         String entityField = qsol1.getLiteral("entityField").getLexicalForm() ;
-        
+        String graphField = qsol1.contains("graphField") ? qsol1.getLiteral("graphField").getLexicalForm() : null;
         String defaultField = qsol1.contains("dftField") ? qsol1.getLiteral("dftField").getLexicalForm() : null ;
         
         String qs2 = StrUtils.strjoinNL("SELECT * { ?map list:member [ :field ?field ; :predicate ?predicate ] }") ;
@@ -107,7 +110,7 @@ public class EntityMapAssembler extends 
         }
         
         
-        EntityDefinition docDef = new EntityDefinition(entityField, defaultField) ;
+        EntityDefinition docDef = new EntityDefinition(entityField, defaultField, graphField) ;
         for ( String f : mapDefs.keys() ) {
             for ( Node p : mapDefs.get(f)) 
                 docDef.set(f, p) ;

Added: jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithGraphTextIndex.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithGraphTextIndex.java?rev=1549557&view=auto
==============================================================================
--- jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithGraphTextIndex.java (added)
+++ jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithGraphTextIndex.java Mon Dec  9 11:59:51 2013
@@ -0,0 +1,187 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.query.text;
+
+import java.io.Reader ;
+import java.io.StringReader ;
+import java.util.Arrays ;
+import java.util.HashSet ;
+import java.util.Set ;
+
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.riot.Lang ;
+import org.apache.jena.riot.RDFDataMgr ;
+import org.junit.Ignore ;
+import org.junit.Test ;
+
+import com.hp.hpl.jena.query.ReadWrite ;
+import com.hp.hpl.jena.rdf.model.Model ;
+
+/**
+ * This abstract class defines tests of the graph-specific indexing.
+ */
+public class AbstractTestDatasetWithGraphTextIndex extends AbstractTestDatasetWithTextIndex {
+
+    private void putTurtleInModel(String turtle, String modelName) {
+        Model model = modelName != null ? dataset.getNamedModel(modelName) : dataset.getDefaultModel() ;
+        Reader reader = new StringReader(turtle) ;
+        dataset.begin(ReadWrite.WRITE) ;
+        model.read(reader, "", "TURTLE") ;
+        dataset.commit() ;
+    }
+
+    @Ignore
+	@Test
+	public void testOneSimpleResultInGraph() {
+		final String turtle = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testOneSimpleResult>",
+				"  rdfs:label 'bar testOneSimpleResult barfoo foo'",
+				"."
+				);
+                putTurtleInModel(turtle, "http://example.org/modelA") ;
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"  GRAPH ?g { ?s text:query ( rdfs:label 'testOneSimpleResult' 10 ) . }",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		expectedURIs.addAll( Arrays.asList(RESOURCE_BASE + "testOneSimpleResult")) ;
+		doTestQuery(dataset, "", queryString, expectedURIs, expectedURIs.size()) ;
+	}
+
+	@Test
+	public void testOneResultTwoGraphs() {
+		final String turtleA = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testResultInModelA>",
+				"  rdfs:label 'bar testOneResult barfoo foo'",
+				"."
+				);
+                putTurtleInModel(turtleA, "http://example.org/modelA") ;
+		final String turtleB = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testResultInModelB>",
+				"  rdfs:label 'bar testOneResult barfoo foo'",
+				"."
+				);
+                putTurtleInModel(turtleB, "http://example.org/modelB") ;
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"  GRAPH <http://example.org/modelA> { ?s text:query ( rdfs:label 'testOneResult' 10 ) . }",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		expectedURIs.addAll( Arrays.asList(RESOURCE_BASE + "testResultInModelA")) ;
+		doTestQuery(dataset, "", queryString, expectedURIs, expectedURIs.size()) ;
+	}
+
+	@Test
+	public void testORFromGraphs() {
+		final String turtleA = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testResultOneInModelA>",
+				"  rdfs:label 'bar testResultOne barfoo foo'",
+				".",
+				"<" + RESOURCE_BASE + "testResultTwoInModelA>",
+				"  rdfs:label 'bar testResultTwo barfoo foo'",
+				".",
+				"<" + RESOURCE_BASE + "testResultThreeInModelA>",
+				"  rdfs:label 'bar testResultThree barfoo foo'",
+				"."
+				);
+                putTurtleInModel(turtleA, "http://example.org/modelA") ;
+		final String turtleB = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testResultOneInModelB>",
+				"  rdfs:label 'bar testResultOne barfoo foo'",
+				"."
+				);
+                putTurtleInModel(turtleB, "http://example.org/modelB") ;
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"  GRAPH <http://example.org/modelA> { ?s text:query ( rdfs:label 'testResultOne OR testResultTwo' 10 ) . }",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		expectedURIs.addAll( Arrays.asList(RESOURCE_BASE + "testResultOneInModelA", RESOURCE_BASE + "testResultTwoInModelA")) ;
+		doTestQuery(dataset, "", queryString, expectedURIs, expectedURIs.size()) ;
+	}
+
+	@Test
+	public void testQueryFromDefaultGraph() {
+		final String turtleA = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testResultInModelA>",
+				"  rdfs:label 'bar testOneResult barfoo foo'",
+				"."
+				);
+                putTurtleInModel(turtleA, null) ; // put in default graph
+		final String turtleB = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testResultInModelB>",
+				"  rdfs:label 'bar testOneResult barfoo foo'",
+				"."
+				);
+                putTurtleInModel(turtleB, "http://example.org/modelB") ;
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"    ?s text:query ( rdfs:label 'testOneResult' 10 ) .",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		expectedURIs.addAll( Arrays.asList(RESOURCE_BASE + "testResultInModelA")) ;
+		doTestQuery(dataset, "", queryString, expectedURIs, expectedURIs.size()) ;
+	}
+
+	@Ignore
+	@Test
+	public void testBnodeIdentifiedGraph() {
+		final String trig = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"_:graphA {",
+				"  <" + RESOURCE_BASE + "testResultInGraphA>",
+				"    rdfs:label 'bar testResult barfoo foo' .",
+				"}"
+				);
+                StringReader reader = new StringReader(trig);
+                dataset.begin(ReadWrite.WRITE) ;
+                RDFDataMgr.read(dataset.asDatasetGraph(), reader, "", Lang.TRIG);
+                dataset.commit();
+
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"  GRAPH ?g { ?s text:query ( rdfs:label 'testResult' 10 ) . }",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		expectedURIs.addAll( Arrays.asList(RESOURCE_BASE + "testResultInGraphA")) ;
+		doTestQuery(dataset, "", queryString, expectedURIs, expectedURIs.size()) ;
+	}
+}

Added: jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithLuceneGraphTextIndex.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithLuceneGraphTextIndex.java?rev=1549557&view=auto
==============================================================================
--- jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithLuceneGraphTextIndex.java (added)
+++ jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithLuceneGraphTextIndex.java Mon Dec  9 11:59:51 2013
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.query.text;
+
+import org.apache.lucene.store.Directory ;
+import org.apache.lucene.store.RAMDirectory ;
+
+import com.hp.hpl.jena.query.Dataset ;
+import com.hp.hpl.jena.tdb.TDBFactory ;
+import com.hp.hpl.jena.vocabulary.RDFS ;
+
+
+/**
+ * This abstract class defines a setup configuration for a dataset with a graph-enabled Lucene index.
+ */
+public class AbstractTestDatasetWithLuceneGraphTextIndex extends AbstractTestDatasetWithGraphTextIndex {
+
+	public static void init() {
+	        Dataset ds1 = TDBFactory.createDataset() ;
+                Directory dir = new RAMDirectory() ;
+                EntityDefinition eDef = new EntityDefinition("iri", "text", "graph", RDFS.label.asNode()) ;
+                eDef.set("comment", RDFS.comment.asNode()) ; // some tests require indexing rdfs:comment
+                TextIndex tidx = new TextIndexLucene(dir, eDef) ;
+                dataset = TextDatasetFactory.create(ds1, tidx) ;
+	}
+
+}

Modified: jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndex.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndex.java?rev=1549557&r1=1549556&r2=1549557&view=diff
==============================================================================
--- jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndex.java (original)
+++ jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndex.java Mon Dec  9 11:59:51 2013
@@ -40,7 +40,7 @@ import com.hp.hpl.jena.rdf.model.Model ;
  * the actual tests.
  */
 public abstract class AbstractTestDatasetWithTextIndex {
-	private static final String RESOURCE_BASE = "http://example.org/data/resource/";
+	protected static final String RESOURCE_BASE = "http://example.org/data/resource/";
 	protected static Dataset dataset;
 	protected static final String QUERY_PROLOG = 
 			StrUtils.strjoinNL(

Modified: jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java?rev=1549557&r1=1549556&r2=1549557&view=diff
==============================================================================
--- jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java (original)
+++ jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java Mon Dec  9 11:59:51 2013
@@ -30,6 +30,7 @@ import org.junit.runners.Suite.SuiteClas
 @SuiteClasses({
     TestBuildTextDataset.class
     , TestDatasetWithLuceneTextIndex.class
+    , TestDatasetWithLuceneGraphTextIndex.class
     // Embedded solr not supported 
     //, TestDatasetWithEmbeddedSolrTextIndex.class
     , TestSearchBeforeWriteOnDatasetWithLuceneTextIndex.class

Added: jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithLuceneGraphTextIndex.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithLuceneGraphTextIndex.java?rev=1549557&view=auto
==============================================================================
--- jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithLuceneGraphTextIndex.java (added)
+++ jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithLuceneGraphTextIndex.java Mon Dec  9 11:59:51 2013
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.query.text;
+
+import org.junit.BeforeClass ;
+
+public class TestDatasetWithLuceneGraphTextIndex extends AbstractTestDatasetWithLuceneGraphTextIndex {
+	
+	@BeforeClass public static void beforeClass() {
+		init();
+	}
+	
+}