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 2015/09/21 20:30:58 UTC

[04/25] jena git commit: JENA-1029: Subsystem initialization (default, ServiceLoader)

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-csv/src/main/java/org/apache/jena/propertytable/impl/InitJenaCSV.java
----------------------------------------------------------------------
diff --git a/jena-csv/src/main/java/org/apache/jena/propertytable/impl/InitJenaCSV.java b/jena-csv/src/main/java/org/apache/jena/propertytable/impl/InitJenaCSV.java
new file mode 100644
index 0000000..14111f5
--- /dev/null
+++ b/jena-csv/src/main/java/org/apache/jena/propertytable/impl/InitJenaCSV.java
@@ -0,0 +1,34 @@
+/**
+ * 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.propertytable.impl;
+
+import org.apache.jena.propertytable.lang.CSV2RDF ;
+import org.apache.jena.system.JenaSubsystemLifecycle ;
+
+public class InitJenaCSV implements JenaSubsystemLifecycle {
+
+    @Override
+    public void start() {
+        CSV2RDF.init() ;
+    }
+
+    @Override
+    public void stop() {
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-csv/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
----------------------------------------------------------------------
diff --git a/jena-csv/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle b/jena-csv/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
new file mode 100644
index 0000000..0399e1f
--- /dev/null
+++ b/jena-csv/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
@@ -0,0 +1 @@
+org.apache.jena.propertytable.impl.InitJenaCSV

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-fuseki1/src/main/java/org/apache/jena/fuseki/Fuseki.java
----------------------------------------------------------------------
diff --git a/jena-fuseki1/src/main/java/org/apache/jena/fuseki/Fuseki.java b/jena-fuseki1/src/main/java/org/apache/jena/fuseki/Fuseki.java
index 45e6415..b015f83 100644
--- a/jena-fuseki1/src/main/java/org/apache/jena/fuseki/Fuseki.java
+++ b/jena-fuseki1/src/main/java/org/apache/jena/fuseki/Fuseki.java
@@ -20,7 +20,6 @@ package org.apache.jena.fuseki;
 
 import org.apache.jena.fuseki.server.SPARQLServer ;
 import org.apache.jena.query.ARQ ;
-import org.apache.jena.riot.RIOT ;
 import org.apache.jena.riot.system.stream.LocatorFTP ;
 import org.apache.jena.riot.system.stream.LocatorHTTP ;
 import org.apache.jena.riot.system.stream.StreamManager ;
@@ -29,6 +28,7 @@ import org.apache.jena.sparql.lib.Metadata ;
 import org.apache.jena.sparql.mgt.SystemInfo ;
 import org.apache.jena.sparql.util.Context ;
 import org.apache.jena.sparql.util.MappingRegistry ;
+import org.apache.jena.system.JenaSystem ;
 import org.apache.jena.tdb.TDB ;
 import org.apache.jena.tdb.transaction.TransactionManager ;
 import org.slf4j.Logger ;
@@ -137,11 +137,9 @@ public class Fuseki
         if ( initialized )
             return ;
         initialized = true ;
-        ARQ.init() ;
+        JenaSystem.init();
         SystemInfo sysInfo = new SystemInfo(FusekiIRI, PATH, VERSION, BUILD_DATE) ;
         SystemARQ.registerSubSystem(sysInfo) ;
-        RIOT.init() ;
-        TDB.init() ;
         MappingRegistry.addPrefixMapping("fuseki", FusekiSymbolIRI) ;
         
         TDB.setOptimizerWarningFlag(false) ;

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-fuseki1/src/main/java/org/apache/jena/fuseki/migrate/Registry.java
----------------------------------------------------------------------
diff --git a/jena-fuseki1/src/main/java/org/apache/jena/fuseki/migrate/Registry.java b/jena-fuseki1/src/main/java/org/apache/jena/fuseki/migrate/Registry.java
deleted file mode 100644
index ca30be1..0000000
--- a/jena-fuseki1/src/main/java/org/apache/jena/fuseki/migrate/Registry.java
+++ /dev/null
@@ -1,42 +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.fuseki.migrate;
-
-import java.util.Collection ;
-import java.util.HashMap ;
-import java.util.Map ;
-
-public class Registry<T>
-{
-    protected Map<String, T> registry = new HashMap<String, T>() ;
-    
-    public Registry() {}
-    
-    public void put(String key, T value) { registry.put(key, value) ; }
-    
-    public T get(String key) { return registry.get(key) ; }
-    
-    public boolean isRegistered(String key) { return registry.containsKey(key) ; }
-    public void remove(String key) { registry.remove(key) ; } 
-    public Collection<String> keys() { return registry.keySet() ; }
-    //public Iterator<String> keys() { return registry.keySet().iterator() ; }
-    
-    public int size() { return registry.size() ; }
-    public boolean isEmpty() { return registry.isEmpty() ; }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-fuseki1/src/main/java/org/apache/jena/fuseki/server/DatasetRegistry.java
----------------------------------------------------------------------
diff --git a/jena-fuseki1/src/main/java/org/apache/jena/fuseki/server/DatasetRegistry.java b/jena-fuseki1/src/main/java/org/apache/jena/fuseki/server/DatasetRegistry.java
index 152e8cd..60d15a8 100644
--- a/jena-fuseki1/src/main/java/org/apache/jena/fuseki/server/DatasetRegistry.java
+++ b/jena-fuseki1/src/main/java/org/apache/jena/fuseki/server/DatasetRegistry.java
@@ -18,9 +18,9 @@
 
 package org.apache.jena.fuseki.server;
 
-import org.apache.jena.fuseki.migrate.Registry ;
+import org.apache.jena.atlas.lib.Registry ;
 
-public class DatasetRegistry extends Registry<DatasetRef>
+public class DatasetRegistry extends Registry<String, DatasetRef>
 {
     private static DatasetRegistry singleton = new DatasetRegistry() ;
 

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-fuseki1/src/test/java/org/apache/jena/fuseki/ServerTest.java
----------------------------------------------------------------------
diff --git a/jena-fuseki1/src/test/java/org/apache/jena/fuseki/ServerTest.java b/jena-fuseki1/src/test/java/org/apache/jena/fuseki/ServerTest.java
index 9659da0..37b803e 100644
--- a/jena-fuseki1/src/test/java/org/apache/jena/fuseki/ServerTest.java
+++ b/jena-fuseki1/src/test/java/org/apache/jena/fuseki/ServerTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.jena.fuseki;
 
+import org.apache.jena.fuseki.server.DatasetRegistry ;
 import org.apache.jena.graph.Graph ;
 import org.apache.jena.graph.Node ;
 import org.apache.jena.graph.NodeFactory ;
@@ -98,6 +99,7 @@ public class ServerTest
     }
     
     protected static void teardownServer() {
+        DatasetRegistry.get().clear() ;
         if ( server != null )
             server.stop() ;
         server = null ;

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/Fuseki.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/Fuseki.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/Fuseki.java
index a6cfbdc..991c673 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/Fuseki.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/Fuseki.java
@@ -26,7 +26,6 @@ import org.apache.jena.atlas.lib.DateTimeUtils ;
 import org.apache.jena.query.ARQ ;
 import org.apache.jena.query.spatial.SpatialQuery ;
 import org.apache.jena.query.text.TextQuery ;
-import org.apache.jena.riot.RIOT ;
 import org.apache.jena.riot.system.stream.LocatorFTP ;
 import org.apache.jena.riot.system.stream.LocatorHTTP ;
 import org.apache.jena.riot.system.stream.StreamManager ;
@@ -35,6 +34,7 @@ import org.apache.jena.sparql.lib.Metadata ;
 import org.apache.jena.sparql.mgt.SystemInfo ;
 import org.apache.jena.sparql.util.Context ;
 import org.apache.jena.sparql.util.MappingRegistry ;
+import org.apache.jena.system.JenaSystem ;
 import org.apache.jena.tdb.TDB ;
 import org.apache.jena.tdb.transaction.TransactionManager ;
 import org.slf4j.Logger ;
@@ -215,13 +215,11 @@ public class Fuseki {
         if ( initialized )
             return ;
         initialized = true ;
-        // FusekiEnv.setEnvironment() ;
+        JenaSystem.init() ;
+        // Do explicitly so it happens after subsystem initialization. 
         FusekiLogging.setLogging() ;
-        ARQ.init() ;
         SystemInfo sysInfo = new SystemInfo(FusekiIRI, PATH, VERSION, BUILD_DATE) ;
         SystemARQ.registerSubSystem(sysInfo) ;
-        RIOT.init() ;
-        
         TDB.init() ;
         // Initialize anyway (e.g. not to rely on assembler magic).
         try { 

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/cmd/FusekiCmd.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/cmd/FusekiCmd.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/cmd/FusekiCmd.java
index 9970af7..931c6e0 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/cmd/FusekiCmd.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/cmd/FusekiCmd.java
@@ -35,6 +35,7 @@ import org.apache.jena.riot.Lang ;
 import org.apache.jena.riot.RDFDataMgr ;
 import org.apache.jena.riot.RDFLanguages ;
 import org.apache.jena.sparql.core.DatasetGraphFactory ;
+import org.apache.jena.system.JenaSystem ;
 import org.apache.jena.tdb.TDB ;
 import org.apache.jena.tdb.sys.Names ;
 import org.apache.jena.tdb.transaction.TransactionManager ;
@@ -89,9 +90,8 @@ public class FusekiCmd {
         // fuseki [--mem|--desc assembler.ttl] [--port PORT] **** /datasetURI
 
         static public void innerMain(String... argv) {
-            // Just to make sure ...
-            ARQ.init() ;
-            TDB.init() ;
+            JenaSystem.init() ;
+            // Do explicitly so it happens after subsystem initialization. 
             Fuseki.init() ;
             new FusekiCmdInner(argv).mainRun() ;
         }

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiServerEnvironmentInit.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiServerEnvironmentInit.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiServerEnvironmentInit.java
index 356dc82..3142238 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiServerEnvironmentInit.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiServerEnvironmentInit.java
@@ -22,6 +22,7 @@ import javax.servlet.ServletContextEvent ;
 import javax.servlet.ServletContextListener ;
 
 import org.apache.jena.fuseki.FusekiLogging ;
+import org.apache.jena.system.JenaSystem ;
 
 /** Setup the enviroment and logging.
  *  Runs before the ShiroEnvironmentLoader.
@@ -32,8 +33,9 @@ public class FusekiServerEnvironmentInit implements ServletContextListener {
     
     @Override
     public void contextInitialized(ServletContextEvent sce) {
-        FusekiEnv.setEnvironment();
-        FusekiLogging.setLogging();
+        JenaSystem.init() ;
+        FusekiEnv.setEnvironment() ;
+        FusekiLogging.setLogging() ;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-parent/pom.xml
----------------------------------------------------------------------
diff --git a/jena-parent/pom.xml b/jena-parent/pom.xml
index dcbd614..240eee4 100644
--- a/jena-parent/pom.xml
+++ b/jena-parent/pom.xml
@@ -425,7 +425,8 @@
         </executions>
         <configuration>
            <excludeSubProjects>true</excludeSubProjects>
-             <excludes>
+ 	     <excludes>
+	       <exclude>**/META-INF/services/*</exclude>
                <!--
                jena-core exclusions
                -->

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-sdb/src/main/java/org/apache/jena/sdb/SDB.java
----------------------------------------------------------------------
diff --git a/jena-sdb/src/main/java/org/apache/jena/sdb/SDB.java b/jena-sdb/src/main/java/org/apache/jena/sdb/SDB.java
index 0fb5204..544bbe6 100644
--- a/jena-sdb/src/main/java/org/apache/jena/sdb/SDB.java
+++ b/jena-sdb/src/main/java/org/apache/jena/sdb/SDB.java
@@ -18,7 +18,6 @@
 
 package org.apache.jena.sdb;
 
-import org.apache.jena.assembler.assemblers.AssemblerGroup ;
 import org.apache.jena.datatypes.xsd.XSDDatatype ;
 import org.apache.jena.query.ARQ ;
 import org.apache.jena.sdb.assembler.AssemblerVocab ;
@@ -30,17 +29,16 @@ import org.apache.jena.shared.PrefixMapping ;
 import org.apache.jena.shared.impl.PrefixMappingImpl ;
 import org.apache.jena.sparql.ARQInternalErrorException ;
 import org.apache.jena.sparql.SystemARQ ;
-import org.apache.jena.sparql.core.assembler.AssemblerUtils ;
 import org.apache.jena.sparql.lib.Metadata ;
 import org.apache.jena.sparql.mgt.SystemInfo ;
 import org.apache.jena.sparql.util.Context ;
 import org.apache.jena.sparql.util.MappingRegistry ;
 import org.apache.jena.sparql.util.Symbol ;
+import org.apache.jena.system.JenaSystem ;
 import org.apache.jena.vocabulary.OWL ;
 import org.apache.jena.vocabulary.RDF ;
 import org.apache.jena.vocabulary.RDFS ;
 
-
 public class SDB
 {
     /** IRI for SDB */  
