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 2014/03/26 13:07:23 UTC

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

Author: andy
Date: Wed Mar 26 12:07:22 2014
New Revision: 1581801

URL: http://svn.apache.org/r1581801
Log:
JENA-654 : Configurable Lucene Analyzer for index

Added:
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/KeywordAnalyzerAssembler.java
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/SimpleAnalyzerAssembler.java
    jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/StandardAnalyzerAssembler.java
    jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndexBase.java
    jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithKeywordAnalyzer.java
    jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithSimpleAnalyzer.java
    jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithStandardAnalyzer.java

Added: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/KeywordAnalyzerAssembler.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/KeywordAnalyzerAssembler.java?rev=1581801&view=auto
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/KeywordAnalyzerAssembler.java (added)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/KeywordAnalyzerAssembler.java Wed Mar 26 12:07:22 2014
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.query.text.assembler ;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.KeywordAnalyzer;
+
+import com.hp.hpl.jena.assembler.Assembler;
+import com.hp.hpl.jena.assembler.Mode;
+import com.hp.hpl.jena.assembler.assemblers.AssemblerBase;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * Assembler to create keyword analyzers.
+ */
+public class KeywordAnalyzerAssembler extends AssemblerBase {
+    /*
+    text:map (
+         [ text:field "text" ; 
+           text:predicate rdfs:label;
+           text:analyzer [
+               a  lucene:KeywordAnalyzer ;           ]
+         ]
+        .
+    */
+
+    @Override
+    public Analyzer open(Assembler a, Resource root, Mode mode) {
+    	return new KeywordAnalyzer();
+    }
+}

Added: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/SimpleAnalyzerAssembler.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/SimpleAnalyzerAssembler.java?rev=1581801&view=auto
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/SimpleAnalyzerAssembler.java (added)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/SimpleAnalyzerAssembler.java Wed Mar 26 12:07:22 2014
@@ -0,0 +1,49 @@
+/**
+ * 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.assembler ;
+
+import org.apache.jena.query.text.TextIndexLucene;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.SimpleAnalyzer;
+
+import com.hp.hpl.jena.assembler.Assembler;
+import com.hp.hpl.jena.assembler.Mode;
+import com.hp.hpl.jena.assembler.assemblers.AssemblerBase;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * Assembler to create simple analyzers.
+ */
+public class SimpleAnalyzerAssembler extends AssemblerBase {
+    /*
+    text:map (
+         [ text:field "text" ; 
+           text:predicate rdfs:label;
+           text:analyzer [
+               a  lucene:SimpleAnalyzer ;
+           ]
+         ]
+        .
+    */
+
+    @Override
+    public Analyzer open(Assembler a, Resource root, Mode mode) {
+    	return new SimpleAnalyzer(TextIndexLucene.VER);
+    }
+}

