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/04/05 12:51:34 UTC

svn commit: r1464922 [2/3] - in /jena/Experimental/jena-text: ./ src/ src/main/ src/main/java/ src/main/java/examples/ src/main/java/jena/ src/main/java/org/ src/main/java/org/apache/ src/main/java/org/apache/jena/ src/main/java/org/apache/jena/dsg/ sr...

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/QueryPF.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/QueryPF.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/QueryPF.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/QueryPF.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,256 @@
+/**
+ * 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.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 com.hp.hpl.jena.datatypes.RDFDatatype ;
+import com.hp.hpl.jena.datatypes.xsd.XSDDatatype ;
+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.Var ;
+import com.hp.hpl.jena.sparql.engine.ExecutionContext ;
+import com.hp.hpl.jena.sparql.engine.QueryIterator ;
+import com.hp.hpl.jena.sparql.engine.binding.Binding ;
+import com.hp.hpl.jena.sparql.engine.iterator.QueryIterExtendByVar ;
+import com.hp.hpl.jena.sparql.engine.iterator.QueryIterSlice ;
+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;
+
+/** property function that accesses a Solr server */ 
+public class QueryPF extends PropertyFunctionBase
+{
+    private TextIndex server = null ;  
+    private boolean warningIssued = false ;
+    
+    public QueryPF() { } 
+
+    @Override
+    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() )
+        {
+            List<Node> list = argObject.getArgList() ;
+            if ( list.size() == 0 )
+                throw new QueryBuildException("Zero-length argument 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)
+    {
+        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 ( dsg instanceof DatasetGraphText )
+        {
+            DatasetGraphText x = (DatasetGraphText)dsg ;
+            return x.getTextIndex() ;
+        }
+        Log.warn(QueryPF.class, "Failed to find the text index : tried context and as a text-enabled dataset") ;
+        return null ;
+    }
+
+    @Override
+    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() )
+            throw new InternalErrorException("Subject is not a node (it was earlier!)") ;
+            
+        Node s = argSubject.getArg() ;
+        
+        if ( s.isLiteral() )
+            // Does not match
+            return IterLib.noResults(execCxt) ;
+        
+        StrMatch match = objectToStruct(argObject) ;
+
+        // ----
+        
+        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 )
+    {
+        Var v = Var.alloc(s) ;
+        List<Node> r = server.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) ; 
+        }
+        
+        String uri = s.getURI() ;
+        Map<String, Node> x = server.get(uri) ;
+        if ( x == null || x.isEmpty() )
+            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() )
+        {
+            Node o = argObject.getArg() ;
+            
+            if ( ! o.isLiteral() )
+            { System.err.println("Bad/4") ; }
+            
+            RDFDatatype dt = o.getLiteralDatatype() ;
+            if ( dt != null && dt != XSDDatatype.XSDstring )
+            { System.err.println("Bad") ; }
+                
+            String qs = o.getLiteralLexicalForm() ;
+            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") ; 
+
+        Node p = docDef.getPrimaryPredicate() ;
+        String field = docDef.getPrimaryField() ;
+        int idx = 0 ;
+        Node x = list.get(0) ;
+        // Property?
+        if ( x.isURI() )
+        {
+            p = x ;
+            idx++ ;
+            if ( idx >= list.size() )
+                throw new TextIndexException("Property specificied but no query string : "+list) ;
+            x = list.get(idx) ;
+            field = docDef.getField(p) ; 
+        }
+        
+        // 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() ;  
+        idx++ ;
+        
+        int limit = -1 ;
+        float score = 0 ;
+        
+        if ( idx < list.size() )
+        {        
+            // Limit?
+            x = list.get(idx) ;
+            idx++ ;
+            int v = NodeFactoryExtra.nodeToInt(x) ;
+            limit = ( v < 0 ) ? -1 : v ; 
+        }
+
+        String qs = queryString ;
+        if ( field != null )
+            qs = field+":"+qs ;
+        
+        return new StrMatch(p, qs, limit, score) ;
+    }
+}
+
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDatasetFactory.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDatasetFactory.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDatasetFactory.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDatasetFactory.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,96 @@
+/**
+ * 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.jena.query.text.assembler.TextVocab ;
+import org.apache.lucene.store.Directory ;
+import org.apache.solr.client.solrj.SolrServer ;
+
+import com.hp.hpl.jena.query.Dataset ;
+import com.hp.hpl.jena.query.DatasetFactory ;
+import com.hp.hpl.jena.sparql.core.DatasetGraph ;
+import com.hp.hpl.jena.sparql.core.assembler.AssemblerUtils ;
+
+public class TextDatasetFactory
+{
+    /** Use an assembler file to build a dataset with text search capabilities */ 
+    public static Dataset create(String assemblerFile)
+    {
+        return (Dataset)AssemblerUtils.build(assemblerFile, TextVocab.textDataset) ;
+    }
+
+    /** Create a text-indexed dataset */ 
+    public static Dataset create(Dataset base, TextIndex index)
+    {
+        DatasetGraph dsg = base.asDatasetGraph() ;
+        dsg = create(dsg, index) ;
+        return DatasetFactory.create(dsg) ;
+    }
+
+
+    /** Create a text-indexed dataset */ 
+    public static DatasetGraph create(DatasetGraph dsg, TextIndex textIndex)
+    {
+        TextDocProducer producer = new TextDocProducerTriples(textIndex.getDocDef(), textIndex) ;
+        return new DatasetGraphText(dsg, textIndex, producer) ;
+    }
+    
+    /** Create a Lucene TextIndex */ 
+    public static TextIndex createLuceneIndex(Directory directory, EntityDefinition entMap)
+    {
+        TextIndex index = new TextIndexLucene(directory, entMap) ;
+        return index ; 
+    }
+
+    /** Create a text-indexed dataset, using Lucene */ 
+    public static Dataset createLucene(Dataset base, Directory directory, EntityDefinition entMap)
+    {
+        TextIndex index = createLuceneIndex(directory, entMap) ;
+        return create(base, index) ; 
+    }
+
+    /** Create a text-indexed dataset, using Lucene */ 
+    public static DatasetGraph createLucene(DatasetGraph base, Directory directory, EntityDefinition entMap)
+    {
+        TextIndex index = createLuceneIndex(directory, entMap) ;
+        return create(base, index) ; 
+    }
+
+    /** Create a Solr TextIndex */ 
+    public static TextIndex createSolrIndex(SolrServer server, EntityDefinition entMap)
+    {
+        TextIndex index = new TextIndexSolr(server, entMap) ;
+        return index ; 
+    }
+
+    /** Create a text-indexed dataset, using Solr */ 
+    public static Dataset createSolrIndex(Dataset base, SolrServer server, EntityDefinition entMap)
+    {
+        TextIndex index = createSolrIndex(server, entMap) ;
+        return create(base, index) ; 
+    }
+
+    /** Create a text-indexed dataset, using Solr */ 
+    public static DatasetGraph createSolrIndex(DatasetGraph base, SolrServer server, EntityDefinition entMap)
+    {
+        TextIndex index = createSolrIndex(server, entMap) ;
+        return create(base, index) ; 
+    }
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducer.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducer.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducer.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducer.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,27 @@
+/**
+ * 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.jena.dsg.DatasetChanges ;
+
+public interface TextDocProducer extends DatasetChanges
+{
+
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerEntities.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerEntities.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerEntities.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerEntities.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,133 @@
+/**
+ * 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.util.List ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.iterator.Transform ;
+import org.apache.jena.dsg.DatasetChangesBatched ;
+import org.slf4j.Logger ;
+import org.slf4j.LoggerFactory ;
+
+import com.hp.hpl.jena.graph.Node ;
+import com.hp.hpl.jena.graph.Triple ;
+import com.hp.hpl.jena.sparql.core.Quad ;
+import com.hp.hpl.jena.sparql.util.FmtUtils ;
+
+public class TextDocProducerEntities extends DatasetChangesBatched implements TextDocProducer
+{
+    private static Logger log = LoggerFactory.getLogger(TextDocProducer.class) ;
+    private final EntityDefinition defn ;
+    private final TextIndex indexer ;
+    private boolean started = false ;
+    
+    public TextDocProducerEntities(EntityDefinition defn, TextIndex indexer)
+    {
+        this.defn = defn ;
+        this.indexer = indexer ;
+    }
+    
+    @Override
+    protected void startBatched()
+    { indexer.startIndexing() ; started = true ;}
+
+    @Override
+    protected void finishBatched()
+    { indexer.finishIndexing() ; }
+
+    @Override
+    protected void dispatch(List<Quad> batch)
+    {
+        if ( ! started )
+            throw new IllegalStateException("Not started") ;
+        
+        if ( batch.size() == 0 )
+            return ;
+        Quad q = batch.get(0) ;
+        Node g = q.getGraph() ;
+        Node s = q.getSubject() ;
+        List<Triple> triples = quadsToTriples(batch) ;
+        //docEntity(s, triples) ;
+        docTriples(s,triples) ; // Does not need batching.
+    }
+
+    private void docEntity(Node s, List<Triple> batch)
+    {
+        // One document per entity
+        
+        String x = (s.isURI() ) ? s.getURI() : s.getBlankNodeLabel() ;
+        Entity entity = new Entity(x) ;
+        for ( Triple triple : batch )
+        {
+            Node p = triple.getPredicate() ;
+            String field = defn.getField(p) ;
+            if ( field == null )
+                continue ;
+            Node o = triple.getObject() ;
+            String val = null ;
+            if ( o.isURI() )
+                val = o.getURI() ;
+            else if ( o.isLiteral() )
+                val = o.getLiteralLexicalForm() ;
+            else
+            {
+                log.warn("Not a literal value for mapped field-predicate: "+field+" :: "+FmtUtils.stringForString(field)) ;
+                continue ;
+            }
+            entity.put(field, val) ;
+        }
+        indexer.addEntity(entity) ;
+    }
+
+    private void docTriples(Node s, List<Triple> batch)
+    {
+        String x = (s.isURI() ) ? s.getURI() : s.getBlankNodeLabel() ;
+        // One document per triple.
+        for ( Triple triple : batch )
+        {
+            Entity entity = new Entity(x) ;
+            Node p = triple.getPredicate() ;
+            String field = defn.getField(p) ;
+            if ( field == null )
+                continue ;
+            Node o = triple.getObject() ;
+            if ( ! o.isLiteral() )
+            {
+                log.warn("Not a literal value for mapped field-predicate: "+field+" :: "+FmtUtils.stringForString(field)) ;
+                continue ;
+            }
+            entity.put(field, o.getLiteralLexicalForm()) ;
+            indexer.addEntity(entity) ;
+        }
+    }
+
+    static Transform<Quad, Triple> QuadsToTriples = new Transform<Quad, Triple>() 
+    {
+        @Override
+        public Triple convert(Quad item)
+        {
+            return item.asTriple() ;
+        }
+        
+    } ;
+    
+    static private List<Triple> quadsToTriples(List<Quad> quads) { return Iter.map(quads, QuadsToTriples) ; } 
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerTriples.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerTriples.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerTriples.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextDocProducerTriples.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,91 @@
+/**
+ * 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.util.List ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.iterator.Transform ;
+import org.apache.jena.dsg.QuadAction ;
+import org.slf4j.Logger ;
+import org.slf4j.LoggerFactory ;
+
+import com.hp.hpl.jena.graph.Node ;
+import com.hp.hpl.jena.graph.Triple ;
+import com.hp.hpl.jena.sparql.core.Quad ;
+import com.hp.hpl.jena.sparql.util.FmtUtils ;
+
+public class TextDocProducerTriples implements TextDocProducer
+{
+    private static Logger log = LoggerFactory.getLogger(TextDocProducerTriples.class) ;
+    private final EntityDefinition defn ;
+    private final TextIndex indexer ;
+    private boolean started = false ;
+    
+    public TextDocProducerTriples(EntityDefinition defn, TextIndex indexer)
+    {
+        this.defn = defn ;
+        this.indexer = indexer ;
+    }
+    
+    @Override
+    public void start()
+    { indexer.startIndexing() ; started = true ;}
+
+    @Override
+    public void finish()
+    { indexer.finishIndexing() ; }
+
+    @Override
+    public void change(QuadAction qaction, Node g, Node s, Node p, Node o)
+    {
+        // One document per triple/quad
+        
+        if ( qaction != QuadAction.ADD)
+            return ;
+        
+        String field = defn.getField(p) ;
+        if ( field == null )
+            return ;
+
+        String x = (s.isURI() ) ? s.getURI() : s.getBlankNodeLabel() ;
+        Entity entity = new Entity(x) ;
+
+        if ( ! o.isLiteral() )
+        {
+            log.warn("Not a literal value for mapped field-predicate: "+field+" :: "+FmtUtils.stringForString(field)) ;
+            return ;
+        }
+        entity.put(field, o.getLiteralLexicalForm()) ;
+        indexer.addEntity(entity) ;
+    }
+
+    static Transform<Quad, Triple> QuadsToTriples = new Transform<Quad, Triple>() 
+    {
+        @Override
+        public Triple convert(Quad item)
+        {
+            return item.asTriple() ;
+        }
+        
+    } ;
+    
+    static private List<Triple> quadsToTriples(List<Quad> quads) { return Iter.map(quads, QuadsToTriples) ; } 
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndex.java Fri Apr  5 10:51:33 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 java.util.List ;
+import java.util.Map ;
+
+import org.apache.jena.atlas.lib.Closeable ;
+
+import com.hp.hpl.jena.graph.Node ;
+
+public interface TextIndex extends Closeable //, Transactional 
+{
+    public abstract void startIndexing() ;
+    public abstract void addEntity(Entity entity) ;
+    public abstract void finishIndexing() ;
+    public abstract void abortIndexing() ;
+    
+    public abstract Map<String, Node> get(String uri) ;
+
+    /** Access the index - limit if -1 for as many as possible */ 
+    public abstract List<Node> query(String qs, int limit) ;
+    
+    public abstract List<Node> query(String qs) ;
+
+    public abstract EntityDefinition getDocDef() ;
+}

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexException.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexException.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexException.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexException.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,31 @@
+/**
+ * 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 com.hp.hpl.jena.query.QueryExecException ;
+
+public class TextIndexException extends QueryExecException 
+{
+    public TextIndexException() { super() ; }
+    public TextIndexException(Throwable cause) { super(cause) ; }
+    public TextIndexException(String msg) { super(msg) ; }
+    public TextIndexException(String msg, Throwable cause) { super(msg, cause) ; }
+
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexLucene.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,240 @@
+/**
+ * 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.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 com.hp.hpl.jena.graph.Node;
+import com.hp.hpl.jena.graph.NodeFactory ;
+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 ;
+    
+    private final EntityDefinition docDef ;
+    private final Directory directory ;
+    private IndexWriter indexWriter ;
+    private Analyzer analyzer = new StandardAnalyzer(VER);
+    
+    
+    public TextIndexLucene(Directory directory, EntityDefinition def)
+    {
+        this.directory = directory ;
+        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
+        startIndexing();
+        finishIndexing();
+    }
+    
+    @Override
+    public void startIndexing()
+    { 
+        try {
+            IndexWriterConfig wConfig = new IndexWriterConfig(VER, analyzer) ;
+            indexWriter = new IndexWriter(directory, wConfig) ;
+        } catch (IOException e) { exception(e) ; }
+    }
+
+    @Override
+    public void finishIndexing()
+    {
+        try { indexWriter.commit() ; indexWriter.close() ; indexWriter = null ; }
+        catch (IOException e) { exception(e) ; }
+    }
+    
+    @Override
+    public void abortIndexing()
+    {
+        try { indexWriter.rollback() ; }
+        catch ( IOException ex) { exception(ex) ; }
+    }
+
+    @Override
+    public void close()
+    { 
+        if ( indexWriter != null ) 
+            try { indexWriter.close() ; } catch (IOException ex) { exception(ex) ; }
+    }
+
+    @Override
+    public void addEntity(Entity entity)
+    {
+        log.info("Add entity: "+entity) ;
+        try {
+            Document doc = doc(entity) ;
+            indexWriter.addDocument(doc) ;
+        } catch (Exception e) { exception(e) ; }
+    }
+
+    private Document doc(Entity entity)
+    {
+        Document doc = new Document() ;
+        Field entField = new Field(docDef.getEntityField(), entity.getId(), ftIndexedStored) ;
+        doc.add(entField) ;
+        
+        for ( Entry<String, Object> e : entity.getMap().entrySet() )
+        {
+            Field field = new Field(e.getKey(), (String)e.getValue(), ftIndexed) ;
+            doc.add(field) ;
+        }
+        return doc ;
+    }
+
+    @Override
+    public Map<String, Node> get(String uri)
+    {
+        try {
+            IndexReader indexReader = DirectoryReader.open(directory) ;
+            List<Map<String, Node>> x = get$(indexReader, uri) ;
+            if ( x.size() == 0)
+                return null ;
+//            if ( x.size() > 1)
+//                throw new TextIndexException("Multiple entires for "+uri) ;
+            return x.get(0) ;
+        } catch (Exception ex) { exception(ex) ; return null ; } 
+    }
+    
+    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.    
+        List<Map<String, Node>> records = new ArrayList<Map<String, Node>>() ;
+
+        // Align and DRY with Solr.
+        for ( ScoreDoc sd : sDocs )
+        {
+            Document doc = indexSearcher.doc(sd.doc) ;
+            String[] x = doc.getValues(docDef.getEntityField()) ;
+            if ( x.length != 1 )
+            {}
+            String uriStr = x[0] ;
+            Map<String, Node> record = new HashMap<String, Node>() ; 
+            Node entity = NodeFactory.createURI(uriStr) ;
+            record.put(docDef.getEntityField(), entity) ;
+                    
+            for ( String f : docDef.fields() )
+            {
+                //log.info("Field: "+f) ;
+                String[] values = doc.getValues(f) ;
+                for ( String v : values )
+                {
+                    Node n = entryToNode(v) ;
+                    record.put(f, n) ;
+                }
+                records.add(record) ;
+            }
+        }
+        return records ;
+    }
+
+
+    @Override
+    public List<Node> query(String qs) { return query(qs, MAX_N) ; } 
+    
+    @Override
+    public List<Node> query(String qs, int limit)
+    {
+        try {
+            // Upgrade at Java7 ... 
+            IndexReader indexReader = DirectoryReader.open(directory) ;
+            try { return query$(indexReader, qs, limit) ; } 
+            finally { indexReader.close() ; }
+        } catch (Exception ex) { exception(ex) ; return null ; } 
+    }
+        
+    public 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);
+        
+        if ( limit <= 0 )
+            limit = MAX_N ;
+        ScoreDoc[] sDocs = indexSearcher.search(query, limit).scoreDocs ;
+        
+        List<Node> results = new ArrayList<Node>() ;
+        
+        // Align and DRY with Solr.
+        for ( ScoreDoc sd : sDocs )
+        {
+            Document doc = indexSearcher.doc(sd.doc) ;
+            String[] values = doc.getValues(docDef.getEntityField()) ;
+            for ( String v : values )
+            {
+                Node n = NodeFactory.createURI(v);
+                results.add(n) ;
+            }
+        }
+        return results ;
+    }
+
+    
+    @Override
+    public EntityDefinition getDocDef()
+    {
+        return docDef ;
+    }
+
+    private Node entryToNode(String v)
+    {
+        // TEMP
+        return NodeFactoryExtra.createLiteralNode(v, null, null) ;
+    }
+
+    private static void exception(Exception ex)
+    {
+        throw new TextIndexException(ex) ;
+    }
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexSolr.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexSolr.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexSolr.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextIndexSolr.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,210 @@
+/**
+ * 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.util.* ;
+import java.util.Map.Entry ;
+
+import org.apache.solr.client.solrj.SolrQuery ;
+import org.apache.solr.client.solrj.SolrServer ;
+import org.apache.solr.client.solrj.SolrServerException ;
+import org.apache.solr.client.solrj.response.QueryResponse ;
+import org.apache.solr.client.solrj.util.ClientUtils ;
+import org.apache.solr.common.SolrDocument ;
+import org.apache.solr.common.SolrDocumentList ;
+import org.apache.solr.common.SolrInputDocument ;
+import org.slf4j.Logger ;
+import org.slf4j.LoggerFactory ;
+
+import com.hp.hpl.jena.graph.Node ;
+import com.hp.hpl.jena.graph.NodeFactory ;
+import com.hp.hpl.jena.sparql.util.NodeFactoryExtra ;
+
+public class TextIndexSolr implements TextIndex
+{
+    private static Logger log = LoggerFactory.getLogger(TextIndexSolr.class) ;
+    private final SolrServer solrServer ;
+    private EntityDefinition docDef ;
+
+    public TextIndexSolr(SolrServer server, EntityDefinition def)
+    {
+        this.solrServer = server ;
+        this.docDef = def ;
+    }
+    
+    @Override
+    public void startIndexing()
+    {}
+
+    @Override
+    public void finishIndexing()
+    {
+        try { solrServer.commit() ; }
+        catch (Exception ex) { exception(ex) ; }
+    }
+
+    @Override
+    public void abortIndexing()
+    {
+        try { solrServer.rollback() ; }
+        catch (Exception ex) { exception(ex) ; }
+    }
+    
+    @Override
+    public void close()
+    { 
+        if ( solrServer != null ) 
+            solrServer.shutdown() ;
+    }
+
+
+
+    @Override
+    public void addEntity(Entity entity)
+    {
+        //log.info("Add entity: "+entity) ;
+        try {
+            SolrInputDocument doc = solrDoc(entity) ;
+            solrServer.add(doc) ;
+        } catch (Exception e) { exception(e) ; }
+    }
+
+    private SolrInputDocument solrDoc(Entity entity)
+    {
+        SolrInputDocument doc = new SolrInputDocument() ;
+        doc.addField(docDef.getEntityField(), entity.getId()) ;
+        // 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
+        // with only the last field indexed.
+        // see http://stackoverflow.com/questions/12183798/solrj-api-for-partial-document-update
+        // and https://svn.apache.org/repos/asf/lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
+    	HashMap<String,Object> map = new HashMap<String,Object>();
+        for ( Entry<String, Object> e : entity.getMap().entrySet() ) {
+        	map.put("add", e.getValue());
+            doc.addField(e.getKey(), map) ;
+        }
+        return doc ;
+    }
+
+    @Override
+    public Map<String, Node> get(String uri)
+    {
+        String escaped = ClientUtils.escapeQueryChars(uri) ;
+        String qs = docDef.getEntityField()+":"+escaped ; 
+        SolrDocumentList solrResults = solrQuery(qs,1) ;
+        
+        List<Map<String, Node>> records = process(solrResults) ;
+        if ( records.size() == 0 )
+            return null ;
+        if ( records.size() > 1 )
+            log.warn("Multiple docs for one URI: "+uri) ;
+        return records.get(0) ;
+    }
+    
+    private List<Map<String, Node>> process(SolrDocumentList solrResults)
+    {
+        List<Map<String, Node>> records = new ArrayList<Map<String, Node>>() ;
+        
+        for ( SolrDocument sd : solrResults )
+        {
+            Map<String, Node> record = new HashMap<String, Node>() ; 
+            String uriStr = (String)sd.getFieldValue(docDef.getEntityField()) ;
+            Node entity = NodeFactory.createURI(uriStr) ;
+            record.put(docDef.getEntityField(), entity) ;
+            
+            for ( String f : docDef.fields() )
+            {
+                //log.info("Field: "+f) ;
+                Object obj = sd.getFieldValue(f) ;
+                //log.info("Value: "+obj) ;
+                if ( obj == null )
+                    continue ;
+                // Multivalued -> array.
+                // Null means "not stored" or "not present" 
+                if ( obj instanceof List<?> )
+                {
+                    @SuppressWarnings("unchecked")
+                    List<String> vals = (List<String>)obj ;
+                    continue ;
+                }
+
+                String v = (String)obj ;
+                Node n = entryToNode(v) ;
+                record.put(f, n) ;
+            }
+            
+            //log.info("Entity: "+uriStr) ;
+            records.add(record) ;
+        }
+        return records ;
+    }
+    
+    @Override
+    public List<Node> query(String qs) { return query(qs, 0) ; } 
+    
+    @Override
+    public List<Node> query(String qs, int limit)
+    {
+        SolrDocumentList solrResults = solrQuery(qs, limit) ;
+        List<Node> results = new ArrayList<Node>() ;
+
+        for ( SolrDocument sd : solrResults )
+        {
+            String uriStr = (String)sd.getFieldValue(docDef.getEntityField()) ;
+            //log.info("Entity: "+uriStr) ;
+            results.add(NodeFactory.createURI(uriStr)) ;
+        }
+
+        if ( limit > 0 && results.size() > limit )
+            results = results.subList(0, limit) ;
+        
+        return results ; 
+    }
+    
+    private SolrDocumentList solrQuery(String qs, int limit)
+    {
+        SolrQuery sq = new SolrQuery(qs) ;
+        try {
+            QueryResponse rsp = solrServer.query( sq ) ;
+            SolrDocumentList docs = rsp.getResults();
+            return docs ;
+        } catch (SolrServerException e) { exception(e) ; return null ; }
+    }
+
+    @Override
+    public EntityDefinition getDocDef()
+    {
+        return docDef ;
+    }
+
+    private Node entryToNode(String v)
+    {
+        // TEMP
+        return NodeFactoryExtra.createLiteralNode(v, null, null) ;
+    }
+    
+    public SolrServer getServer() { return solrServer ; }
+
+    private static Void exception(Exception ex)
+    {
+        throw new TextIndexException(ex) ;
+    }
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextQuery.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextQuery.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextQuery.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/TextQuery.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,60 @@
+/**
+ * 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.jena.atlas.logging.Log ;
+import org.apache.jena.query.text.assembler.TextAssembler ;
+
+import com.hp.hpl.jena.sparql.pfunction.PropertyFunction ;
+import com.hp.hpl.jena.sparql.pfunction.PropertyFunctionFactory ;
+import com.hp.hpl.jena.sparql.pfunction.PropertyFunctionRegistry ;
+import com.hp.hpl.jena.sparql.util.Symbol ;
+import com.hp.hpl.jena.tdb.TDB ;
+
+public class TextQuery
+{
+    private static boolean initialized = false ;
+    private static Object lock = new Object() ;
+    public static String NS = "http://jena.apache.org/text#" ;
+    public static final Symbol textIndex = Symbol.create(NS+"index") ;
+    
+    static { init() ; }
+    
+    public static void init() 
+    {
+        if ( initialized ) return ;
+        synchronized(lock)
+        {
+            if ( initialized ) return ;
+            initialized = true ;
+            Log.info(TextQuery.class, "Initialized") ;
+            TDB.init() ;
+            TextAssembler.init() ;
+
+            PropertyFunctionRegistry.get().put("http://jena.apache.org/text#query", new PropertyFunctionFactory() {
+                @Override
+                public PropertyFunction create(String uri)
+                {
+                    return new QueryPF() ;
+                }
+            });
+        }
+    }
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/EntityMapAssembler.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/EntityMapAssembler.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/EntityMapAssembler.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/EntityMapAssembler.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,119 @@
+/**
+ * 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 static org.apache.jena.query.text.assembler.TextVocab.NS ;
+
+import java.util.List ;
+
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.query.text.EntityDefinition ;
+import org.apache.jena.query.text.TextIndexException ;
+
+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.graph.Node ;
+import com.hp.hpl.jena.query.* ;
+import com.hp.hpl.jena.rdf.model.Model ;
+import com.hp.hpl.jena.rdf.model.Resource ;
+
+public class EntityMapAssembler extends AssemblerBase implements Assembler
+{
+    
+    // V1
+    
+    /*
+<#entMap> a text:EntityMap ;
+    text:entityField      "uri" ;
+    text:defaultField     "text" ;
+    text:map (
+         [ text:field "text" ; text:predicate rdfs:label ]
+         [ text:field "type" ; text:predicate rdfs:type  ]
+         ) .
+      */
+    
+    @Override
+    public EntityDefinition open(Assembler a, Resource root, Mode mode)
+    {
+        String prologue = "PREFIX : <"+NS+">   PREFIX list: <http://jena.hpl.hp.com/ARQ/list#> " ;
+        Model model = root.getModel() ;
+        
+        
+        // ParameterizedSparqlString
+//        String qs1 = StrUtils.strjoinNL("SELECT * {",
+//                                       "  <"+root.getURI()+">    :entityField  ?entityField ;",
+//                                       "                         :defaultField ?dftField 
+//                                       "          :map ?map ;",
+//                                       "  OPTIONAL { <"+root.getURI()+"> :defaultField ?dftField } ",
+//                                       "}") ;
+        
+        String qs1 = StrUtils.strjoinNL(prologue,
+                                        "SELECT * {" ,
+                                        "  ?eMap  :entityField  ?entityField ;" ,
+                                        "         :defaultField ?dftField ;" , 
+                                        "         :map ?map" ,
+                                         "}") ;
+        ParameterizedSparqlString pss = new ParameterizedSparqlString(qs1) ;
+        pss.setIri("eMap", root.getURI()) ;
+        
+        Query query1 = QueryFactory.create(pss.toString()) ;
+        QueryExecution qexec1 = QueryExecutionFactory.create(query1, model) ;
+        ResultSet rs1 = qexec1.execSelect() ;
+        List<QuerySolution> results = ResultSetFormatter.toList(rs1) ;
+        if ( results.size() !=1 )
+            throw new IllegalStateException() ;
+        
+        QuerySolution qsol1 = results.get(0) ;
+        String entityField = qsol1.getLiteral("entityField").getLexicalForm() ;
+        String defaultField = qsol1.contains("dftField") ? qsol1.getLiteral("dftField").getLexicalForm() : null ;
+        
+        String qs2 = StrUtils.strjoinNL("SELECT * {",
+                                        "  ?map list:member [ :field ?field ; :predicate ?predicate ]" ,
+                                        "}") ;
+        Query query2 = QueryFactory.create(prologue+" "+qs2) ;
+        QueryExecution qexec2 = QueryExecutionFactory.create(query2, model, qsol1) ;
+        ResultSet rs2 = qexec2.execSelect() ;
+        List<QuerySolution> mapEntries = ResultSetFormatter.toList(rs2) ;
+        // Find primary entity property.
+        Node primaryProperty = null ;
+        for ( QuerySolution qsol : mapEntries )
+        {
+            String field =  qsol.getLiteral("field").getLexicalForm() ;
+            if ( defaultField.equals(field) )
+            {
+                primaryProperty = qsol.getResource("predicate").asNode() ;
+                break ;
+            }
+        }
+        
+        if ( primaryProperty == null )
+            throw new TextIndexException("No definition of primary field '"+defaultField+"'") ; 
+        
+        EntityDefinition docDef = new EntityDefinition(entityField, defaultField, null) ;
+        for ( QuerySolution qsol : mapEntries )
+        {
+            String field =  qsol.getLiteral("field").getLexicalForm() ;
+            Resource p = qsol.getResource("predicate") ;
+            docDef.set(field, p.asNode()) ;
+        }
+        return docDef ;
+    }
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextAssembler.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextAssembler.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextAssembler.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextAssembler.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,36 @@
+/**
+ * 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 com.hp.hpl.jena.assembler.Assembler ;
+import com.hp.hpl.jena.sparql.core.assembler.AssemblerUtils ;
+
+public class TextAssembler
+{
+    public static void init()
+    {
+        AssemblerUtils.init() ;
+        Assembler.general.implementWith(TextVocab.textDataset,      new TextDatasetAssembler()) ;
+        Assembler.general.implementWith(TextVocab.entityMap,        new EntityMapAssembler()) ;
+        Assembler.general.implementWith(TextVocab.textIndexSolr,    new TextIndexSolrAssembler()) ;
+        Assembler.general.implementWith(TextVocab.textIndexLucene,  new TextIndexLuceneAssembler()) ;
+
+    }
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextDatasetAssembler.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextDatasetAssembler.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextDatasetAssembler.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextDatasetAssembler.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,66 @@
+/**
+ * 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 static org.apache.jena.query.text.assembler.TextVocab.pDataset ;
+import static org.apache.jena.query.text.assembler.TextVocab.pIndex ;
+import static org.apache.jena.query.text.assembler.TextVocab.textDataset ;
+import org.apache.jena.query.text.TextDatasetFactory ;
+import org.apache.jena.query.text.TextIndex ;
+import org.apache.jena.query.text.TextQuery ;
+
+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.query.Dataset ;
+import com.hp.hpl.jena.rdf.model.Resource ;
+import com.hp.hpl.jena.sparql.core.assembler.DatasetAssembler ;
+import com.hp.hpl.jena.sparql.util.graph.GraphUtils ;
+
+public class TextDatasetAssembler extends AssemblerBase implements Assembler
+{
+    private DatasetAssembler datasetAssembler = new DatasetAssembler() ;
+    
+    public static Resource getType() { return textDataset ; }
+        
+    /*
+<#text_dataset> rdf:type     text:Dataset ;
+    text:dataset <#dataset> ;
+    text:index   <#index> ;
+    .
+
+    */
+    
+    @Override
+    public Dataset open(Assembler a, Resource root, Mode mode)
+    {
+        //Log.info(TextDatasetAssembler.class, "Text dataset index") ;
+        Resource dataset = GraphUtils.getResourceValue(root, pDataset) ;
+        Resource index   = GraphUtils.getResourceValue(root, pIndex) ;
+        
+        Dataset ds = (Dataset)a.open(dataset) ;
+        TextIndex textIndex = (TextIndex)a.open(index) ;
+        
+        Dataset dst = TextDatasetFactory.create(ds, textIndex) ;
+        dst.getContext().set(TextQuery.textIndex, textIndex) ;
+        return dst ;
+        
+    }
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexLuceneAssembler.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexLuceneAssembler.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexLuceneAssembler.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexLuceneAssembler.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,71 @@
+/**
+ * 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 static org.apache.jena.query.text.assembler.TextVocab.pDirectory ;
+import static org.apache.jena.query.text.assembler.TextVocab.pEntityMap ;
+
+import java.io.File ;
+import java.io.IOException ;
+
+import org.apache.jena.atlas.io.IO ;
+import org.apache.jena.atlas.lib.IRILib ;
+import org.apache.jena.atlas.logging.Log ;
+import org.apache.jena.query.text.EntityDefinition ;
+import org.apache.jena.query.text.TextDatasetFactory ;
+import org.apache.jena.query.text.TextIndex ;
+import org.apache.lucene.store.Directory ;
+import org.apache.lucene.store.FSDirectory ;
+
+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 ;
+import com.hp.hpl.jena.sparql.util.graph.GraphUtils ;
+
+public class TextIndexLuceneAssembler extends AssemblerBase
+{
+    /*
+    <#index> a :TextIndexLucene ;
+        #text:directory "mem" ;
+        text:directory <file:DIR> ;
+        text:entityMap <#endMap> ;
+        .
+    */
+
+    @Override
+    public TextIndex open(Assembler a, Resource root, Mode mode)
+    {
+        try
+        {
+            Resource x = GraphUtils.getResourceValue(root, pDirectory) ;
+            String path = IRILib.IRIToFilename(x.getURI()) ; 
+            File dir = new File(path) ; 
+            Directory directory = FSDirectory.open(dir) ;
+            Log.info(TextIndexLuceneAssembler.class, "Lucene text index : "+dir) ;
+        
+            Resource r = GraphUtils.getResourceValue(root, pEntityMap) ;
+            EntityDefinition docDef = (EntityDefinition)a.open(r) ; 
+            
+            return TextDatasetFactory.createLuceneIndex(directory, docDef) ;
+        } catch (IOException e) { IO.exception(e) ; return null ;}
+    }
+
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexSolrAssembler.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexSolrAssembler.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexSolrAssembler.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextIndexSolrAssembler.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,69 @@
+/**
+ * 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 static org.apache.jena.query.text.assembler.TextVocab.pEntityMap ;
+import static org.apache.jena.query.text.assembler.TextVocab.pServer ;
+import org.apache.jena.query.text.* ;
+import org.apache.solr.client.solrj.SolrServer ;
+import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer ;
+import org.apache.solr.client.solrj.impl.HttpSolrServer ;
+import org.apache.solr.core.CoreContainer ;
+
+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 ;
+import com.hp.hpl.jena.sparql.util.graph.GraphUtils ;
+
+public class TextIndexSolrAssembler extends AssemblerBase
+{
+    /*
+    <#index> a :TextIndexSolr ;
+        #:server <http://localhost:8983/solr/COLLECTION> ;
+        text:server <embedded:SolrARQ> ;
+        text:entityMap <#endMap> ;
+        .
+    */
+
+    @Override
+    public TextIndex open(Assembler a, Resource root, Mode mode)
+    {
+        String uri = GraphUtils.getResourceValue(root, pServer).getURI() ;
+        SolrServer server ;
+        if ( uri.startsWith("embedded:") )
+        {
+            String coreName = uri.substring("embedded:".length()) ;
+            CoreContainer.Initializer initializer = new CoreContainer.Initializer();
+            CoreContainer coreContainer = initializer.initialize();
+            server = new EmbeddedSolrServer(coreContainer, coreName);
+        }   
+        else if ( uri.startsWith("http://") )
+        {
+            server = new HttpSolrServer( uri );
+        }
+        else
+            throw new TextIndexException("URI for the server must begin 'http://' or 'embedded:'") ;
+        
+        Resource r = GraphUtils.getResourceValue(root, pEntityMap) ;
+        EntityDefinition docDef = (EntityDefinition)a.open(r) ; 
+        return TextDatasetFactory.createSolrIndex(server, docDef) ;
+    }
+}
+