@@ -71,49 +69,58 @@ public class SDB
     // Global context is the ARQ context.
     public static Context getContext() { return ARQ.getContext() ; }
     
-    static { initWorker() ; }
-    public static void init() { }
-    
-    /** Used by Jena assemblers for registration */ 
-    public static void whenRequiredByAssembler( AssemblerGroup g )
-    {
-        AssemblerUtils.init() ;         // ARQ 
-        AssemblerVocab.register(g) ;    // SDB
+    static { 
+        JenaSystem.init(); 
     }
     
-    private static boolean initialized = false ;
-    private static synchronized void initWorker()
-    {
-        // Called from 
-        // + StoreFactory
-        // + DatasetStore
-        // Commands call AssemblerVocab.init() ;
-
-        if ( initialized )
+//    /** Used by Jena assemblers for registration */ 
+//    public static void whenRequiredByAssembler( AssemblerGroup g )
+//    {
+//        AssemblerUtils.init() ;         // ARQ 
+//        AssemblerVocab.register(g) ;    // SDB
+//    }
+    
+    private static volatile boolean initialized = false ;
+    private static Object initLock = new Object() ;
+    
+    public static void init() {
+        if ( initialized ) 
             return ;
-        
-        // Set this immediately in case code below causes init() to be called.
-        // (It's better if there are no dependences but ...)
-        initialized = true ;
-        
-        // Better not to break up BGPs too much.
-        ARQ.getContext().set(ARQ.optFilterPlacement, false) ;
-        MappingRegistry.addPrefixMapping(SDB.symbolPrefix, SDB.symbolSpace) ;
+        synchronized(initLock) {
+            if ( initialized ) {
+                if ( JenaSystem.DEBUG_INIT )
+                    System.err.println("SDB.init - skip") ;
+                return ;
+            }
+            initialized = true ;
+            if ( JenaSystem.DEBUG_INIT )
+                System.err.println("SDB.init - start") ;
+            // Set this immediately in case code below causes init() to be called.
+            // (It's better if there are no dependences but ...)
+            initialized = true ;
+
+            // Better not to break up BGPs too much.
+            ARQ.getContext().set(ARQ.optFilterPlacement, false) ;
+            MappingRegistry.addPrefixMapping(SDB.symbolPrefix, SDB.symbolSpace) ;
+
+            // Default is 1000 4Kpages.
+            DerbyUtils.setDerbyPageCacheSize(10000) ;
 
-        // Default is 1000 4Kpages.
-        DerbyUtils.setDerbyPageCacheSize(10000) ;
+            // Wire in the SDB query engine
+            QueryEngineSDB.register() ;
+            // Wire in the SDB update engine
+            UpdateEngineSDB.register() ;
 
-        // Wire in the SDB query engine
-        QueryEngineSDB.register() ;
-        // Wire in the SDB update engine
-        UpdateEngineSDB.register() ;
-        
-        SDB.getContext().setIfUndef(useQuadRewrite,        false) ;
-        SDB.getContext().setIfUndef(streamGraphAPI,        false) ;
-        SDB.getContext().setIfUndef(jdbcStream,            true) ;
-        //SDB.getContext().setIfUndef(jdbcFetchSize,         ???) ;
-        SDB.getContext().setIfUndef(annotateGeneratedSQL,  true) ;
-        //SDB.getContext().setIfUndef(unionDefaultGraph,     false) ;
+            SDB.getContext().setIfUndef(useQuadRewrite,        false) ;
+            SDB.getContext().setIfUndef(streamGraphAPI,        false) ;
+            SDB.getContext().setIfUndef(jdbcStream,            true) ;
+            //SDB.getContext().setIfUndef(jdbcFetchSize,         ???) ;
+            SDB.getContext().setIfUndef(annotateGeneratedSQL,  true) ;
+            //SDB.getContext().setIfUndef(unionDefaultGraph,     false) ;
+            AssemblerVocab.init(); 
+            if ( JenaSystem.DEBUG_INIT )
+                System.err.println("SDB.init - finish") ;
+        }
     }
     
     /** RDF namespace prefix */

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-sdb/src/main/java/org/apache/jena/sdb/SDBFactory.java
----------------------------------------------------------------------
diff --git a/jena-sdb/src/main/java/org/apache/jena/sdb/SDBFactory.java b/jena-sdb/src/main/java/org/apache/jena/sdb/SDBFactory.java
index ce61c90..62b7288 100644
--- a/jena-sdb/src/main/java/org/apache/jena/sdb/SDBFactory.java
+++ b/jena-sdb/src/main/java/org/apache/jena/sdb/SDBFactory.java
@@ -35,6 +35,7 @@ import org.apache.jena.sdb.store.DatasetStore ;
 import org.apache.jena.sdb.store.StoreFactory ;
 import org.apache.jena.sparql.core.DatasetGraph ;
 import org.apache.jena.sparql.modify.GraphStoreBasic ;
+import org.apache.jena.system.JenaSystem ;
 import org.apache.jena.update.GraphStore ;
 
 /** Various operations to create or connect objects to do with SDB:
@@ -45,6 +46,7 @@ import org.apache.jena.update.GraphStore ;
 @SuppressWarnings("deprecation")
 public class SDBFactory
 {
+    static { JenaSystem.init() ; }
     // ---- Connections
     /**
      * Create a connection to a database.

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-sdb/src/main/java/org/apache/jena/sdb/assembler/StoreDescAssembler.java
----------------------------------------------------------------------
diff --git a/jena-sdb/src/main/java/org/apache/jena/sdb/assembler/StoreDescAssembler.java b/jena-sdb/src/main/java/org/apache/jena/sdb/assembler/StoreDescAssembler.java
index de8d9f5..6e5c140 100644
--- a/jena-sdb/src/main/java/org/apache/jena/sdb/assembler/StoreDescAssembler.java
+++ b/jena-sdb/src/main/java/org/apache/jena/sdb/assembler/StoreDescAssembler.java
@@ -24,7 +24,6 @@ import org.apache.jena.assembler.Assembler ;
 import org.apache.jena.assembler.Mode ;
 import org.apache.jena.assembler.assemblers.AssemblerBase ;
 import org.apache.jena.rdf.model.Resource ;
-import org.apache.jena.sdb.SDB ;
 import org.apache.jena.sdb.SDBException ;
 import org.apache.jena.sdb.StoreDesc ;
 import org.apache.jena.sdb.sql.MySQLEngineType ;
@@ -38,8 +37,6 @@ import org.slf4j.LoggerFactory;
 
 public class StoreDescAssembler extends AssemblerBase implements Assembler
 {
-    static { SDB.init() ; }
-    
     private static Logger log = LoggerFactory.getLogger(StoreDescAssembler.class) ;
     
     @Override

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-sdb/src/main/java/org/apache/jena/sdb/core/InitSDB.java
----------------------------------------------------------------------
diff --git a/jena-sdb/src/main/java/org/apache/jena/sdb/core/InitSDB.java b/jena-sdb/src/main/java/org/apache/jena/sdb/core/InitSDB.java
new file mode 100644
index 0000000..3461f15
--- /dev/null
+++ b/jena-sdb/src/main/java/org/apache/jena/sdb/core/InitSDB.java
@@ -0,0 +1,39 @@
+/**
+ * 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.sdb.core;
+
+import org.apache.jena.sdb.SDB ;
+import org.apache.jena.system.JenaSubsystemLifecycle ;
+
+/** Jena subsystem initialization */
+public class InitSDB implements JenaSubsystemLifecycle {
+    @Override
+    public void start() {
+        SDB.init() ;
+    }
+
+    @Override
+    public void stop() {
+    }
+    
+    @Override
+    public int level() {
+        return 9 ;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-sdb/src/main/java/sdb/cmd/CmdArgsDB.java
----------------------------------------------------------------------
diff --git a/jena-sdb/src/main/java/sdb/cmd/CmdArgsDB.java b/jena-sdb/src/main/java/sdb/cmd/CmdArgsDB.java
index 5431ac5..0b039aa 100644
--- a/jena-sdb/src/main/java/sdb/cmd/CmdArgsDB.java
+++ b/jena-sdb/src/main/java/sdb/cmd/CmdArgsDB.java
@@ -49,7 +49,6 @@ public abstract class CmdArgsDB extends CmdARQ
         addModule(modStore) ;
         addModule(modLogSQL) ;
         addModule(modTime) ;
-        ARQ.init() ;
         modVersion.addClass(Jena.class) ;
         modVersion.addClass(ARQ.class) ;
         modVersion.addClass(SDB.class) ;

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-sdb/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
----------------------------------------------------------------------
diff --git a/jena-sdb/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle b/jena-sdb/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
new file mode 100644
index 0000000..6c8c9b0
--- /dev/null
+++ b/jena-sdb/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
@@ -0,0 +1 @@
+org.apache.jena.sdb.core.InitSDB

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-sdb/src/test/java/org/apache/jena/sdb/test/SDBQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/jena-sdb/src/test/java/org/apache/jena/sdb/test/SDBQueryTestSuite.java b/jena-sdb/src/test/java/org/apache/jena/sdb/test/SDBQueryTestSuite.java
index c2f6d7d..f451a07 100644
--- a/jena-sdb/src/test/java/org/apache/jena/sdb/test/SDBQueryTestSuite.java
+++ b/jena-sdb/src/test/java/org/apache/jena/sdb/test/SDBQueryTestSuite.java
@@ -22,6 +22,7 @@ import junit.framework.TestSuite;
 import org.apache.jena.query.ARQ ;
 import org.apache.jena.sdb.SDB ;
 import org.apache.jena.sdb.test.junit.QueryTestSDBFactory ;
+import org.apache.jena.system.JenaSystem ;
 import org.junit.runner.RunWith;
 import org.junit.runners.AllTests;
 
@@ -40,14 +41,14 @@ public class SDBQueryTestSuite extends TestSuite
     
     private SDBQueryTestSuite()
     {
-        super("SDB") ;
-        
+        super("SDB Query") ;
+        SDB.init();
         if ( true )
             // PostgreSQL gets upset with comments in comments??
             ARQ.getContext().setFalse(SDB.annotateGeneratedSQL) ;
 
         QueryTestSDBFactory.make(this, SDBTestSetup.storeList, SDBTestSetup.manifestMain) ;
-        QueryTestSDBFactory.make(this, SDBTestSetup.storeListSimple, SDBTestSetup.manifestSimple) ;
+        //QueryTestSDBFactory.make(this, SDBTestSetup.storeListSimple, SDBTestSetup.manifestSimple) ;
     }
     
  

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-spatial/src/main/java/org/apache/jena/query/spatial/InitJenaSpatial.java
----------------------------------------------------------------------
diff --git a/jena-spatial/src/main/java/org/apache/jena/query/spatial/InitJenaSpatial.java b/jena-spatial/src/main/java/org/apache/jena/query/spatial/InitJenaSpatial.java
new file mode 100644
index 0000000..f6f5d33
--- /dev/null
+++ b/jena-spatial/src/main/java/org/apache/jena/query/spatial/InitJenaSpatial.java
@@ -0,0 +1,32 @@
+/**
+ * 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.spatial;
+
+import org.apache.jena.system.JenaSubsystemLifecycle ;
+
+public class InitJenaSpatial implements JenaSubsystemLifecycle {
+    @Override
+    public void start() {
+        SpatialQuery.init() ;
+    }
+
+    @Override
+    public void stop() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-spatial/src/main/java/org/apache/jena/query/spatial/SpatialQuery.java
----------------------------------------------------------------------
diff --git a/jena-spatial/src/main/java/org/apache/jena/query/spatial/SpatialQuery.java b/jena-spatial/src/main/java/org/apache/jena/query/spatial/SpatialQuery.java
index 630ec69..6788033 100644
--- a/jena-spatial/src/main/java/org/apache/jena/query/spatial/SpatialQuery.java
+++ b/jena-spatial/src/main/java/org/apache/jena/query/spatial/SpatialQuery.java
@@ -18,6 +18,8 @@
 
 package org.apache.jena.query.spatial;
 
+import com.spatial4j.core.context.SpatialContext ;
+
 import org.apache.jena.query.spatial.assembler.SpatialAssembler ;
 import org.apache.jena.query.spatial.pfunction.library.* ;
 import org.apache.jena.sparql.SystemARQ ;
@@ -27,14 +29,13 @@ import org.apache.jena.sparql.pfunction.PropertyFunction ;
 import org.apache.jena.sparql.pfunction.PropertyFunctionFactory ;
 import org.apache.jena.sparql.pfunction.PropertyFunctionRegistry ;
 import org.apache.jena.sparql.util.Symbol ;
-import org.apache.jena.tdb.TDB ;
-
-import com.spatial4j.core.context.SpatialContext ;
+import org.apache.jena.system.JenaSystem ;
 
 public class SpatialQuery
 {
-    private static boolean initialized = false ;
+    private static volatile boolean initialized = false ;
     private static Object lock = new Object() ;
+
     public static String NS = "http://jena.apache.org/spatial#" ;
     public static String IRI = "http://jena.apache.org/#spatial" ;
     public static final Symbol spatialIndex = Symbol.create(NS+"index") ;
@@ -52,16 +53,23 @@ public class SpatialQuery
     // an optional feature for WKT literals, loaded when necessary, but not required
     public static final String JTS_SPATIAL_CONTEXT_FACTORY_CLASS = "com.spatial4j.core.context.jts.JtsSpatialContextFactory"; 
     
-    static { init() ; }
+    static { JenaSystem.init(); }
     
     public static void init() 
     {
-        if ( initialized ) return ;
+        if ( initialized ) 
+            return ;
         synchronized(lock)
         {
-            if ( initialized ) return ;
+            if ( initialized ) {
+                if ( JenaSystem.DEBUG_INIT )
+                    System.err.println("SpatialQuery.init - skip") ;
+                return ; 
+            }
             initialized = true ;
-            TDB.init() ;
+            if ( JenaSystem.DEBUG_INIT )
+                System.err.println("SpatialQuery.init - start") ;
+
             SpatialAssembler.init() ;
             
             SystemInfo sysInfo = new SystemInfo(IRI, PATH, VERSION, BUILD_DATE) ;
@@ -122,6 +130,10 @@ public class SpatialQuery
                     return new WestPF() ;
                 }
             });
+            
+            if ( JenaSystem.DEBUG_INIT )
+                System.err.println("SpatialQuery.init - finish") ;
+
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-spatial/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
----------------------------------------------------------------------
diff --git a/jena-spatial/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle b/jena-spatial/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
new file mode 100644
index 0000000..c04f1fc
--- /dev/null
+++ b/jena-spatial/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
@@ -0,0 +1 @@
+org.apache.jena.query.spatial.InitJenaSpatial

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-tdb/src/main/java/org/apache/jena/tdb/TDB.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/TDB.java b/jena-tdb/src/main/java/org/apache/jena/tdb/TDB.java
index 786ded9..1c08724 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/TDB.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/TDB.java
@@ -30,7 +30,6 @@ import org.apache.jena.query.ARQ ;
 import org.apache.jena.query.Dataset ;
 import org.apache.jena.rdf.model.Model ;
 import org.apache.jena.reasoner.InfGraph ;
-import org.apache.jena.riot.RIOT ;
 import org.apache.jena.riot.lang.LangRDFXML ;
 import org.apache.jena.sparql.SystemARQ ;
 import org.apache.jena.sparql.core.DatasetGraph ;
@@ -42,6 +41,7 @@ import org.apache.jena.sparql.mgt.SystemInfo ;
 import org.apache.jena.sparql.util.Context ;
 import org.apache.jena.sparql.util.MappingRegistry ;
 import org.apache.jena.sparql.util.Symbol ;
+import org.apache.jena.system.JenaSystem ;
 import org.apache.jena.tdb.assembler.AssemblerTDB ;
 import org.apache.jena.tdb.modify.UpdateEngineTDB ;
 import org.apache.jena.tdb.setup.DatasetBuilderStd ;
@@ -102,15 +102,6 @@ public class TDB {
         return ARQ.getContext() ;
     }
 
-    // Called on assembler loading.
-    // Real initializtion happnes due to class static blocks.
-    /**
-     * TDB System initialization - normally, this is not explicitly called
-     * because all routes to use TDB will cause initialization to occur.
-     * However, calling it repeatedly is safe and low cost.
-     */
-    public static void init() {}
-
     /**
      * Release any and all system resources held by TDB. This does NOT close or
      * release datasets or graphs held by client code.
@@ -208,39 +199,52 @@ public class TDB {
             ((Sync)object).sync() ;
     }
 
-    private static boolean initialized = false ;
-    static {
-        initialization1() ;
-    }
+    private static final Object initLock = new Object() ;
+    private static volatile boolean initialized = false ;
+    static { JenaSystem.init(); }
 
-    private static synchronized void initialization1() {
+    /**
+     * TDB System initialization - normally, this is not explicitly called
+     * because all routes to use TDB will cause initialization to occur.
+     * However, calling it repeatedly is safe and low cost.
+     */
+    public static void init() {
         // Called at start.
-        if ( initialized )
+        if ( initialized ) {
             return ;
-        initialized = true ;
-
-        RIOT.init() ;
-        SystemTDB.init() ;
-        ARQ.init() ;
-        LangRDFXML.RiotUniformCompatibility = true ;
-        EnvTDB.processGlobalSystemProperties() ;
-
-        MappingRegistry.addPrefixMapping(SystemTDB.tdbSymbolPrefix, SystemTDB.symbolNamespace) ;
-        AssemblerUtils.init() ;
-        AssemblerTDB.init() ;
-        QueryEngineTDB.register() ;
-        UpdateEngineTDB.register() ;
-        MappingRegistry.addPrefixMapping(TDB.tdbSymbolPrefix, TDB.tdbParamNS) ;
-
-        wireIntoExecution() ;
-
-        if ( log.isDebugEnabled() )
-            log.debug("\n" + ARQ.getContext()) ;
+        }
+        
+        synchronized(initLock) {
+            if ( initialized ) {
+                if ( JenaSystem.DEBUG_INIT )
+                    System.err.println("TDB.init - return") ;
+                return ;
+            }
+            initialized = true ;
+            if ( JenaSystem.DEBUG_INIT )
+                System.err.println("TDB.init - start") ;
+            ARQ.init() ;
+            SystemTDB.init() ;
+            LangRDFXML.RiotUniformCompatibility = true ;
+            EnvTDB.processGlobalSystemProperties() ;
+
+            MappingRegistry.addPrefixMapping(SystemTDB.tdbSymbolPrefix, SystemTDB.symbolNamespace) ;
+            AssemblerUtils.init() ;
+            AssemblerTDB.init() ;
+            QueryEngineTDB.register() ;
+            UpdateEngineTDB.register() ;
+            MappingRegistry.addPrefixMapping(TDB.tdbSymbolPrefix, TDB.tdbParamNS) ;
+
+            wireIntoExecution() ;
+            if ( JenaSystem.DEBUG_INIT )
+                System.err.println("TDB.init - finish") ;
+        }
     }
 
     private static void wireIntoExecution() {
         // Globally change the stage generator to intercept BGP on TDB
-        StageGenerator orig = (StageGenerator)ARQ.getContext().get(ARQ.stageGenerator) ;
+        Context cxt = ARQ.getContext() ;
+        StageGenerator orig = StageBuilder.chooseStageGenerator(cxt) ; 
 
         // Wire in the TDB stage generator which will make TDB work whether
         // or not the TDB executor is used. This means that datasets of mixed

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-tdb/src/main/java/org/apache/jena/tdb/TDBFactory.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/TDBFactory.java b/jena-tdb/src/main/java/org/apache/jena/tdb/TDBFactory.java
index 26ea8c1..8b5021b 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/TDBFactory.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/TDBFactory.java
@@ -25,6 +25,7 @@ import org.apache.jena.query.Dataset ;
 import org.apache.jena.query.DatasetFactory ;
 import org.apache.jena.sparql.core.DatasetGraph ;
 import org.apache.jena.sparql.core.assembler.AssemblerUtils ;
+import org.apache.jena.system.JenaSystem ;
 import org.apache.jena.tdb.assembler.VocabTDB ;
 import org.apache.jena.tdb.base.file.Location ;
 import org.apache.jena.tdb.setup.StoreParams ;
@@ -36,6 +37,8 @@ import org.apache.jena.tdb.transaction.DatasetGraphTransaction ;
 /** Public factory for creating objects datasets backed by TDB storage */
 public class TDBFactory
 {
+    static { JenaSystem.init(); }
+    
     private TDBFactory() {} 
     
     /** Read the file and assembler a dataset */

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-tdb/src/main/java/org/apache/jena/tdb/store/GraphTDB.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/store/GraphTDB.java b/jena-tdb/src/main/java/org/apache/jena/tdb/store/GraphTDB.java
index be2d8b8..012b206 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/store/GraphTDB.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/store/GraphTDB.java
@@ -92,9 +92,9 @@ public class GraphTDB extends GraphView implements Closeable, Sync {
     @Override
     final public void close() {
         sync() ;
-        // Ignore - graphs are projectsion of the overlying database.
+        // Ignore - graphs are projections of the overlying database.
         // "Close graph" is messy in this projection world. 
-        //super.close() ;
+        super.close() ;
     }
 
     protected static ExtendedIterator<Triple> graphBaseFindDft(DatasetGraphTDB dataset, Triple triple) {

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-tdb/src/main/java/org/apache/jena/tdb/sys/InitTDB.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/sys/InitTDB.java b/jena-tdb/src/main/java/org/apache/jena/tdb/sys/InitTDB.java
new file mode 100644
index 0000000..4b3f50d
--- /dev/null
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/sys/InitTDB.java
@@ -0,0 +1,42 @@
+/**
+ * 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.tdb.sys;
+
+import org.apache.jena.system.JenaSubsystemLifecycle ;
+import org.apache.jena.tdb.TDB ;
+
+/** Jena subsystem initialization */
+public class InitTDB implements JenaSubsystemLifecycle {
+
+    @Override
+    public void start() {
+        TDB.init() ;
+    }
+
+    @Override
+    public void stop() {
+        // This is savage and does not take account of in-flight transactions.
+        TDB.closedown() ; 
+    }
+    
+    @Override
+    public int level() {
+        return 40 ;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-tdb/src/main/java/org/apache/jena/tdb/sys/SystemTDB.java
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/java/org/apache/jena/tdb/sys/SystemTDB.java b/jena-tdb/src/main/java/org/apache/jena/tdb/sys/SystemTDB.java
index fc278fa..d7c0e84 100644
--- a/jena-tdb/src/main/java/org/apache/jena/tdb/sys/SystemTDB.java
+++ b/jena-tdb/src/main/java/org/apache/jena/tdb/sys/SystemTDB.java
@@ -30,6 +30,7 @@ import org.apache.jena.query.ARQ ;
 import org.apache.jena.sparql.engine.optimizer.reorder.ReorderLib ;
 import org.apache.jena.sparql.engine.optimizer.reorder.ReorderTransformation ;
 import org.apache.jena.sparql.util.Symbol ;
+import org.apache.jena.system.JenaSystem ;
 import org.apache.jena.tdb.TDB ;
 import org.apache.jena.tdb.TDBException ;
 import org.apache.jena.tdb.base.block.FileMode ;
@@ -40,6 +41,8 @@ import org.slf4j.LoggerFactory ;
 
 public class SystemTDB
 {
+    static { JenaSystem.init(); }
+    
     // NB Same logger as the TDB class because this class is the system info but kept out of TDB javadoc.
     // It's visibility is TDB, not really public. 
     private static final Logger log = LoggerFactory.getLogger(TDB.class) ;

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-tdb/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
----------------------------------------------------------------------
diff --git a/jena-tdb/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle b/jena-tdb/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
new file mode 100644
index 0000000..a31e570
--- /dev/null
+++ b/jena-tdb/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
@@ -0,0 +1 @@
+org.apache.jena.tdb.sys.InitTDB

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-text/src/main/java/org/apache/jena/query/text/InitJenaText.java
----------------------------------------------------------------------
diff --git a/jena-text/src/main/java/org/apache/jena/query/text/InitJenaText.java b/jena-text/src/main/java/org/apache/jena/query/text/InitJenaText.java
new file mode 100644
index 0000000..2aae38e
--- /dev/null
+++ b/jena-text/src/main/java/org/apache/jena/query/text/InitJenaText.java
@@ -0,0 +1,33 @@
+/**
+ * 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.system.JenaSubsystemLifecycle ;
+
+public class InitJenaText implements JenaSubsystemLifecycle {
+    @Override
+    public void start() {
+        TextQuery.init() ;
+    }
+
+    @Override
+    public void stop() {
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-text/src/main/java/org/apache/jena/query/text/TextQuery.java
----------------------------------------------------------------------
diff --git a/jena-text/src/main/java/org/apache/jena/query/text/TextQuery.java b/jena-text/src/main/java/org/apache/jena/query/text/TextQuery.java
index 7fc01e9..5e57341 100644
--- a/jena-text/src/main/java/org/apache/jena/query/text/TextQuery.java
+++ b/jena-text/src/main/java/org/apache/jena/query/text/TextQuery.java
@@ -26,11 +26,11 @@ import org.apache.jena.sparql.pfunction.PropertyFunction ;
 import org.apache.jena.sparql.pfunction.PropertyFunctionFactory ;
 import org.apache.jena.sparql.pfunction.PropertyFunctionRegistry ;
 import org.apache.jena.sparql.util.Symbol ;
-import org.apache.jena.tdb.TDB ;
+import org.apache.jena.system.JenaSystem ;
 
 public class TextQuery
 {
-    private static boolean initialized = false ;
+    private static volatile boolean initialized = false ;
     private static Object lock = new Object() ;
     public static String NS = "http://jena.apache.org/text#" ;
     public static String IRI = "http://jena.apache.org/#text" ;
@@ -44,16 +44,22 @@ public class TextQuery
     public static final String VERSION      = metadata.get(PATH+".version", "unknown") ;
     public static final String BUILD_DATE   = metadata.get(PATH+".build.datetime", "unset") ;
     
-    static { init() ; }
+    static { JenaSystem.init(); }
     
     public static void init() 
     {
-        if ( initialized ) return ;
-        synchronized(lock)
-        {
-            if ( initialized ) return ;
+        if ( initialized ) 
+            return ;
+        synchronized(lock) {
+            if ( initialized ) {
+                if ( JenaSystem.DEBUG_INIT )
+                    System.err.println("TextQuery.init - skip") ;
+                return ; 
+            }
+            initialized = true ;
+            if ( JenaSystem.DEBUG_INIT )
+                System.err.println("TextQuery.init - start") ;
             initialized = true ;
-            TDB.init() ;
             TextAssembler.init() ;
             
             SystemInfo sysInfo = new SystemInfo(IRI, PATH, VERSION, BUILD_DATE) ;
@@ -65,6 +71,8 @@ public class TextQuery
                     return new TextQueryPF() ;
                 }
             });
+            if ( JenaSystem.DEBUG_INIT )
+                System.err.println("TextQuery.init - finish") ;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/55a8eab3/jena-text/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
----------------------------------------------------------------------
diff --git a/jena-text/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle b/jena-text/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
new file mode 100644
index 0000000..bef1d0b
--- /dev/null
+++ b/jena-text/src/main/resources/META-INF/services/org.apache.jena.system.JenaSubsystemLifecycle
@@ -0,0 +1 @@
+org.apache.jena.query.text.InitJenaText