Added: jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/StandardAnalyzerAssembler.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/StandardAnalyzerAssembler.java?rev=1581801&view=auto
==============================================================================
--- jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/StandardAnalyzerAssembler.java (added)
+++ jena/trunk/jena-text/src/main/java/org/apache/jena/query/text/assembler/StandardAnalyzerAssembler.java Wed Mar 26 12:07:22 2014
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.query.text.assembler ;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.query.text.TextIndexException;
+import org.apache.jena.query.text.TextIndexLucene;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.analysis.util.CharArraySet;
+
+import com.hp.hpl.jena.assembler.Assembler;
+import com.hp.hpl.jena.assembler.Mode;
+import com.hp.hpl.jena.assembler.assemblers.AssemblerBase;
+import com.hp.hpl.jena.rdf.model.Literal;
+import com.hp.hpl.jena.rdf.model.RDFNode;
+import com.hp.hpl.jena.rdf.model.Resource;
+import com.hp.hpl.jena.rdf.model.Statement;
+import com.hp.hpl.jena.vocabulary.RDF;
+
+/**
+ * Assembler to create standard analyzers with keyword list.
+ */
+public class StandardAnalyzerAssembler extends AssemblerBase {
+    /*
+    text:map (
+         [ text:field "text" ; 
+           text:predicate rdfs:label;
+           text:analyzer [
+               a  lucene:StandardAnalyzer ;
+               text:stopWords ("foo" "bar" "baz") # optional
+           ]
+         ]
+        .
+    */
+
+    @Override
+    public Analyzer open(Assembler a, Resource root, Mode mode) {
+    	if (root.hasProperty(TextVocab.pStopWords)) {
+    		return analyzerWithStopWords(root);
+    	} else {
+    		return new StandardAnalyzer(TextIndexLucene.VER);
+    	}
+    }
+    
+    private Analyzer analyzerWithStopWords(Resource root) {
+    	RDFNode node = root.getProperty(TextVocab.pStopWords).getObject();
+    	if (! node.isResource()) {
+    		throw new TextIndexException("text:stopWords property takes a list as a value : " + node);
+    	}
+    	CharArraySet stopWords = toCharArraySet((Resource) node);
+    	return new StandardAnalyzer(TextIndexLucene.VER, stopWords);
+    }
+    
+    private CharArraySet toCharArraySet(Resource list) {
+    	return new CharArraySet(TextIndexLucene.VER, toList(list), false);
+    }
+    
+    private List<String> toList(Resource list) {
+    	List<String> result = new ArrayList<String>();
+    	Resource current = list;
+    	while (current != null && ! current.equals(RDF.nil)){
+    		Statement stmt = current.getProperty(RDF.first);
+    		if (stmt == null) {
+    			throw new TextIndexException("stop word list not well formed");
+    		}
+    		RDFNode node = stmt.getObject();
+    		if (! node.isLiteral()) {
+    			throw new TextIndexException("stop word is not a literal : " + node);
+    		}
+    		result.add(((Literal)node).getLexicalForm());
+    		stmt = current.getProperty(RDF.rest);
+    		if (stmt == null) {
+    			throw new TextIndexException("stop word list not terminated by rdf:nil");
+    		}
+    		node = stmt.getObject();
+    		if (! node.isResource()) {
+    			throw new TextIndexException("stop word list node is not a resource : " + node);
+    		}
+    		current = (Resource) node;
+    	}
+    	return result;
+    }
+}

Added: jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndexBase.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndexBase.java?rev=1581801&view=auto
==============================================================================
--- jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndexBase.java (added)
+++ jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithTextIndexBase.java Wed Mar 26 12:07:22 2014
@@ -0,0 +1,93 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.Reader;
+import java.io.StringReader;
+import java.util.Set;
+
+import org.apache.jena.atlas.lib.StrUtils;
+
+import com.hp.hpl.jena.query.Dataset;
+import com.hp.hpl.jena.query.Query;
+import com.hp.hpl.jena.query.QueryExecution;
+import com.hp.hpl.jena.query.QueryExecutionFactory;
+import com.hp.hpl.jena.query.QueryFactory;
+import com.hp.hpl.jena.query.ReadWrite;
+import com.hp.hpl.jena.query.ResultSet;
+import com.hp.hpl.jena.rdf.model.Model;
+
+/*
+ * This abstract class defines a collection of test methods for testing
+ * test searches.  Its subclasses create a dataset using the index to 
+ * to be tested and then call the test methods in this class to run
+ * the actual tests.
+ */
+public abstract class AbstractTestDatasetWithTextIndexBase {
+	protected static final String RESOURCE_BASE = "http://example.org/data/resource/";
+	protected static Dataset dataset;
+	protected static final String QUERY_PROLOG = 
+			StrUtils.strjoinNL(
+				"PREFIX text: <http://jena.apache.org/text#>",
+				"PREFIX rdfs: <http://www.w3.org/2000/01/rdf-schema#>"
+				);
+	
+	protected static final String TURTLE_PROLOG = 
+				StrUtils.strjoinNL(
+						"@prefix text: <http://jena.apache.org/text#> .",
+						"@prefix rdfs: <http://www.w3.org/2000/01/rdf-schema#> ."
+						);
+	
+	protected void doTestSearch(String turtle, String queryString, Set<String> expectedEntityURIs) {
+		doTestSearch("", turtle, queryString, expectedEntityURIs);
+	}
+	
+	protected void doTestSearch(String label, String turtle, String queryString, Set<String> expectedEntityURIs) {
+		doTestSearch(label, turtle, queryString, expectedEntityURIs, expectedEntityURIs.size());
+	}
+	
+	protected void doTestSearch(String label, String turtle, String queryString, Set<String> expectedEntityURIs, int expectedNumResults) {
+		Model model = dataset.getDefaultModel();
+		Reader reader = new StringReader(turtle);
+		dataset.begin(ReadWrite.WRITE);
+		model.read(reader, "", "TURTLE");
+		dataset.commit();
+		doTestQuery(dataset, label, queryString, expectedEntityURIs, expectedNumResults);
+	}
+	
+	public static void doTestQuery(Dataset dataset, String label, String queryString, Set<String> expectedEntityURIs, int expectedNumResults) {
+		Query query = QueryFactory.create(queryString) ;
+		QueryExecution qexec = QueryExecutionFactory.create(query, dataset) ;
+		try {
+			dataset.begin(ReadWrite.READ);
+		    ResultSet results = qexec.execSelect() ;
+		    
+		    assertEquals(label, expectedNumResults > 0, results.hasNext());
+		    int count;
+		    for (count=0; results.hasNext(); count++) {
+		    	String entityURI = results.next().getResource("s").getURI();
+		        assertTrue(label + ": unexpected result: " + entityURI, expectedEntityURIs.contains(entityURI));
+		    }
+		    assertEquals(label, expectedNumResults, count);
+		} finally { qexec.close() ; dataset.end() ; }		
+	}
+}

