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 2018/05/11 11:45:19 UTC

[5/6] jena git commit: JENA-1542, JENA-1302: Integrate jena-text with TDB1 and TDB2 transactions.

JENA-1542, JENA-1302: Integrate jena-text with TDB1 and TDB2 transactions.


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

Branch: refs/heads/master
Commit: 2d6330f4712c26390213861f3891126f1ae69e2d
Parents: 27ad5b7
Author: Andy Seaborne <an...@apache.org>
Authored: Sun May 6 14:24:45 2018 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Sun May 6 15:10:01 2018 +0100

----------------------------------------------------------------------
 .../jena/sparql/core/NamedGraphWrapper.java     |  11 +-
 .../transaction/txn/TransactionCoordinator.java |  14 +-
 .../org/apache/jena/tdb2/sys/TDBInternal.java   |   8 +
 .../org/apache/jena/tdb/sys/TDBInternal.java    |   8 +
 .../jena/tdb/transaction/Transaction.java       |   9 -
 .../jena/query/text/DatasetGraphText.java       | 158 +++++----
 .../org/apache/jena/query/text/TextIndexDB.java |  80 +++++
 .../apache/jena/query/text/TextIndexTDB1.java   |  60 ++++
 .../org/apache/jena/query/text/TS_Text.java     |   8 +-
 .../apache/jena/query/text/TestTextNonTxn.java  | 224 ++++++++++++
 .../jena/query/text/TestTextNonTxnTDB1.java     | 273 +++++++++++++++
 .../org/apache/jena/query/text/TestTextTDB.java | 272 ---------------
 .../org/apache/jena/query/text/TestTextTxn.java | 341 +++++++++++++++++++
 .../apache/jena/query/text/TestTextTxnTDB.java  | 329 ++++++++++++++++++
 14 files changed, 1445 insertions(+), 350 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-arq/src/main/java/org/apache/jena/sparql/core/NamedGraphWrapper.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/NamedGraphWrapper.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/NamedGraphWrapper.java