Added: jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextVocab.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextVocab.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextVocab.java (added)
+++ jena/Experimental/jena-text/src/main/java/org/apache/jena/query/text/assembler/TextVocab.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,52 @@
+/**
+ * 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 com.hp.hpl.jena.rdf.model.Property ;
+import com.hp.hpl.jena.rdf.model.Resource ;
+import com.hp.hpl.jena.tdb.assembler.Vocab ;
+
+public class TextVocab
+{
+    public static final String NS                   =  "http://jena.apache.org/text#" ;
+    
+    public static final String pfQuery              =  NS+"query" ;
+
+    public static final Resource textDataset        = Vocab.resource(NS, "TextDataset") ;
+    public static final Property pDataset           = Vocab.property(NS, "dataset") ;
+    public static final Property pIndex             = Vocab.property(NS, "index") ;
+    
+    public static final Resource textIndex          = Vocab.resource(NS, "TextIndex") ;
+    public static final Resource textIndexSolr      = Vocab.resource(NS, "TextIndexSolr") ;
+    public static final Resource textIndexLucene    = Vocab.resource(NS, "TextIndexLucene") ;
+    public static final Property pServer            = Vocab.property(NS, "server") ;            // Solr
+    public static final Property pDirectory         = Vocab.property(NS, "directory") ;         // Lucene
+    public static final Property pEntityMap         = Vocab.property(NS, "entityMap") ;
+    
+    // Entity definition
+    public static final Resource entityMap          = Vocab.resource(NS, "EntityMap") ;
+    public static final Property pEntityField       = Vocab.property(NS, "entityField") ;
+    public static final Property pDefaultField      = Vocab.property(NS, "defaultField") ;
+    public static final Property pMap               = Vocab.property(NS, "map") ;
+    public static final Property pField             = Vocab.property(NS, "field") ;
+    public static final Property pPredicate         = Vocab.property(NS, "predicate") ;
+    public static final Property pOptional          = Vocab.property(NS, "optional") ;
+
+}
+

Added: jena/Experimental/jena-text/src/test/java/jena/TestTextIndexer.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/test/java/jena/TestTextIndexer.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/test/java/jena/TestTextIndexer.java (added)
+++ jena/Experimental/jena-text/src/test/java/jena/TestTextIndexer.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,177 @@
+/**
+ * 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 jena;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+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.AbstractTestDatasetWithTextIndex;
+import org.apache.jena.query.text.TextSearchUtil;
+import org.apache.jena.query.text.assembler.TextAssembler;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import arq.cmd.CmdException;
+
+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;
+import com.hp.hpl.jena.tdb.TDBFactory;
+import com.hp.hpl.jena.util.FileUtils;
+
+public class TestTextIndexer {
+	private static final String RESOURCE_BASE = "http://example.org/data/resource/";
+	private static final String INDEX_PATH = "target/test/simpleLuceneIndex";
+	private static final File indexDir = new File(INDEX_PATH);
+	private static final String TDB_PATH = "target/test/tdb";
+	private static final File tdbDir = new File(TDB_PATH);
+	private static final String SPEC_BASE = "http://example.org/specbase/";
+	private static final String SPEC_ROOT_LOCAL = "spec";
+	private static final String SPEC_ROOT_URI = SPEC_BASE + SPEC_ROOT_LOCAL;
+	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#> ."
+						);
+
+	private static final String SPEC_PATH = "test/asbl-tdbWithTextIndex.ttl";
+	
+	@Before public void before() {
+		after();
+		indexDir.mkdirs();
+	}
+	
+	@After public void after() {
+		TextSearchUtil.emptyAndDeleteDirectory(indexDir);
+		TextSearchUtil.emptyAndDeleteDirectory(tdbDir);
+	}
+	
+	@Test public void testDetectsNoDataset() {
+		try {
+			textindexer.testMain( new String[] {} );
+			fail("should have thrown an exception with no dataset");
+		} catch (CmdException e) {}		
+	}
+	
+	@Test public void testDetectsNotTextIndexedDataset() {
+		try {
+			textindexer.testMain( new String[] 
+		        {
+				    "--desc",
+				 	"test/asbl-memNoTextIndex.ttl"
+			    } );
+			fail("should have thrown an exception with no dataset");
+		} catch (CmdException e) {
+			assertTrue("wrong exception: " + e.getMessage(), e.getMessage().contains("no text index"));
+		}		
+	}
+	
+	@Test public void testIndexEmptyDataset() {
+		textindexer.testMain( new String[] 
+		    {
+			    "--desc",
+			 	"test/asbl-tdbWithTextIndex.ttl"
+			} );
+	}
+	
+	@Test public void testIndexAndQueryDataset() throws FileNotFoundException {
+		String label = "testIndexAndQueryDataset";
+		final String turtle = StrUtils.strjoinNL(
+				TURTLE_PROLOG,
+				"<" + RESOURCE_BASE + label + "1>",
+				"  rdfs:label \"" + label + " label innnnnnnnnn\" ;", // in is not indexed
+				"  rdfs:comment \"" + label + " comment out\" ;",
+				".",
+				"<" + RESOURCE_BASE + label + "2>",
+				"  rdfs:label \"" + label + " label out\" ;",
+				"  rdfs:comment \"" + label + " comment out\" ;",
+				".",
+				"<" + RESOURCE_BASE + label + "3>",
+				"  rdfs:label \"" + label + " label out\" ;",
+				"  rdfs:comment \"" + label + " comment innnnnnnnnn\" ;",
+				".",
+				"<" + RESOURCE_BASE + label + "4>",
+				"  rdfs:label \"" + label + " label out\" ;",
+				"  rdfs:comment \"" + label + " comment out\" ;",
+				"."
+				);
+		String queryString = StrUtils.strjoinNL(
+				QUERY_PROLOG,
+				"SELECT ?s",
+				"WHERE {",
+				"    {",
+				"        ?s text:query ( rdfs:label \"innnnnnnnnn\") .",
+				"    } UNION {",
+				"        ?s text:query ( rdfs:comment \"innnnnnnnnn\" ) .",
+				"    }",
+				"}"
+				);
+		Set<String> expectedURIs = (new HashSet<String>());
+		expectedURIs.addAll( Arrays.asList((
+				new String[]
+				{
+					    RESOURCE_BASE + label + "1",
+					    RESOURCE_BASE + label + "3",
+				}
+		)));
+		
+		Dataset dataset = TDBFactory.createDataset(TDB_PATH) ;
+		Model model = dataset.getDefaultModel();
+		Reader reader = new StringReader(turtle);
+		model.read(reader, "", "TURTLE");
+		model.close() ;
+		dataset.close();
+		
+		textindexer.testMain( new String[] 
+		    {
+			    "--desc",
+			 	"test/asbl-tdbWithTextIndex.ttl"
+			} );
+		
+		AbstractTestDatasetWithTextIndex.doTestQuery(getDataset(), label, queryString, expectedURIs, expectedURIs.size());
+	}
+	
+	private Dataset getDataset() throws FileNotFoundException {
+		Reader reader = new FileReader(SPEC_PATH);
+		Model specModel = ModelFactory.createDefaultModel();
+		specModel.read(reader, FileUtils.toURL(SPEC_PATH), "TURTLE");
+		TextAssembler.init();
+		Resource root = specModel.getResource(SPEC_ROOT_URI);
+		return (Dataset) Assembler.general.open(root);
+	}
+}

Added: jena/Experimental/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithLuceneTextIndex.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithLuceneTextIndex.java?rev=1464922&view=auto
==============================================================================
--- jena/Experimental/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithLuceneTextIndex.java (added)
+++ jena/Experimental/jena-text/src/test/java/org/apache/jena/query/text/AbstractTestDatasetWithLuceneTextIndex.java Fri Apr  5 10:51:33 2013
@@ -0,0 +1,103 @@
+/**
+ * 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 org.apache.jena.atlas.lib.StrUtils;
+import org.apache.jena.query.text.assembler.TextAssembler;
+
+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 abstract class defines a setup configuration for a dataset with a Lucene index.
+ */
+public class AbstractTestDatasetWithLuceneTextIndex extends AbstractTestDatasetWithTextIndex {
+	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: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);
+	}
+}