Added: jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithKeywordAnalyzer.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithKeywordAnalyzer.java?rev=1581801&view=auto
==============================================================================
--- jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithKeywordAnalyzer.java (added)
+++ jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithKeywordAnalyzer.java Wed Mar 26 12:07:22 2014
@@ -0,0 +1,161 @@
+/**
+ * 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.File;
+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.query.text.assembler.TextAssembler;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.hp.hpl.jena.assembler.Assembler;
+import com.hp.hpl.jena.query.Dataset;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * This class defines a setup configuration for a dataset that uses a keyword analyzer with a Lucene index.
+ */
+public class TestDatasetWithKeywordAnalyzer extends AbstractTestDatasetWithTextIndexBase {
+	private static final String INDEX_PATH = "target/test/TestDatasetWithLuceneIndex";
+	private static final File indexDir = new File(INDEX_PATH);
+	
+	private static final String SPEC_BASE = "http://example.org/spec#";
+	private static final String SPEC_ROOT_LOCAL = "lucene_text_dataset";
+	private static final String SPEC_ROOT_URI = SPEC_BASE + SPEC_ROOT_LOCAL;
+	private static final String SPEC;
+	static {
+	    SPEC = StrUtils.strjoinNL(
+					"prefix rdfs: <http://www.w3.org/2000/01/rdf-schema#> ",
+					"prefix ja:   <http://jena.hpl.hp.com/2005/11/Assembler#> ",
+					"prefix tdb:  <http://jena.hpl.hp.com/2008/tdb#>",
+					"prefix text: <http://jena.apache.org/text#>",
+					"prefix :     <" + SPEC_BASE + ">",
+					"",
+					"[] ja:loadClass    \"org.apache.jena.query.text.TextQuery\" .",
+				    "text:TextDataset      rdfs:subClassOf   ja:RDFDataset .",
+				    "text:TextIndexLucene  rdfs:subClassOf   text:TextIndex .",
+				    
+				    ":" + SPEC_ROOT_LOCAL,
+				    "    a              text:TextDataset ;",
+				    "    text:dataset   :dataset ;",
+				    "    text:index     :indexLucene ;",
+				    "    .",
+				    "",
+                    ":dataset",
+                    "    a               ja:RDFDataset ;",
+                    "    ja:defaultGraph :graph ;",
+                    ".",
+                    ":graph",
+                    "    a               ja:MemoryModel ;",
+                    ".",
+                    "",
+				    ":indexLucene",
+                    "    a text:TextIndexLucene ;",
+				    "    text:directory <file:" + INDEX_PATH + "> ;",
+				    "    text:entityMap :entMap ;",
+				    "    .",
+                    "",
+				    ":entMap",
+                    "    a text:EntityMap ;",
+				    "    text:entityField      \"uri\" ;",
+				    "    text:defaultField     \"label\" ;",
+				    "    text:map (",
+				    "         [ text:field \"label\" ; ",
+				    "           text:predicate rdfs:label ;",
+				    "           text:analyzer [ a text:KeywordAnalyzer ]",
+				    "         ]",
+				    "         [ text:field \"comment\" ; text:predicate rdfs:comment ]",
+				    "         ) ."
+				    );
+	}      
+	
+	public static void init() {
+		Reader reader = new StringReader(SPEC);
+		Model specModel = ModelFactory.createDefaultModel();
+		specModel.read(reader, "", "TURTLE");
+		TextAssembler.init();			
+		deleteOldFiles();
+		indexDir.mkdirs();
+		Resource root = specModel.getResource(SPEC_ROOT_URI);
+		dataset = (Dataset) Assembler.general.open(root);
+	}
+	
+	
+	public static void deleteOldFiles() {
+		if (indexDir.exists()) TextSearchUtil.emptyAndDeleteDirectory(indexDir);
+	}	
+
+	@BeforeClass public static void beforeClass() {
+		init();
+	}	
+	
+	@AfterClass public static void afterClass() {
+		deleteOldFiles();
+	}
+	
+	@Test
+	public void testKeywordAnalyzerDoesNotSplitTokensAtSpace() {
+		final String testName = "testKeywordAnalyzerDoesNotSplitTokensAtSpace";
+		final String turtle = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + testName + ">",
+				"  rdfs:label 'EC1V 9BE'",
+				"."
+				);
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"    ?s text:query ( rdfs:label 'EC1V' 10 ) .",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		doTestSearch(turtle, queryString, expectedURIs);
+	}
+	
+	@Test
+	public void testKeywordAnalyzerMatchesWholeField() {
+		final String testName = "testKeywordAnalyzerMatchesWholeField";
+		final String turtle = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + testName + ">",
+				"  rdfs:label 'EC2V 9BE'",
+				"."
+				);
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"    ?s text:query ( rdfs:label '\"EC2V 9BE\"' 10 ) .",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		expectedURIs.addAll( Arrays.asList(RESOURCE_BASE + testName)) ;
+		doTestSearch(turtle, queryString, expectedURIs);
+	}
+}