index 7b826c6..110d1ea 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/core/NamedGraphWrapper.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/NamedGraphWrapper.java
@@ -30,15 +30,20 @@ import org.apache.jena.sparql.graph.GraphWrapper ;
 
 public class NamedGraphWrapper extends GraphWrapper implements NamedGraph {
 
-    private final Node graphName ;
+    private final Node graphName;
 
     public NamedGraphWrapper(Node graphName, Graph graph) {
-        super(graph) ;
+        super(graph);
         this.graphName = graphName;
     }
 
     @Override
     public Node getGraphName() {
-        return graphName ;
+        return graphName;
+    }
+    
+    @Override
+    public String toString() {
+        return "NamedGraphWrapper("+graphName+")";
     }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-db/jena-dboe-transaction/src/main/java/org/apache/jena/dboe/transaction/txn/TransactionCoordinator.java
----------------------------------------------------------------------
diff --git a/jena-db/jena-dboe-transaction/src/main/java/org/apache/jena/dboe/transaction/txn/TransactionCoordinator.java b/jena-db/jena-dboe-transaction/src/main/java/org/apache/jena/dboe/transaction/txn/TransactionCoordinator.java
index cf99afe..cef7dcf 100644
--- a/jena-db/jena-dboe-transaction/src/main/java/org/apache/jena/dboe/transaction/txn/TransactionCoordinator.java
+++ b/jena-db/jena-dboe-transaction/src/main/java/org/apache/jena/dboe/transaction/txn/TransactionCoordinator.java
@@ -131,7 +131,7 @@ public class TransactionCoordinator {
      * This must be setup before recovery is attempted. 
      */
     public TransactionCoordinator add(TransactionalComponent elt) {
-        checklAllowModication() ;
+        checklAllowModification() ;
         components.add(elt) ;
         return this ;
     }
@@ -141,7 +141,7 @@ public class TransactionCoordinator {
      * @see #add 
      */
     public TransactionCoordinator remove(TransactionalComponent elt) {
-        checklAllowModication() ;
+        checklAllowModification() ;
         components.remove(elt.getComponentId()) ;
         return this ;
     }
@@ -172,23 +172,23 @@ public class TransactionCoordinator {
      * and hence hooks may not get called.
      */
     public void add(TransactionCoordinator.ShutdownHook hook) {
-        checklAllowModication() ;
+        checklAllowModification() ;
         shutdownHooks.add(hook) ;
     }
 
     /** Remove a shutdown hook */
     public void remove(TransactionCoordinator.ShutdownHook hook) {
-        checklAllowModication() ;
+        checklAllowModification() ;
         shutdownHooks.remove(hook) ;
     }
     
     public void setQuorumGenerator(QuorumGenerator qGen) {
-        checklAllowModication() ;
+        checklAllowModification() ;
         this.quorumGenerator = qGen ;
     }
 
     public void start() {
-        checklAllowModication() ;
+        checklAllowModification() ;
         recovery() ;
         configurable = false ;
     }
@@ -283,7 +283,7 @@ public class TransactionCoordinator {
     }
 
     // Can modifications be made? 
-    private void checklAllowModication() {
+    private void checklAllowModification() {
         if ( ! configurable )
             throw new TransactionException("TransactionCoordinator configuration is locked") ;
     }

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/sys/TDBInternal.java
----------------------------------------------------------------------
diff --git a/jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/sys/TDBInternal.java b/jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/sys/TDBInternal.java
index 8957185..56c3f40 100644
--- a/jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/sys/TDBInternal.java
+++ b/jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/sys/TDBInternal.java
@@ -38,6 +38,14 @@ import org.apache.jena.tdb2.store.nodetable.NodeTable;
  * 
  */
 public class TDBInternal {
+    
+    /**
+     * Return true if this is a TDB2 backed DatasetGraph. 
+     */
+    public static boolean isTDB2(DatasetGraph dsg) {
+        return ( dsg instanceof DatasetGraphSwitchable );
+    }
+
     /**
      * Return the NodeId for a node. Returns NodeId.NodeDoesNotExist when the node is not
      * found. Returns null when not a TDB-backed dataset.

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-tdb/src/main/java/org/apache/jena/tdb/sys/TDBInternal.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/sys/TDBInternal.java b/jena-tdb/src/main/java/org/apache/jena/tdb/sys/TDBInternal.java
index c674782..c241bf8 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/sys/TDBInternal.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/sys/TDBInternal.java
@@ -41,6 +41,13 @@ import org.apache.jena.tdb.transaction.TransactionManager ;
 public class TDBInternal
 {
     /**
+     * Return true if this is a TDB1 backed DatasetGraph. 
+     */
+    public static boolean isTDB1(DatasetGraph dsg) {
+        return ( dsg instanceof DatasetGraphTransaction );
+    }
+
+    /**
      * Return the NodeId for a node. Returns NodeId.NodeDoesNotExist when the
      * node is not found. Returns null when not a TDB-backed dataset.
      */
@@ -169,4 +176,5 @@ public class TDBInternal
                 return false ;
             return true ;
         } ;
+
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java
index 4d07836..138aab3 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/transaction/Transaction.java
@@ -332,15 +332,6 @@ public class Transaction
         }
     }
     
-//    /** Return the list of items registered for the transaction lifecycle */ 
-//    public List<TransactionLifecycle> lifecycleComponents() {
-//        List<TransactionLifecycle> x = new ArrayList<>() ;
-//        x.addAll(objectFileTrans) ;
-//        x.addAll(blkMgrs) ;
-//        x.addAll(others);
-//        return x ;
-//    }
-    
     // For development and tracking, keep these as separate lists.
     
     /*package*/ void addComponent(ObjectFileTrans oft) {

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-text/src/main/java/org/apache/jena/query/text/DatasetGraphText.java
----------------------------------------------------------------------
diff --git a/jena-text/src/main/java/org/apache/jena/query/text/DatasetGraphText.java b/jena-text/src/main/java/org/apache/jena/query/text/DatasetGraphText.java
index a1dcb92..63276dc 100644
--- a/jena-text/src/main/java/org/apache/jena/query/text/DatasetGraphText.java
+++ b/jena-text/src/main/java/org/apache/jena/query/text/DatasetGraphText.java
@@ -21,11 +21,18 @@ package org.apache.jena.query.text ;
 import java.util.Iterator ;
 import java.util.List ;
 
+import org.apache.jena.dboe.transaction.txn.ComponentId;
+import org.apache.jena.dboe.transaction.txn.TransactionCoordinator;
+import org.apache.jena.dboe.transaction.txn.TransactionalComponent;
 import org.apache.jena.graph.Graph ;
 import org.apache.jena.graph.Node ;
 import org.apache.jena.query.ReadWrite ;
 import org.apache.jena.query.TxnType;
-import org.apache.jena.sparql.core.* ;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.DatasetGraphMonitor;
+import org.apache.jena.sparql.core.GraphView;
+import org.apache.jena.sparql.core.Transactional;
+import org.apache.jena.tdb.transaction.TransactionManager;
 import org.apache.lucene.queryparser.classic.QueryParserBase ;
 import org.slf4j.Logger ;
 import org.slf4j.LoggerFactory ;
@@ -37,25 +44,65 @@ public class DatasetGraphText extends DatasetGraphMonitor implements Transaction
     private final Graph         dftGraph ;
     private final boolean       closeIndexOnClose;
     // Lock needed for commit/abort that perform an index operation and a dataset operation
-    // which need it happen without a W thread coming in between them.
-    // JENA-1302.
+    // when the underlying datsetGraph does not coordinate the commit.
     private final Object        txnExitLock = new Object();
     
     // If we are going to implement Transactional, then we are going to have to do as DatasetGraphWithLock and
     // TDB's DatasetGraphTransaction do and track transaction state in a ThreadLocal
     private final ThreadLocal<ReadWrite> readWriteMode = new ThreadLocal<>();
     
-    public DatasetGraphText(DatasetGraph dsg, TextIndex index, TextDocProducer producer)
-    { 
+    private Runnable delegateCommit = ()-> {
+        super.commit();
+    };
+    
+    private Runnable delegateAbort = ()-> {
+        super.abort();
+    };
+    
+    private Runnable nonDelegatedCommit = ()-> {
+        if (readWriteMode.get() == ReadWrite.WRITE)
+            commit_W();
+        else
+            commit_R();
+    };
+    
+    private Runnable nonDelegatedAbort = ()-> {
+        if (readWriteMode.get() == ReadWrite.WRITE)
+            abort_W();
+        else
+            abort_R();
+    };
+
+    private Runnable commitAction = null;
+    private Runnable abortAction = null;
+    
+    public DatasetGraphText(DatasetGraph dsg, TextIndex index, TextDocProducer producer) {
         this(dsg, index, producer, false);
     }
-    
-    public DatasetGraphText(DatasetGraph dsg, TextIndex index, TextDocProducer producer, boolean closeIndexOnClose)
-    {
+
+    public DatasetGraphText(DatasetGraph dsg, TextIndex index, TextDocProducer producer, boolean closeIndexOnClose) {
         super(dsg, producer) ;
         this.textIndex = index ;
         dftGraph = GraphView.createDefaultGraph(this) ;
         this.closeIndexOnClose = closeIndexOnClose;
+        
+        if ( org.apache.jena.tdb.sys.TDBInternal.isTDB1(dsg) ) {
+            TransactionManager txnMgr = org.apache.jena.tdb.sys.TDBInternal.getTransactionManager(dsg);
+            txnMgr.addAdditionComponent(new TextIndexTDB1(textIndex));
+            commitAction = delegateCommit;
+            abortAction = delegateAbort;
+        } else if ( org.apache.jena.tdb2.sys.TDBInternal.isTDB2(dsg) ) {
+            TransactionCoordinator coord = org.apache.jena.tdb2.sys.TDBInternal.getTransactionCoordinator(dsg);
+            // Does not overlap with the ids used by TDB2.
+            byte[] componentID = { 2,4,6,10 } ;
+            TransactionalComponent tc = new TextIndexDB(ComponentId.create(null, componentID), textIndex);
+            coord.modify(()->coord.add(tc));
+            commitAction = delegateCommit;
+            abortAction = delegateAbort;
+        } else {
+            commitAction = nonDelegatedCommit;
+            abortAction = nonDelegatedAbort;
+        }
     }
 
     // ---- Intercept these and force the use of views.
@@ -128,48 +175,27 @@ public class DatasetGraphText extends DatasetGraphMonitor implements Transaction
         super.getMonitor().start() ;
     }
     
-    // JENA-1302 :: txnExitLock
-    // We need to 
-    //   textIndex.prepareCommit();
-    //   super.commit();
-    //   textIndex.commit();
-    // without another thread getting in.
+    @Override
+    public void commit() {
+        super.getMonitor().finish() ;
+        commitAction.run();
+        readWriteMode.set(null);
+    }
     
-    // Concurrency control most of the time is because we use the transaction
-    // capability of the wrapped dataset but here we need to do an action before
-    // wrapped dataset commit and also an action after.
-    // 
-    // At the point of super.commit, it let in a new writer in begin() which
-    // races to commit before text index commit.
-    //
-    // txnExitLock extends the time of exclusive access.    
     
     /**
-     * Perform a 2-phase commit by first calling prepareCommit() on the TextIndex
-     * followed by committing the Transaction object, and then calling commit()
-     * on the TextIndex().
-     * <p> 
-     * If either of the objects fail on either the preparation or actual commit,
-     * it terminates and calls {@link #abort()} on both of them.
-     * <p>
-     * <b>NOTE:</b> it may happen that the TextIndex fails to commit, after the
-     * Transactional has already successfully committed.  A rollback instruction will
-     * still be issued, but depending on the implementation, it may not have any effect.
+     * Rollback all changes, discarding any exceptions that occur.
      */
     @Override
-    public void commit() {
-        if (readWriteMode.get() == ReadWrite.WRITE)
-            commit_W();
-        else
-            commit_R();
+    public void abort() {
+        super.getMonitor().finish() ;
+        abortAction.run();
+        readWriteMode.set(null);
     }
-    
-    
+
     private void commit_R() {
         // No index action needed.
-        super.getMonitor().finish() ;
         super.commit();
-        readWriteMode.set(null);
     }
 
     private void commit_W() {
@@ -185,6 +211,38 @@ public class DatasetGraphText extends DatasetGraphMonitor implements Transaction
             
             // Phase 2
             try {
+                // JENA-1302: This needs the exclusive lock for flushing the queue.
+                // TDB1
+                // Thread 1(W) is running, holds the exclusivitylock=R
+                
+                // Thread 2(W) starts, tries to commit
+                //   Takes txnExitLock
+                //   Calls super.commit
+                //     Find an excessive flush queue.
+                //     It tries to TransactionManger.exclusiveFlushQueue
+                //       This needs exclusivitylock=W
+                //       So Thread 2 blocks, waiting for thread 1
+                //       but still holds txnExitLock
+                //
+                // Thread 1 tries to commit. 
+                //   Can't take the txnExitLock because of thread 2.
+                //
+                // ==> Deadlock.
+                // Fix:
+                //   Put index commit into TDB TransactionLifecycle.
+                //   No txnExitLock.
+                
+                // Doing a non-blocking exclusive attempt in TransactionManger.exclusiveFlushQueue
+                // does not help - we are in a situation where the queue is growing and unflushable
+                // which is why we entered emergency measures. Eventually, RAM will run out as well as
+                // the system becoming slow due to Journal layers. 
+                
+                // TDB2
+                //   All work takes place on the W commiting thread.
+                //   There is no pause point so this can't happen.
+                //     txnExitLock isn't needed, the overall TDB2 transaction 
+                //     means W is unique and all work happens without any potential blocking.
+                
                 super.commit();
                 textIndex.commit();
             }
@@ -193,35 +251,19 @@ public class DatasetGraphText extends DatasetGraphMonitor implements Transaction
                 abort();
                 throw new TextIndexException(t);
             }
-            readWriteMode.set(null);
         }
     }
 
-    /**
-     * Rollback all changes, discarding any exceptions that occur.
-     */
-    @Override
-    public void abort() {
-        if (readWriteMode.get() == ReadWrite.WRITE)
-            abort_W();
-        else
-            abort_R();
-    }
-    
     private void abort_R() {
-        super.getMonitor().finish() ;
         try { super.abort() ; }
         catch (Throwable t) { log.warn("Exception in abort: " + t.getMessage(), t); }
-        readWriteMode.set(null) ;
     }
     
     private void abort_W() {
         synchronized(txnExitLock) {
-            super.getMonitor().finish() ;
             // Roll back on both objects, discarding any exceptions that occur
             try { super.abort(); } catch (Throwable t) { log.warn("Exception in abort: " + t.getMessage(), t); }
             try { textIndex.rollback(); } catch (Throwable t) { log.warn("Exception in abort: " + t.getMessage(), t); }
-            readWriteMode.set(null) ;
         }
     }
 
@@ -239,7 +281,7 @@ public class DatasetGraphText extends DatasetGraphMonitor implements Transaction
         if (readWriteMode.get() == ReadWrite.WRITE) {
             // If we are still in a write transaction at this point, then commit
             // was never called, so rollback the TextIndex and the dataset.
-            abort();
+            abortAction.run();
         }
         super.end() ;
         super.getMonitor().finish() ;

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-text/src/main/java/org/apache/jena/query/text/TextIndexDB.java
----------------------------------------------------------------------
diff --git a/jena-text/src/main/java/org/apache/jena/query/text/TextIndexDB.java b/jena-text/src/main/java/org/apache/jena/query/text/TextIndexDB.java
new file mode 100644
index 0000000..df42fda
--- /dev/null
+++ b/jena-text/src/main/java/org/apache/jena/query/text/TextIndexDB.java
@@ -0,0 +1,80 @@
+/*
+ * 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.nio.ByteBuffer;
+
+import org.apache.jena.dboe.transaction.txn.ComponentId;
+import org.apache.jena.dboe.transaction.txn.TransactionalComponentBase;
+import org.apache.jena.dboe.transaction.txn.TxnId;
+import org.apache.jena.query.ReadWrite;
+import org.apache.jena.query.text.TextIndex;
+
+/** 
+ * Adapter to put Lucene into DBOE transactions.
+ */
+public class TextIndexDB extends TransactionalComponentBase<TextIndexDB.TextState> {
+
+    private final TextIndex textIndex;
+
+    public TextIndexDB(ComponentId id, TextIndex textIndex) {
+        super(id);
+        this.textIndex = textIndex;
+    }
+
+    static class TextState {}
+
+    @Override
+    protected TextState _begin(ReadWrite readWrite, TxnId txnId) {
+        // Need to MRSW?
+        return new TextState();
+    }
+    
+//    @Override
+//    protected TextState _promote(TxnId txnId, TextState oldState) {
+//        return null;
+//    }
+//
+    @Override
+    protected ByteBuffer _commitPrepare(TxnId txnId, TextState state) {
+        textIndex.prepareCommit();
+        return null;
+    }
+
+    // Check.
+    @Override
+    protected void _commit(TxnId txnId, TextState state) {
+        textIndex.commit();
+    }
+
+    @Override
+    protected void _commitEnd(TxnId txnId, TextState state) {}
+
+    @Override
+    protected void _abort(TxnId txnId, TextState state) {
+        textIndex.rollback();
+    }
+//
+//    @Override
+//    protected void _complete(TxnId txnId, TextState state) {}
+//
+//    @Override
+//    protected void _shutdown() {}
+    
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-text/src/main/java/org/apache/jena/query/text/TextIndexTDB1.java
----------------------------------------------------------------------
diff --git a/jena-text/src/main/java/org/apache/jena/query/text/TextIndexTDB1.java b/jena-text/src/main/java/org/apache/jena/query/text/TextIndexTDB1.java
new file mode 100644
index 0000000..740f261
--- /dev/null
+++ b/jena-text/src/main/java/org/apache/jena/query/text/TextIndexTDB1.java
@@ -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.query.text.TextIndex;
+import org.apache.jena.tdb.transaction.Transaction;
+import org.apache.jena.tdb.transaction.TransactionLifecycle;
+
+/*&
+ * Adapter to put Lucene text indexes into the TDB1 trsnaction system.
+ */
+public class TextIndexTDB1 implements TransactionLifecycle {
+
+    private final TextIndex textIndex;
+
+    public TextIndexTDB1(TextIndex textIndex) {
+        this.textIndex = textIndex;
+    }
+    
+    @Override
+    public void begin(Transaction txn) {
+    }
+
+    @Override
+    public void abort(Transaction txn) {
+        textIndex.rollback();
+    }
+
+    @Override
+    public void commitPrepare(Transaction txn) {
+        textIndex.prepareCommit();
+        textIndex.commit();
+    }
+
+    @Override
+    public void committed(Transaction txn) { }
+
+    @Override
+    public void enactCommitted(Transaction txn) {}
+
+    @Override
+    public void clearupCommitted(Transaction txn) {}
+    
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java
----------------------------------------------------------------------
diff --git a/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java b/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java
index 0034632..362a578 100644
--- a/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java
+++ b/jena-text/src/test/java/org/apache/jena/query/text/TS_Text.java
@@ -28,6 +28,7 @@ import org.junit.runners.Suite.SuiteClasses;
 
 @RunWith(Suite.class)
 @SuiteClasses({
+
     TestBuildTextDataset.class
     , TestDatasetWithLuceneTextIndex.class
     , TestDatasetWithLuceneMultilingualTextIndex.class
@@ -35,10 +36,15 @@ import org.junit.runners.Suite.SuiteClasses;
     , TestDatasetWithLuceneGraphTextIndex.class
     , TestDatasetWithLuceneTextIndexDeletionSupport.class
     , TestDatasetWithLuceneStoredLiterals.class
+    
+    , TestTextNonTxn.class
+    , TestTextTxn.class
+    , TestTextNonTxnTDB1.class
+    , TestTextTxnTDB.class
+
     , TestEntityMapAssembler.class
     , TestTextDatasetAssembler.class
     , TestTextIndexLuceneAssembler.class
-    , TestTextTDB.class
     , TestDatasetWithSimpleAnalyzer.class
     , TestDatasetWithStandardAnalyzer.class
     , TestDatasetWithKeywordAnalyzer.class

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-text/src/test/java/org/apache/jena/query/text/TestTextNonTxn.java
----------------------------------------------------------------------
diff --git a/jena-text/src/test/java/org/apache/jena/query/text/TestTextNonTxn.java b/jena-text/src/test/java/org/apache/jena/query/text/TestTextNonTxn.java
new file mode 100644
index 0000000..8c1df1c
--- /dev/null
+++ b/jena-text/src/test/java/org/apache/jena/query/text/TestTextNonTxn.java
@@ -0,0 +1,224 @@
+/**
+ * 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.*;
+import static org.junit.Assume.assumeTrue;
+
+import java.io.Reader;
+import java.io.StringReader;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.lib.Creator;
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.query.* ;
+import org.apache.jena.rdf.model.Model ;
+import org.apache.jena.sparql.core.Quad ;
+import org.apache.jena.sparql.sse.SSE ;
+import org.apache.jena.tdb.TDBFactory ;
+import org.apache.jena.vocabulary.RDFS ;
+import org.apache.lucene.store.Directory ;
+import org.apache.lucene.store.RAMDirectory ;
+import org.junit.Test ;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/** Test using various dataset implmentations without transactions
+ *  No context-set union graph usage either.
+ */  
+@RunWith(Parameterized.class)
+public class TestTextNonTxn
+{
+    @Parameters(name = "{index}: {0}")
+    public static Collection<Object[]>  data() {
+        Creator<Dataset> plainFactory = ()->DatasetFactory.create();
+        Creator<Dataset> timFactory = ()->DatasetFactory.createTxnMem();
+        Creator<Dataset> tdb1Factory = ()->TDBFactory.createDataset();
+        // TDB2 does not work with these, non transactional, tests.
+        return Arrays.asList( new Object[][]{
+            { "Plain", plainFactory , false } ,
+            { "TIM",  timFactory , false } ,
+            { "TDB1", tdb1Factory , true }
+            // TDB2 requires transactions.
+        });
+    }
+    private Dataset create() {
+        Dataset ds1 = factory.create();
+        Directory dir = new RAMDirectory() ;
+        EntityDefinition eDef = new EntityDefinition("iri", "text");
+        eDef.setPrimaryPredicate(RDFS.label);
+        TextIndex tidx = new TextIndexLucene(dir, new TextIndexConfig(eDef)) ;
+        Dataset ds = TextDatasetFactory.create(ds1, tidx) ;
+        return ds ;
+    }
+    
+    private final Creator<Dataset> factory;
+    private final boolean dsFrom;
+    
+    public TestTextNonTxn(String name, Creator<Dataset> factory, Boolean dsFrom) {
+        this.factory = factory;
+        // Does FROM work by pulling graphs from the dataset?
+        this.dsFrom = dsFrom;
+    }
+
+    @Test public void textNonTxn_from_named_graph_1() {
+        assumeTrue(dsFrom);
+        Dataset ds = create() ;
+        data(ds, 
+             "(<ex:g1> <s1> rdfs:label 'foo')",
+             "(<ex:g1> <s2> rdfs:label 'apple')",
+             "(<ex:g2> <s3> rdfs:label 'bar')") ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "FROM <ex:g1>",
+            "{  ?s text:query 'foo*' . ?s rdfs:label ?o }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        assertEquals(1,x.size());
+    }
+
+    @Test public void textNonTxn_from_union_graph() {
+        assumeTrue(dsFrom);
+        Dataset ds = create() ;
+        data(ds, 
+             "(<ex:g1> <s1> rdfs:label 'foo')",
+             "(<ex:g1> <s2> rdfs:label 'apple')",
+             "(<ex:g2> <s3> rdfs:label 'food')") ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "FROM <"+Quad.unionGraph+">",
+            "{ ?s text:query 'foo*' . ?s rdfs:label ?o }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        assertEquals(2,x.size());
+    }
+
+    @Test public void textNonTxn_graph_union_graph() {
+        Dataset ds = create() ;
+        data(ds, 
+             "(<ex:g1> <s1> rdfs:label 'foo')",
+             "(<ex:g1> <s2> rdfs:label 'apple')",
+             "(<ex:g2> <s3> rdfs:label 'food')") ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "{ GRAPH <"+Quad.unionGraph+">",
+            "    { ?s text:query 'foo*' . ?s rdfs:label ?o }",
+            "}"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        assertEquals(2,x.size());
+    }
+    
+    @Test public void textDB_7_subject_bound_first() {
+        Dataset ds = create() ;
+        data(ds, 
+            "(<ex:g1> <s1> rdfs:label 'foo')",
+            "(<ex:g1> <s1> rdf:type <http://example.org/Entity>)",
+            "(<ex:g1> <s2> rdfs:label 'apple')",
+            "(<ex:g1> <s2> rdf:type <http://example.org/Entity>)",
+            "(<ex:g2> <s3> rdfs:label 'food')",
+            "(<ex:g2> <s3> rdf:type <http://example.org/Entity>)");
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "{ GRAPH ?g { ?s a <http://example.org/Entity> . ?s text:query 'foo' } }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        assertEquals(1,x.size());
+    }
+
+    @Test public void textDB_8_bnode_subject() {
+        Dataset ds = create() ;
+        dataTurtle(ds,
+            StrUtils.strjoinNL(
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
+            )
+        );
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "{ ?s text:query 'foo' . ?s a <http://example.org/Entity> }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        assertEquals(1,x.size());
+    }
+
+    @Test public void textDB_9_bnode_subject_bound_first() {
+        Dataset ds = create() ;
+        dataTurtle(ds,
+            StrUtils.strjoinNL(
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
+            )
+        );
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "{ ?s a <http://example.org/Entity> . ?s text:query 'foo' }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        assertEquals(1,x.size());
+    }
+
+    private static void data(Dataset ds, String... quadStrs) {
+        for ( String qs : quadStrs ) {
+            Quad quad = SSE.parseQuad(qs) ;
+            ds.asDatasetGraph().add(quad) ;
+        }
+    }
+
+    private static void dataTurtle(Dataset ds, String turtle) {
+        Model model = ds.getDefaultModel();
+        Reader reader = new StringReader(turtle);
+        model.read(reader, "", "TURTLE");
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-text/src/test/java/org/apache/jena/query/text/TestTextNonTxnTDB1.java
----------------------------------------------------------------------
diff --git a/jena-text/src/test/java/org/apache/jena/query/text/TestTextNonTxnTDB1.java b/jena-text/src/test/java/org/apache/jena/query/text/TestTextNonTxnTDB1.java
new file mode 100644
index 0000000..c8a9985
--- /dev/null
+++ b/jena-text/src/test/java/org/apache/jena/query/text/TestTextNonTxnTDB1.java
@@ -0,0 +1,273 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.query.text;
+
+import java.io.Reader;
+import java.io.StringReader;
+import java.util.List ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.junit.BaseTest ;
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.query.* ;
+import org.apache.jena.rdf.model.Model ;
+import org.apache.jena.sparql.core.Quad ;
+import org.apache.jena.sparql.sse.SSE ;
+import org.apache.jena.tdb.TDB ;
+import org.apache.jena.tdb.TDBFactory ;
+import org.apache.jena.vocabulary.RDFS ;
+import org.apache.lucene.store.Directory ;
+import org.apache.lucene.store.RAMDirectory ;
+import org.junit.Test ;
+
+/** Text dataset tests using TDB1 non-transactionally, including context unionDefaultGraph */
+public class TestTextNonTxnTDB1 extends BaseTest
+{
+    private static Dataset create() {
+        Dataset ds1 = TDBFactory.createDataset() ;
+        Directory dir = new RAMDirectory() ;
+        EntityDefinition eDef = new EntityDefinition("iri", "text");
+        eDef.setPrimaryPredicate(RDFS.label);
+        TextIndex tidx = new TextIndexLucene(dir, new TextIndexConfig(eDef)) ;
+        Dataset ds = TextDatasetFactory.create(ds1, tidx) ;
+        return ds ;
+    }
+
+    @Test public void textTDB1_1() {
+        // Check the union graph stil works  
+        Dataset ds = create() ;
+        ds.getContext().set(TDB.symUnionDefaultGraph, true) ;
+        Quad quad = SSE.parseQuad("(<g> <p> rdfs:label 'foo')") ;
+        ds.asDatasetGraph().add(quad) ;
+        Query q = QueryFactory.create("SELECT * { ?s ?p ?o }") ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        assertEquals(1,x.size());
+    }
+    
+    @Test public void textTDB1_2() {
+        // Check text query and union graph
+        Dataset ds = create() ;
+        ds.getContext().set(TDB.symUnionDefaultGraph, true) ;
+        Quad quad = SSE.parseQuad("(<g> <s> rdfs:label 'foo')") ;
+        ds.asDatasetGraph().add(quad) ;
+        
+        String qs = StrUtils.strjoinNL("PREFIX text: <http://jena.apache.org/text#>",
+                                       "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
+                                       "SELECT *",
+                                       "{ ?s text:query 'foo' ;",
+                                       "     rdfs:label 'foo'",
+                                       "}"
+                                       ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        assertEquals(1,x.size());
+    }
+    
+    @Test public void textTDB1_3() {
+        Dataset ds = create() ;
+        ds.getContext().set(TDB.symUnionDefaultGraph, true) ;
+        data(ds, 
+             "(<ex:g1> <s1> rdfs:label 'foo')",
+             "(<ex:g2> <s2> rdfs:label 'bar')") ;
+
+        ds.begin(ReadWrite.READ) ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text: <http://jena.apache.org/text#>",
+            "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "{ ?s text:query 'foo' ;",
+            "     rdfs:label 'foo'",
+            "}"
+            ) ;
+        
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        ds.end() ;
+        assertEquals(1,x.size());
+    }
+    
+    @Test public void textTDB1_4() {
+        Dataset ds = create() ;
+        data(ds, 
+             "(<ex:g1> <s1> rdfs:label 'foo')",
+             "(<ex:g1> <s2> rdfs:label 'apple')",
+             "(<ex:g2> <s3> rdfs:label 'bar')") ;
+        
+        ds.begin(ReadWrite.READ) ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "FROM <ex:g1>",
+            "{ ?s text:query 'foo' . ?s rdfs:label ?o }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        ds.end() ;
+        assertEquals(1,x.size());
+    }
+
+    @Test public void textTDB1_5() {
+        Dataset ds = create() ;
+        data(ds, 
+             "(<ex:g1> <s1> rdfs:label 'foo')",
+             "(<ex:g1> <s2> rdfs:label 'apple')",
+             "(<ex:g2> <s3> rdfs:label 'food')") ;
+        
+        ds.begin(ReadWrite.READ) ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "FROM <"+Quad.unionGraph+">",
+            "{ ?s text:query 'foo*' . ?s rdfs:label ?o }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        ds.end() ;
+        assertEquals(2,x.size());
+    }
+
+    @Test public void textTDB1_6() {
+        Dataset ds = create() ;
+        data(ds, 
+             "(<ex:g1> <s1> rdfs:label 'foo')",
+             "(<ex:g1> <s2> rdfs:label 'apple')",
+             "(<ex:g2> <s3> rdfs:label 'food')") ;
+        
+        ds.begin(ReadWrite.READ) ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "{ GRAPH <"+Quad.unionGraph+">",
+            "    { ?s text:query 'foo*' . ?s rdfs:label ?o }",
+            "}"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        ds.end() ;
+        assertEquals(2,x.size());
+    }
+    
+    @Test public void textTDB1_7_subject_bound_first() {
+        Dataset ds = create() ;
+        data(ds, 
+            "(<ex:g1> <s1> rdfs:label 'foo')",
+            "(<ex:g1> <s1> rdf:type <http://example.org/Entity>)",
+            "(<ex:g1> <s2> rdfs:label 'apple')",
+            "(<ex:g1> <s2> rdf:type <http://example.org/Entity>)",
+            "(<ex:g2> <s3> rdfs:label 'food')",
+            "(<ex:g2> <s3> rdf:type <http://example.org/Entity>)");
+        
+        ds.begin(ReadWrite.READ) ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "FROM <ex:g1>",
+            "{ ?s a <http://example.org/Entity> . ?s text:query 'foo' }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        ds.end() ;
+        assertEquals(1,x.size());
+    }
+
+    @Test public void textTDB1_8_bnode_subject() {
+        Dataset ds = create() ;
+        dataTurtle(ds,
+            StrUtils.strjoinNL(
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
+            )
+        );
+        
+        ds.begin(ReadWrite.READ) ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "{ ?s text:query 'foo' . ?s a <http://example.org/Entity> }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        ds.end() ;
+        assertEquals(1,x.size());
+    }
+
+    @Test public void textTDB1_9_bnode_subject_bound_first() {
+        Dataset ds = create() ;
+        dataTurtle(ds,
+            StrUtils.strjoinNL(
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
+            )
+        );
+        
+        ds.begin(ReadWrite.READ) ;
+        String qs = StrUtils.strjoinNL(
+            "PREFIX text:   <http://jena.apache.org/text#>",
+            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+            "SELECT *",
+            "{ ?s a <http://example.org/Entity> . ?s text:query 'foo' }"
+            ) ;
+        Query q = QueryFactory.create(qs) ;
+        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+        ResultSet rs = qexec.execSelect() ;
+        List<QuerySolution> x = Iter.toList(rs) ;
+        ds.end() ;
+        assertEquals(1,x.size());
+    }
+
+    private static void data(Dataset ds, String... quadStrs) {
+        for ( String qs : quadStrs ) {
+            Quad quad = SSE.parseQuad(qs) ;
+            ds.asDatasetGraph().add(quad) ;
+        }
+    }
+
+    private static void dataTurtle(Dataset ds, String turtle) {
+        Model model = ds.getDefaultModel();
+        Reader reader = new StringReader(turtle);
+        ds.begin(ReadWrite.WRITE);
+        model.read(reader, "", "TURTLE");
+        ds.commit();
+    }
+    
+    // With transactions
+    // With FROM and FROM NAMED + TDB
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-text/src/test/java/org/apache/jena/query/text/TestTextTDB.java
----------------------------------------------------------------------
diff --git a/jena-text/src/test/java/org/apache/jena/query/text/TestTextTDB.java b/jena-text/src/test/java/org/apache/jena/query/text/TestTextTDB.java
deleted file mode 100644
index 502ec8d..0000000
--- a/jena-text/src/test/java/org/apache/jena/query/text/TestTextTDB.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.query.text;
-
-import java.io.Reader;
-import java.io.StringReader;
-import java.util.List ;
-
-import org.apache.jena.atlas.iterator.Iter ;
-import org.apache.jena.atlas.junit.BaseTest ;
-import org.apache.jena.atlas.lib.StrUtils ;
-import org.apache.jena.query.* ;
-import org.apache.jena.rdf.model.Model ;
-import org.apache.jena.sparql.core.Quad ;
-import org.apache.jena.sparql.sse.SSE ;
-import org.apache.jena.tdb.TDB ;
-import org.apache.jena.tdb.TDBFactory ;
-import org.apache.jena.vocabulary.RDFS ;
-import org.apache.lucene.store.Directory ;
-import org.apache.lucene.store.RAMDirectory ;
-import org.junit.Test ;
-
-public class TestTextTDB extends BaseTest
-{
-    private static Dataset create() {
-        Dataset ds1 = TDBFactory.createDataset() ;
-        Directory dir = new RAMDirectory() ;
-        EntityDefinition eDef = new EntityDefinition("iri", "text");
-        eDef.setPrimaryPredicate(RDFS.label);
-        TextIndex tidx = new TextIndexLucene(dir, new TextIndexConfig(eDef)) ;
-        Dataset ds = TextDatasetFactory.create(ds1, tidx) ;
-        return ds ;
-    }
-
-    @Test public void textDB_1() {
-        // Check the union graph stil works  
-        Dataset ds = create() ;
-        ds.getContext().set(TDB.symUnionDefaultGraph, true) ;
-        Quad quad = SSE.parseQuad("(<g> <p> rdfs:label 'foo')") ;
-        ds.asDatasetGraph().add(quad) ;
-        Query q = QueryFactory.create("SELECT * { ?s ?p ?o }") ;
-        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
-        ResultSet rs = qexec.execSelect() ;
-        List<QuerySolution> x = Iter.toList(rs) ;
-        assertEquals(1,x.size());
-    }
-    
-    @Test public void textDB_2() {
-        // Check text query and union graph
-        Dataset ds = create() ;
-        ds.getContext().set(TDB.symUnionDefaultGraph, true) ;
-        Quad quad = SSE.parseQuad("(<g> <s> rdfs:label 'foo')") ;
-        ds.asDatasetGraph().add(quad) ;
-        
-        String qs = StrUtils.strjoinNL("PREFIX text: <http://jena.apache.org/text#>",
-                                       "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
-                                       "SELECT *",
-                                       "{ ?s text:query 'foo' ;",
-                                       "     rdfs:label 'foo'",
-                                       "}"
-                                       ) ;
-        Query q = QueryFactory.create(qs) ;
-        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
-        ResultSet rs = qexec.execSelect() ;
-        List<QuerySolution> x = Iter.toList(rs) ;
-        assertEquals(1,x.size());
-    }
-    
-    @Test public void textDB_3() {
-        Dataset ds = create() ;
-        ds.getContext().set(TDB.symUnionDefaultGraph, true) ;
-        data(ds, 
-             "(<ex:g1> <s1> rdfs:label 'foo')",
-             "(<ex:g2> <s2> rdfs:label 'bar')") ;
-
-        ds.begin(ReadWrite.READ) ;
-        String qs = StrUtils.strjoinNL(
-            "PREFIX text: <http://jena.apache.org/text#>",
-            "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
-            "SELECT *",
-            "{ ?s text:query 'foo' ;",
-            "     rdfs:label 'foo'",
-            "}"
-            ) ;
-        
-        Query q = QueryFactory.create(qs) ;
-        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
-        ResultSet rs = qexec.execSelect() ;
-        List<QuerySolution> x = Iter.toList(rs) ;
-        ds.end() ;
-        assertEquals(1,x.size());
-    }
-    
-    @Test public void textDB_4() {
-        Dataset ds = create() ;
-        data(ds, 
-             "(<ex:g1> <s1> rdfs:label 'foo')",
-             "(<ex:g1> <s2> rdfs:label 'apple')",
-             "(<ex:g2> <s3> rdfs:label 'bar')") ;
-        
-        ds.begin(ReadWrite.READ) ;
-        String qs = StrUtils.strjoinNL(
-            "PREFIX text:   <http://jena.apache.org/text#>",
-            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
-            "SELECT *",
-            "FROM <ex:g1>",
-            "{ ?s text:query 'foo' . ?s rdfs:label ?o }"
-            ) ;
-        Query q = QueryFactory.create(qs) ;
-        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
-        ResultSet rs = qexec.execSelect() ;
-        List<QuerySolution> x = Iter.toList(rs) ;
-        ds.end() ;
-        assertEquals(1,x.size());
-    }
-
-    @Test public void textDB_5() {
-        Dataset ds = create() ;
-        data(ds, 
-             "(<ex:g1> <s1> rdfs:label 'foo')",
-             "(<ex:g1> <s2> rdfs:label 'apple')",
-             "(<ex:g2> <s3> rdfs:label 'food')") ;
-        
-        ds.begin(ReadWrite.READ) ;
-        String qs = StrUtils.strjoinNL(
-            "PREFIX text:   <http://jena.apache.org/text#>",
-            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
-            "SELECT *",
-            "FROM <"+Quad.unionGraph+">",
-            "{ ?s text:query 'foo*' . ?s rdfs:label ?o }"
-            ) ;
-        Query q = QueryFactory.create(qs) ;
-        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
-        ResultSet rs = qexec.execSelect() ;
-        List<QuerySolution> x = Iter.toList(rs) ;
-        ds.end() ;
-        assertEquals(2,x.size());
-    }
-
-    @Test public void textDB_6() {
-        Dataset ds = create() ;
-        data(ds, 
-             "(<ex:g1> <s1> rdfs:label 'foo')",
-             "(<ex:g1> <s2> rdfs:label 'apple')",
-             "(<ex:g2> <s3> rdfs:label 'food')") ;
-        
-        ds.begin(ReadWrite.READ) ;
-        String qs = StrUtils.strjoinNL(
-            "PREFIX text:   <http://jena.apache.org/text#>",
-            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
-            "SELECT *",
-            "{ GRAPH <"+Quad.unionGraph+">",
-            "    { ?s text:query 'foo*' . ?s rdfs:label ?o }",
-            "}"
-            ) ;
-        Query q = QueryFactory.create(qs) ;
-        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
-        ResultSet rs = qexec.execSelect() ;
-        List<QuerySolution> x = Iter.toList(rs) ;
-        ds.end() ;
-        assertEquals(2,x.size());
-    }
-    
-    @Test public void textDB_7_subject_bound_first() {
-        Dataset ds = create() ;
-        data(ds, 
-            "(<ex:g1> <s1> rdfs:label 'foo')",
-            "(<ex:g1> <s1> rdf:type <http://example.org/Entity>)",
-            "(<ex:g1> <s2> rdfs:label 'apple')",
-            "(<ex:g1> <s2> rdf:type <http://example.org/Entity>)",
-            "(<ex:g2> <s3> rdfs:label 'food')",
-            "(<ex:g2> <s3> rdf:type <http://example.org/Entity>)");
-        
-        ds.begin(ReadWrite.READ) ;
-        String qs = StrUtils.strjoinNL(
-            "PREFIX text:   <http://jena.apache.org/text#>",
-            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
-            "SELECT *",
-            "FROM <ex:g1>",
-            "{ ?s a <http://example.org/Entity> . ?s text:query 'foo' }"
-            ) ;
-        Query q = QueryFactory.create(qs) ;
-        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
-        ResultSet rs = qexec.execSelect() ;
-        List<QuerySolution> x = Iter.toList(rs) ;
-        ds.end() ;
-        assertEquals(1,x.size());
-    }
-
-    @Test public void textDB_8_bnode_subject() {
-        Dataset ds = create() ;
-        dataTurtle(ds,
-            StrUtils.strjoinNL(
-                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
-                "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
-            )
-        );
-        
-        ds.begin(ReadWrite.READ) ;
-        String qs = StrUtils.strjoinNL(
-            "PREFIX text:   <http://jena.apache.org/text#>",
-            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
-            "SELECT *",
-            "{ ?s text:query 'foo' . ?s a <http://example.org/Entity> }"
-            ) ;
-        Query q = QueryFactory.create(qs) ;
-        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
-        ResultSet rs = qexec.execSelect() ;
-        List<QuerySolution> x = Iter.toList(rs) ;
-        ds.end() ;
-        assertEquals(1,x.size());
-    }
-
-    @Test public void textDB_9_bnode_subject_bound_first() {
-        Dataset ds = create() ;
-        dataTurtle(ds,
-            StrUtils.strjoinNL(
-                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
-                "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
-            )
-        );
-        
-        ds.begin(ReadWrite.READ) ;
-        String qs = StrUtils.strjoinNL(
-            "PREFIX text:   <http://jena.apache.org/text#>",
-            "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
-            "SELECT *",
-            "{ ?s a <http://example.org/Entity> . ?s text:query 'foo' }"
-            ) ;
-        Query q = QueryFactory.create(qs) ;
-        QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
-        ResultSet rs = qexec.execSelect() ;
-        List<QuerySolution> x = Iter.toList(rs) ;
-        ds.end() ;
-        assertEquals(1,x.size());
-    }
-
-    private static void data(Dataset ds, String... quadStrs) {
-        for ( String qs : quadStrs ) {
-            Quad quad = SSE.parseQuad(qs) ;
-            ds.asDatasetGraph().add(quad) ;
-        }
-    }
-
-    private static void dataTurtle(Dataset ds, String turtle) {
-        Model model = ds.getDefaultModel();
-        Reader reader = new StringReader(turtle);
-        ds.begin(ReadWrite.WRITE);
-        model.read(reader, "", "TURTLE");
-        ds.commit();
-    }
-    
-    // With transactions
-    // With FROM and FROM NAMED + TDB
-}
-

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-text/src/test/java/org/apache/jena/query/text/TestTextTxn.java
----------------------------------------------------------------------
diff --git a/jena-text/src/test/java/org/apache/jena/query/text/TestTextTxn.java b/jena-text/src/test/java/org/apache/jena/query/text/TestTextTxn.java
new file mode 100644
index 0000000..5569e15
--- /dev/null
+++ b/jena-text/src/test/java/org/apache/jena/query/text/TestTextTxn.java
@@ -0,0 +1,341 @@
+/**
+ * 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.Assume.assumeTrue;
+
+import java.io.Reader;
+import java.io.StringReader;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.lib.Creator;
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.query.* ;
+import org.apache.jena.rdf.model.Model ;
+import org.apache.jena.sparql.core.Quad ;
+import org.apache.jena.sparql.sse.SSE ;
+import org.apache.jena.system.Txn;
+import org.apache.jena.tdb.TDBFactory ;
+import org.apache.jena.tdb2.TDB2Factory;
+import org.apache.jena.vocabulary.RDFS ;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.store.Directory ;
+import org.apache.lucene.store.RAMDirectory ;
+import org.junit.Test ;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/** Text dataset tests using datasets transactionally, named unionDefaultGraph.
+ * <p>
+ * Note that in Lucene, writes are not visible to a reader of the {@code Directory}
+ * until committed. A special {@link IndexReader} is needed. See
+ * {@link DirectoryReader#open(IndexWriter)}. jena-text does not currentyl do this.
+ * <p>
+ * When used outside a transaction, writes are "autocommit" (see
+ * {@link TextDocProducerTriples}) and so are visible immediately.
+ * <p>
+ * TDB2 is transactional only. 
+ * <p>Union graph support by context is required for these tests.
+ */
+
+@RunWith(Parameterized.class)
+public class TestTextTxn
+{
+    @Parameters(name = "{index}: {0}")
+    public static Collection<Object[]>  data() {
+        Creator<Dataset> plainFactory = ()->DatasetFactory.create();
+        Creator<Dataset> timFactory = ()->DatasetFactory.createTxnMem();
+        Creator<Dataset> tdb1Factory = ()->TDBFactory.createDataset();
+        Creator<Dataset> tdb2Factory = ()->TDB2Factory.createDataset();
+        return Arrays.asList( new Object[][]{
+            { "Plain", plainFactory, false } ,
+            { "TIM",   timFactory, false } , 
+            { "TDB1", tdb1Factory, true } ,
+            { "TDB2", tdb2Factory, true }
+        });
+    }
+    
+    private final Creator<Dataset> factory;
+    private final Boolean dsFrom;
+
+    public TestTextTxn(String name, Creator<Dataset> factory, Boolean dsFrom) {
+        this.factory = factory;
+        // Does FROM work by pulling graphs from the dataset?
+        this.dsFrom = dsFrom;
+    }
+    
+    private Dataset create() {
+        Dataset ds1 = factory.create();
+        Directory dir = new RAMDirectory() ;
+        EntityDefinition eDef = new EntityDefinition("iri", "text");
+        eDef.setPrimaryPredicate(RDFS.label);
+        TextIndex tidx = new TextIndexLucene(dir, new TextIndexConfig(eDef)) ;
+        Dataset ds = TextDatasetFactory.create(ds1, tidx) ;
+        return ds ;
+    }
+
+    @Test public void textTxn_dftGraph_1() {
+        Dataset ds = create() ;
+        Quad quad = SSE.parseQuad("(_ <p> rdfs:label 'foo')") ;
+        Txn.executeWrite(ds, ()->ds.asDatasetGraph().add(quad));
+        
+        Txn.executeRead(ds, ()->{
+            Query q = QueryFactory.create("SELECT * { ?s ?p ?o }") ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+    
+    @Test public void textTxn_namedGraphplain_1() {
+        Dataset ds = create() ;
+        Quad quad = SSE.parseQuad("(<g> <p> rdfs:label 'foo')") ;
+        Txn.executeWrite(ds, ()->ds.asDatasetGraph().add(quad));
+        
+        Txn.executeRead(ds, ()->{
+            Query q = QueryFactory.create("SELECT * { GRAPH ?g { ?s ?p ?o } }") ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+    
+    @Test public void textTxn_query_1() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            Quad quad = SSE.parseQuad("(_ <s> rdfs:label 'foo')") ;
+            ds.asDatasetGraph().add(quad) ;
+        });
+
+        Txn.executeRead(ds, ()->{
+            String qs = StrUtils.strjoinNL("PREFIX text: <http://jena.apache.org/text#>",
+                "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ ?s text:query 'foo' ;",
+                "     rdfs:label 'foo'",
+                "}"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+    
+    @Test public void textTxn_query_2() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            data(ds, 
+                "(_ <s1> rdfs:label 'foo')",
+                "(_ <s2> rdfs:label 'bar')") ;
+        });
+        Txn.executeRead(ds, ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text: <http://jena.apache.org/text#>",
+                "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ ?s text:query 'foo' ;",
+                "     rdfs:label 'foo'",
+                "}"
+                ) ;
+
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+    
+    @Test public void textTxn_from_namedGraph_query_1() {
+        assumeTrue(dsFrom);
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            data(ds, 
+                "(<ex:g1> <s1> rdfs:label 'foo')",
+                "(<ex:g1> <s2> rdfs:label 'apple')",
+                "(<ex:g2> <s3> rdfs:label 'bar')") ;
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "FROM <ex:g1>",
+                "{ ?s text:query 'foo' . ?s rdfs:label ?o }"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+
+    @Test public void textTxn_from_unionGraph() {
+        assumeTrue(dsFrom);
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+        data(ds, 
+             "(<ex:g1> <s1> rdfs:label 'foo')",
+             "(<ex:g1> <s2> rdfs:label 'apple')",
+             "(<ex:g2> <s3> rdfs:label 'food')") ;
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "FROM <"+Quad.unionGraph+">",
+                "{ ?s text:query 'foo*' . ?s rdfs:label ?o }"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+
+            assertEquals(2,x.size());
+        });
+    }
+
+    @Test public void textTxn_graphUnionGraph() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            data(ds, 
+                "(<ex:g1> <s1> rdfs:label 'foo')",
+                "(<ex:g1> <s2> rdfs:label 'apple')",
+                "(<ex:g2> <s3> rdfs:label 'food')") ;
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ GRAPH <"+Quad.unionGraph+">",
+                "    { ?s text:query 'foo*' . ?s rdfs:label ?o }",
+                "}"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(2,x.size());
+        });
+    }
+    
+    @Test public void textTxn_subject_bound_first() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            data(ds, 
+                "(<ex:g1> <s1> rdfs:label 'foo')",
+                "(<ex:g1> <s1> rdf:type <http://example.org/Entity>)",
+                "(<ex:g1> <s2> rdfs:label 'apple')",
+                "(<ex:g1> <s2> rdf:type <http://example.org/Entity>)",
+                "(<ex:g2> <s3> rdfs:label 'food')",
+                "(<ex:g2> <s3> rdf:type <http://example.org/Entity>)");
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ GRAPH ?g { ?s a <http://example.org/Entity> . ?s text:query 'foo' } }"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+
+    @Test public void textTxn_bnode_subject() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            dataTurtle(ds,
+                StrUtils.strjoinNL(
+                    "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                    "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
+                    )
+                );
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ ?s text:query 'foo' . ?s a <http://example.org/Entity> }"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+
+    @Test public void textTxn_bnode_subject_bound_first() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            dataTurtle(ds,
+                StrUtils.strjoinNL(
+                    "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                    "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
+                    )
+                );
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ ?s a <http://example.org/Entity> . ?s text:query 'foo' }"
+                );
+            Query q = QueryFactory.create(qs);
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds);
+            ResultSet rs = qexec.execSelect();
+            List<QuerySolution> x = Iter.toList(rs);
+            assertEquals(1,x.size());
+        });
+    }
+
+    private static void data(Dataset ds, String... quadStrs) {
+        for ( String qs : quadStrs ) {
+            Quad quad = SSE.parseQuad(qs) ;
+            ds.asDatasetGraph().add(quad) ;
+        }
+    }
+
+    private static void dataTurtle(Dataset ds, String turtle) {
+        Model model = ds.getDefaultModel();
+        Reader reader = new StringReader(turtle);
+        model.read(reader, "", "TURTLE");
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/2d6330f4/jena-text/src/test/java/org/apache/jena/query/text/TestTextTxnTDB.java
----------------------------------------------------------------------
diff --git a/jena-text/src/test/java/org/apache/jena/query/text/TestTextTxnTDB.java b/jena-text/src/test/java/org/apache/jena/query/text/TestTextTxnTDB.java
new file mode 100644
index 0000000..55f1818
--- /dev/null
+++ b/jena-text/src/test/java/org/apache/jena/query/text/TestTextTxnTDB.java
@@ -0,0 +1,329 @@
+/**
+ * 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 java.io.Reader;
+import java.io.StringReader;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.lib.Creator;
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.query.* ;
+import org.apache.jena.rdf.model.Model ;
+import org.apache.jena.sparql.core.Quad ;
+import org.apache.jena.sparql.sse.SSE ;
+import org.apache.jena.system.Txn;
+import org.apache.jena.tdb.TDB ;
+import org.apache.jena.tdb.TDBFactory ;
+import org.apache.jena.tdb2.TDB2;
+import org.apache.jena.tdb2.TDB2Factory;
+import org.apache.jena.vocabulary.RDFS ;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.store.Directory ;
+import org.apache.lucene.store.RAMDirectory ;
+import org.junit.Test ;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/** Text dataset tests using TDB1 and TDB2 transactionally, including context unionDefaultGraph.
+ * <p>
+ * Note that in Lucene, writes are not visible to a reader of the {@code Directory}
+ * until committed. A special {@link IndexReader} is needed. See
+ * {@link DirectoryReader#open(IndexWriter)}. jena-text does not currentyl do this.
+ * <p>
+ * When used outside a transaction, writes are "autocommit" (see
+ * {@link TextDocProducerTriples}) and so are visible immediately.
+ * <p>
+ * TDB2 is transactional only. 
+ * <p>Union graph support by context is required for these tests.
+ */
+
+@RunWith(Parameterized.class)
+public class TestTextTxnTDB
+{
+    @Parameters(name = "{index}: {0}")
+    public static Collection<Object[]>  data() {
+        Creator<Dataset> tdb1Factory = ()->TDBFactory.createDataset();
+        Creator<Dataset> tdb2Factory = ()->TDB2Factory.createDataset();
+        return Arrays.asList( new Object[][]{
+            { "TDB1", tdb1Factory } ,
+            { "TDB2", tdb2Factory }
+        });
+    }
+    
+    private final Creator<Dataset> factory;
+
+    public TestTextTxnTDB(String name, Creator<Dataset> factory) {
+        this.factory = factory;
+    }
+    
+    private Dataset create() {
+        Dataset ds1 = factory.create();
+        Directory dir = new RAMDirectory() ;
+        EntityDefinition eDef = new EntityDefinition("iri", "text");
+        eDef.setPrimaryPredicate(RDFS.label);
+        TextIndex tidx = new TextIndexLucene(dir, new TextIndexConfig(eDef)) ;
+        Dataset ds = TextDatasetFactory.create(ds1, tidx) ;
+        return ds ;
+    }
+
+    @Test public void textTDB_union_1() {
+        // Check the union graph still works  
+        Dataset ds = create() ;
+        ds.getContext().set(TDB.symUnionDefaultGraph, true) ;
+        ds.getContext().set(TDB2.symUnionDefaultGraph, true) ;
+        Quad quad = SSE.parseQuad("(<g> <p> rdfs:label 'foo')") ;
+        Txn.executeWrite(ds, ()->ds.asDatasetGraph().add(quad));
+        
+        Txn.executeRead(ds, ()->{
+            Query q = QueryFactory.create("SELECT * { ?s ?p ?o }") ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+    
+    @Test public void textTDB_union_2() {
+        // Check text query and union graph
+        Dataset ds = create() ;
+        ds.getContext().set(TDB.symUnionDefaultGraph, true) ;
+        ds.getContext().set(TDB2.symUnionDefaultGraph, true) ;
+        Txn.executeWrite(ds, ()->{
+            Quad quad = SSE.parseQuad("(<g> <s> rdfs:label 'foo')") ;
+            ds.asDatasetGraph().add(quad) ;
+        });
+        Txn.executeRead(ds, ()->{
+            String qs = StrUtils.strjoinNL("PREFIX text: <http://jena.apache.org/text#>",
+                "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ ?s text:query 'foo' ;",
+                "     rdfs:label 'foo'",
+                "}"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+    
+    @Test public void textTDB_3() {
+        Dataset ds = create() ;
+        ds.getContext().set(TDB.symUnionDefaultGraph, true) ;
+        ds.getContext().set(TDB2.symUnionDefaultGraph, true) ;
+        Txn.executeWrite(ds, ()->{
+            data(ds, 
+                "(<ex:g1> <s1> rdfs:label 'foo')",
+                "(<ex:g2> <s2> rdfs:label 'bar')") ;
+        });
+        Txn.executeRead(ds, ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text: <http://jena.apache.org/text#>",
+                "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ ?s text:query 'foo' ;",
+                "     rdfs:label 'foo'",
+                "}"
+                ) ;
+
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+    
+    @Test public void textTDB_4() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            data(ds, 
+                "(<ex:g1> <s1> rdfs:label 'foo')",
+                "(<ex:g1> <s2> rdfs:label 'apple')",
+                "(<ex:g2> <s3> rdfs:label 'bar')") ;
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "FROM <ex:g1>",
+                "{ ?s text:query 'foo' . ?s rdfs:label ?o }"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+
+    @Test public void textTDB_5() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+        data(ds, 
+             "(<ex:g1> <s1> rdfs:label 'foo')",
+             "(<ex:g1> <s2> rdfs:label 'apple')",
+             "(<ex:g2> <s3> rdfs:label 'food')") ;
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "FROM <"+Quad.unionGraph+">",
+                "{ ?s text:query 'foo*' . ?s rdfs:label ?o }"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+
+            assertEquals(2,x.size());
+        });
+    }
+
+    @Test public void textTDB_6() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            data(ds, 
+                "(<ex:g1> <s1> rdfs:label 'foo')",
+                "(<ex:g1> <s2> rdfs:label 'apple')",
+                "(<ex:g2> <s3> rdfs:label 'food')") ;
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ GRAPH <"+Quad.unionGraph+">",
+                "    { ?s text:query 'foo*' . ?s rdfs:label ?o }",
+                "}"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(2,x.size());
+        });
+    }
+    
+    @Test public void textTDB_7_subject_bound_first() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            data(ds, 
+                "(<ex:g1> <s1> rdfs:label 'foo')",
+                "(<ex:g1> <s1> rdf:type <http://example.org/Entity>)",
+                "(<ex:g1> <s2> rdfs:label 'apple')",
+                "(<ex:g1> <s2> rdf:type <http://example.org/Entity>)",
+                "(<ex:g2> <s3> rdfs:label 'food')",
+                "(<ex:g2> <s3> rdf:type <http://example.org/Entity>)");
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "FROM <ex:g1>",
+                "{ ?s a <http://example.org/Entity> . ?s text:query 'foo' }"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+
+    @Test public void textTDB_8_bnode_subject() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            dataTurtle(ds,
+                StrUtils.strjoinNL(
+                    "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                    "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
+                    )
+                );
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ ?s text:query 'foo' . ?s a <http://example.org/Entity> }"
+                ) ;
+            Query q = QueryFactory.create(qs) ;
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds) ;
+            ResultSet rs = qexec.execSelect() ;
+            List<QuerySolution> x = Iter.toList(rs) ;
+            assertEquals(1,x.size());
+        });
+    }
+
+    @Test public void textTDB_9_bnode_subject_bound_first() {
+        Dataset ds = create() ;
+        Txn.executeWrite(ds, ()->{
+            dataTurtle(ds,
+                StrUtils.strjoinNL(
+                    "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                    "[] a <http://example.org/Entity>; rdfs:label 'foo' ."
+                    )
+                );
+        });
+        Txn.executeRead(ds,  ()->{
+            String qs = StrUtils.strjoinNL(
+                "PREFIX text:   <http://jena.apache.org/text#>",
+                "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+                "SELECT *",
+                "{ ?s a <http://example.org/Entity> . ?s text:query 'foo' }"
+                );
+            Query q = QueryFactory.create(qs);
+            QueryExecution qexec = QueryExecutionFactory.create(q, ds);
+            ResultSet rs = qexec.execSelect();
+            List<QuerySolution> x = Iter.toList(rs);
+            assertEquals(1,x.size());
+        });
+    }
+
+    private static void data(Dataset ds, String... quadStrs) {
+        for ( String qs : quadStrs ) {
+            Quad quad = SSE.parseQuad(qs) ;
+            ds.asDatasetGraph().add(quad) ;
+        }
+    }
+
+    private static void dataTurtle(Dataset ds, String turtle) {
+        Model model = ds.getDefaultModel();
+        Reader reader = new StringReader(turtle);
+        model.read(reader, "", "TURTLE");
+    }
+    
+    // With FROM and FROM NAMED + TDB
+}
+