Added: jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithSimpleAnalyzer.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithSimpleAnalyzer.java?rev=1581801&view=auto
==============================================================================
--- jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithSimpleAnalyzer.java (added)
+++ jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithSimpleAnalyzer.java Wed Mar 26 12:07:22 2014
@@ -0,0 +1,141 @@
+/**
+ * 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.File;
+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.query.text.assembler.TextAssembler;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.hp.hpl.jena.assembler.Assembler;
+import com.hp.hpl.jena.query.Dataset;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * This class defines a setup configuration for a dataset that uses a simple analyzer with a Lucene index.
+ */
+public class TestDatasetWithSimpleAnalyzer extends AbstractTestDatasetWithTextIndexBase {
+	private static final String INDEX_PATH = "target/test/TestDatasetWithLuceneIndex";
+	private static final File indexDir = new File(INDEX_PATH);
+	
+	private static final String SPEC_BASE = "http://example.org/spec#";
+	private static final String SPEC_ROOT_LOCAL = "lucene_text_dataset";
+	private static final String SPEC_ROOT_URI = SPEC_BASE + SPEC_ROOT_LOCAL;
+	private static final String SPEC;
+	static {
+	    SPEC = StrUtils.strjoinNL(
+					"prefix rdfs: <http://www.w3.org/2000/01/rdf-schema#> ",
+					"prefix ja:   <http://jena.hpl.hp.com/2005/11/Assembler#> ",
+					"prefix tdb:  <http://jena.hpl.hp.com/2008/tdb#>",
+					"prefix text: <http://jena.apache.org/text#>",
+					"prefix :     <" + SPEC_BASE + ">",
+					"",
+					"[] ja:loadClass    \"org.apache.jena.query.text.TextQuery\" .",
+				    "text:TextDataset      rdfs:subClassOf   ja:RDFDataset .",
+				    "text:TextIndexLucene  rdfs:subClassOf   text:TextIndex .",
+				    
+				    ":" + SPEC_ROOT_LOCAL,
+				    "    a              text:TextDataset ;",
+				    "    text:dataset   :dataset ;",
+				    "    text:index     :indexLucene ;",
+				    "    .",
+				    "",
+                    ":dataset",
+                    "    a               ja:RDFDataset ;",
+                    "    ja:defaultGraph :graph ;",
+                    ".",
+                    ":graph",
+                    "    a               ja:MemoryModel ;",
+                    ".",
+                    "",
+				    ":indexLucene",
+                    "    a text:TextIndexLucene ;",
+				    "    text:directory <file:" + INDEX_PATH + "> ;",
+				    "    text:entityMap :entMap ;",
+				    "    .",
+                    "",
+				    ":entMap",
+                    "    a text:EntityMap ;",
+				    "    text:entityField      \"uri\" ;",
+				    "    text:defaultField     \"label\" ;",
+				    "    text:map (",
+				    "         [ text:field \"label\" ; ",
+				    "           text:predicate rdfs:label ;",
+				    "           text:analyzer [ a text:SimpleAnalyzer ]",
+				    "         ]",
+				    "         [ text:field \"comment\" ; text:predicate rdfs:comment ]",
+				    "         ) ."
+				    );
+	}      
+	
+	public static void init() {
+		Reader reader = new StringReader(SPEC);
+		Model specModel = ModelFactory.createDefaultModel();
+		specModel.read(reader, "", "TURTLE");
+		TextAssembler.init();			
+		deleteOldFiles();
+		indexDir.mkdirs();
+		Resource root = specModel.getResource(SPEC_ROOT_URI);
+		dataset = (Dataset) Assembler.general.open(root);
+	}
+	
+	
+	public static void deleteOldFiles() {
+		if (indexDir.exists()) TextSearchUtil.emptyAndDeleteDirectory(indexDir);
+	}	
+
+	@BeforeClass public static void beforeClass() {
+		init();
+	}	
+	
+	@AfterClass public static void afterClass() {
+		deleteOldFiles();
+	}
+	
+	@Test
+	public void testSimpleAnalyzer() {
+		final String turtle = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testSimpleAnalyzer>",
+				"  rdfs:label 'bar the barfoo foo'",
+				"."
+				);
+		// the simple analyzer should not filter out the 'the' word
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"    ?s text:query ( rdfs:label 'the' 10 ) .",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		expectedURIs.addAll( Arrays.asList("http://example.org/data/resource/testSimpleAnalyzer")) ;
+		doTestSearch(turtle, queryString, expectedURIs);
+	}
+}

Added: jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithStandardAnalyzer.java
URL: http://svn.apache.org/viewvc/jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithStandardAnalyzer.java?rev=1581801&view=auto
==============================================================================
--- jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithStandardAnalyzer.java (added)
+++ jena/trunk/jena-text/src/test/java/org/apache/jena/query/text/TestDatasetWithStandardAnalyzer.java Wed Mar 26 12:07:22 2014
@@ -0,0 +1,161 @@
+/**
+ * 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.File;
+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.query.text.assembler.TextAssembler;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.hp.hpl.jena.assembler.Assembler;
+import com.hp.hpl.jena.query.Dataset;
+import com.hp.hpl.jena.rdf.model.Model;
+import com.hp.hpl.jena.rdf.model.ModelFactory;
+import com.hp.hpl.jena.rdf.model.Resource;
+
+/**
+ * This class defines a setup configuration for a dataset that uses a standard analyzer with a Lucene index.
+ */
+public class TestDatasetWithStandardAnalyzer extends AbstractTestDatasetWithTextIndexBase {
+	private static final String INDEX_PATH = "target/test/TestDatasetWithLuceneIndex";
+	private static final File indexDir = new File(INDEX_PATH);
+	
+	private static final String SPEC_BASE = "http://example.org/spec#";
+	private static final String SPEC_ROOT_LOCAL = "lucene_text_dataset";
+	private static final String SPEC_ROOT_URI = SPEC_BASE + SPEC_ROOT_LOCAL;
+	private static final String SPEC;
+	static {
+	    SPEC = StrUtils.strjoinNL(
+					"prefix rdfs: <http://www.w3.org/2000/01/rdf-schema#> ",
+					"prefix ja:   <http://jena.hpl.hp.com/2005/11/Assembler#> ",
+					"prefix tdb:  <http://jena.hpl.hp.com/2008/tdb#>",
+					"prefix text: <http://jena.apache.org/text#>",
+					"prefix :     <" + SPEC_BASE + ">",
+					"",
+					"[] ja:loadClass    \"org.apache.jena.query.text.TextQuery\" .",
+				    "text:TextDataset      rdfs:subClassOf   ja:RDFDataset .",
+				    "text:TextIndexLucene  rdfs:subClassOf   text:TextIndex .",
+				    
+				    ":" + SPEC_ROOT_LOCAL,
+				    "    a              text:TextDataset ;",
+				    "    text:dataset   :dataset ;",
+				    "    text:index     :indexLucene ;",
+				    "    .",
+				    "",
+                    ":dataset",
+                    "    a               ja:RDFDataset ;",
+                    "    ja:defaultGraph :graph ;",
+                    ".",
+                    ":graph",
+                    "    a               ja:MemoryModel ;",
+                    ".",
+                    "",
+				    ":indexLucene",
+                    "    a text:TextIndexLucene ;",
+				    "    text:directory <file:" + INDEX_PATH + "> ;",
+				    "    text:entityMap :entMap ;",
+				    "    .",
+                    "",
+				    ":entMap",
+                    "    a text:EntityMap ;",
+				    "    text:entityField      \"uri\" ;",
+				    "    text:defaultField     \"label\" ;",
+				    "    text:map (",
+				    "         [ text:field \"label\" ; ",
+				    "           text:predicate rdfs:label ;",
+				    "           text:analyzer [ a text:StandardAnalyzer ; text:stopWords ( 'foo'  'bar' ) ]",
+				    "         ]",
+				    "         [ text:field \"comment\" ; text:predicate rdfs:comment ]",
+				    "         ) ."
+				    );
+	}      
+	
+	public static void init() {
+		Reader reader = new StringReader(SPEC);
+		Model specModel = ModelFactory.createDefaultModel();
+		specModel.read(reader, "", "TURTLE");
+		TextAssembler.init();			
+		deleteOldFiles();
+		indexDir.mkdirs();
+		Resource root = specModel.getResource(SPEC_ROOT_URI);
+		dataset = (Dataset) Assembler.general.open(root);
+	}
+	
+	
+	public static void deleteOldFiles() {
+		if (indexDir.exists()) TextSearchUtil.emptyAndDeleteDirectory(indexDir);
+	}	
+
+	@BeforeClass public static void beforeClass() {
+		init();
+	}	
+	
+	@AfterClass public static void afterClass() {
+		deleteOldFiles();
+	}
+	
+	@Test
+	public void testStandardAnalyzerWithSpecifiedStopWords() {
+		final String turtle = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testOneSimpleResult>",
+				"  rdfs:label 'bar the barfoo foo'",
+				"."
+				);
+		// the standard analyzer not to have 'the' as a stop word
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"    ?s text:query ( rdfs:label 'the' 10 ) .",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		expectedURIs.addAll( Arrays.asList("http://example.org/data/resource/testOneSimpleResult")) ;
+		doTestSearch(turtle, queryString, expectedURIs);
+	}
+	
+	@Test
+	public void testStandardAnalyzerIgnoresSpecifiedStopWords() {
+		final String turtle = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + "testOneSimpleResult>",
+				"  rdfs:label 'bar the barfoo foo'",
+				"."
+				);
+		// the standard analyzer not to have 'the' as a stop word
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"    ?s text:query ( rdfs:label 'foo' 10 ) .",
+				"}"
+				);
+		Set<String> expectedURIs = new HashSet<String>() ;
+		doTestSearch(turtle, queryString, expectedURIs);
+	}
+}