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 2020/08/16 21:37:47 UTC

[jena] branch master updated: JENA-1929: Detect and use TDB database type for existing database.

This is an automated email from the ASF dual-hosted git repository.

andy pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/jena.git


The following commit(s) were added to refs/heads/master by this push:
     new 655e605  JENA-1929: Detect and use TDB database type for existing database.
     new 44e381a  Merge pull request #780 from afs/jena1929-detect-tdb
655e605 is described below

commit 655e605cbe219e929707931c160f90d8896b3ec5
Author: Andy Seaborne <an...@apache.org>
AuthorDate: Tue Aug 11 19:50:07 2020 +0100

    JENA-1929: Detect and use TDB database type for existing database.
---
 .../apache/jena/sparql/core/TransactionalLock.java |  26 +--
 .../src/main/java/org/apache/jena/atlas/io/IO.java |  24 +-
 jena-cmds/src/main/java/shacl/shacl_parse.java     |   4 +-
 .../java/org/apache/jena/tdb2/sys/DatabaseOps.java |   8 +-
 .../org/apache/jena/fuseki/server/FusekiInfo.java  |   2 +-
 .../apache/jena/fuseki/system/spot/SpotTDB1.java   | 229 ++++++++++++++++++
 .../apache/jena/fuseki/system/spot/SpotTDB2.java   | 259 +++++++++++++++++++++
 .../org/apache/jena/fuseki/system/spot/TDBOps.java |  59 +++++
 .../org/apache/jena/fuseki/main/cmds/DSGSetup.java |  67 ++++++
 .../apache/jena/fuseki/main/cmds/FusekiMain.java   |  10 +-
 .../{FusekiInitialConfig.java => FusekiArgs.java}  |  10 +-
 .../java/org/apache/jena/fuseki/cmd/FusekiCmd.java |  75 +++---
 .../org/apache/jena/fuseki/cmd/WebappDSGSetup.java |  69 ++++++
 .../jena/fuseki/webapp/FusekiServerListener.java   |   6 +-
 .../apache/jena/fuseki/webapp/FusekiWebapp.java    |  26 +--
 .../java/org/apache/jena/fuseki/ServerCtl.java     |   4 +-
 16 files changed, 784 insertions(+), 94 deletions(-)

diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/TransactionalLock.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/TransactionalLock.java
index df32c88..a23f96f 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/core/TransactionalLock.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/TransactionalLock.java
@@ -48,7 +48,7 @@ import org.apache.jena.sparql.JenaTransactionException ;
  *      {@literal @}Override public boolean supportsTransactions()     { return true; }
  *      {@literal @}Override public boolean supportsTransactionAbort() { return false; }
  *   </pre>
- */ 
+ */
 public class TransactionalLock implements Transactional {
 /*
     private final Transactional txn                     = TransactionalLock.createMRSW() ;
@@ -65,7 +65,7 @@ public class TransactionalLock implements Transactional {
     @Override public boolean supportsTransactions()     { return true; }
     @Override public boolean supportsTransactionAbort() { return false; }
  */
-    
+
     private ThreadLocal<Boolean>   inTransaction = ThreadLocal.withInitial(() -> Boolean.FALSE);
     private ThreadLocal<TxnType>   txnType = ThreadLocal.withInitial(() -> null);
     private ThreadLocal<ReadWrite> txnMode = ThreadLocal.withInitial(() -> null);
@@ -85,12 +85,12 @@ public class TransactionalLock implements Transactional {
     public static TransactionalLock createMRSW() {
         return create(new LockMRSW()) ;
     }
-    
+
     /** Create a Transactional using a mutex (exclusive - one at a time) lock */
     public static TransactionalLock createMutex() {
         return create(new LockMutex()) ;
     }
-    
+
     protected TransactionalLock(Lock lock) {
         this.lock = lock ;
     }
@@ -99,7 +99,7 @@ public class TransactionalLock implements Transactional {
     public void begin(ReadWrite readWrite) {
         begin(TxnType.convert(readWrite));
     }
-    
+
     @Override
     public void begin(TxnType txnType) {
         if ( isInTransaction() )
@@ -107,10 +107,10 @@ public class TransactionalLock implements Transactional {
         switch(txnType) {
             case READ_PROMOTE:
             case READ_COMMITTED_PROMOTE:
-                throw new UnsupportedOperationException("begin("+txnType+")");
+                throw new UnsupportedOperationException("begin(TxnType."+txnType+")");
             default:
         }
-        ReadWrite readWrite = TxnType.convert(txnType);  
+        ReadWrite readWrite = TxnType.convert(txnType);
         boolean isRead = readWrite.equals(ReadWrite.READ) ;
         lock.enterCriticalSection(isRead);
         this.inTransaction.set(true);
@@ -125,11 +125,11 @@ public class TransactionalLock implements Transactional {
     @Override public TxnType transactionType() {
         return Lib.readThreadLocal(txnType) ;
     }
-    
-    // Lock promotion required (Ok for mutex) 
-    
+
+    // Lock promotion required (Ok for mutex)
+
     @Override
-    public boolean promote(Promote txnType) { 
+    public boolean promote(Promote txnType) {
         return false;
     }
 
@@ -174,8 +174,8 @@ public class TransactionalLock implements Transactional {
             inTransaction.remove();
         }
     }
-    
+
     protected void error(String msg) {
-        throw new JenaTransactionException(msg) ; 
+        throw new JenaTransactionException(msg) ;
     }
 }
diff --git a/jena-base/src/main/java/org/apache/jena/atlas/io/IO.java b/jena-base/src/main/java/org/apache/jena/atlas/io/IO.java
index 3fa4fd2..fbc5e15 100644
--- a/jena-base/src/main/java/org/apache/jena/atlas/io/IO.java
+++ b/jena-base/src/main/java/org/apache/jena/atlas/io/IO.java
@@ -22,10 +22,7 @@ import java.io.*;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
-import java.nio.file.FileVisitResult;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.SimpleFileVisitor;
+import java.nio.file.*;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.GZIPOutputStream;
@@ -207,6 +204,25 @@ public class IO
         return new PrintWriter(asUTF8(out));
     }
 
+    public static boolean isEmptyDirectory(String directory) {
+        Path path = Paths.get(directory);
+        try(DirectoryStream<Path> dirStream = Files.newDirectoryStream(path)) {
+            return !dirStream.iterator().hasNext();
+        }
+        catch (NotDirectoryException ex) { return false ; }
+        catch (IOException ex) { IO.exception(ex); return false; }
+    }
+
+    public static boolean exists(String directory) {
+        Path path = Paths.get(directory);
+        return Files.exists(path);
+    }
+
+    public static boolean isDirectory(String directory) {
+        Path path = Paths.get(directory);
+        return Files.isDirectory(path);
+    }
+
     public static void close(org.apache.jena.atlas.lib.Closeable resource) {
         resource.close();
     }
diff --git a/jena-cmds/src/main/java/shacl/shacl_parse.java b/jena-cmds/src/main/java/shacl/shacl_parse.java
index 2070ebb..cdc35f1 100644
--- a/jena-cmds/src/main/java/shacl/shacl_parse.java
+++ b/jena-cmds/src/main/java/shacl/shacl_parse.java
@@ -71,7 +71,7 @@ public class shacl_parse extends CmdGeneral {
 
     @Override
     protected String getSummary() {
-        return getCommandName()+" -out=FMT,FMT FILE";
+        return getCommandName()+" --out=FMT[,FMT] FILE";
     }
 
     @Override
@@ -105,7 +105,7 @@ public class shacl_parse extends CmdGeneral {
                  printText = true;
              }
              if ( ! values.isEmpty() )
-                 throw new CmdException("Formats not recognized: "+values);
+                 throw new CmdException("Formats not recognized: "+values+" : Formats are 'text', 'compact', 'rdf' and 'all'");
 
          } else {
              printCompact = false;
diff --git a/jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/sys/DatabaseOps.java b/jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/sys/DatabaseOps.java
index cbb81b0..c6da6a5 100644
--- a/jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/sys/DatabaseOps.java
+++ b/jena-db/jena-tdb2/src/main/java/org/apache/jena/tdb2/sys/DatabaseOps.java
@@ -60,8 +60,8 @@ import org.slf4j.LoggerFactory;
  */
 public class DatabaseOps {
     private static Logger LOG = LoggerFactory.getLogger(DatabaseOps.class);
-    private static final String dbPrefix     = "Data";
-    private static final String SEP          = "-";
+    public static final String dbPrefix     = "Data";
+    public static final String SEP          = "-";
     private static final String startCount   = "0001";
 
     private static final String BACKUPS_DIR  = "Backups";
@@ -207,7 +207,7 @@ public class DatabaseOps {
     private static void compact(DatasetGraphSwitchable container, Location loc1, Location loc2) {
         if ( loc1.isMem() || loc2.isMem() )
             throw new TDBException("Compact involves a memory location: "+loc1+" : "+loc2);
-        
+
         copyFiles(loc1, loc2);
         StoreConnection srcConn = StoreConnection.connectExisting(loc1);
 
@@ -276,7 +276,7 @@ public class DatabaseOps {
             return;
         for ( File f : files ) {
             String fn = f.getName();
-            IOX.copy(loc1.getPath(fn), loc2.getPath(fn));  
+            IOX.copy(loc1.getPath(fn), loc2.getPath(fn));
         }
     }
 
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiInfo.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiInfo.java
index c4a5a65..a600df9 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiInfo.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiInfo.java
@@ -57,7 +57,7 @@ public class FusekiInfo {
                                       DataAccessPointRegistry dapRegistry,
                                       String datasetPath, String datasetDescription, String serverConfigFile, String staticFiles) {
         if ( datasetPath != null ) {
-            FmtLog.info(log,  "Dataset Type: %s", datasetDescription);
+            FmtLog.info(log,  "Database: %s", datasetDescription);
         }
         if ( serverConfigFile != null )
             FmtLog.info(log,  "Configuration file: %s", serverConfigFile);
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/system/spot/SpotTDB1.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/system/spot/SpotTDB1.java
new file mode 100644
index 0000000..5da5933
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/system/spot/SpotTDB1.java
@@ -0,0 +1,229 @@
+/*
+ * 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.system.spot;
+
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.jena.atlas.io.IO;
+import org.apache.jena.tdb.TDBException;
+import org.apache.jena.tdb.base.file.Location;
+import org.apache.jena.tdb.setup.StoreParams;
+import org.apache.jena.tdb.setup.StoreParamsCodec;
+import org.apache.jena.tdb.sys.Names;
+
+class SpotTDB1 {
+    /* TDB1 layout
+     *
+     * GOSP.dat  GSPO.dat      node2id.idn  OSPG.idn  POSG.idn       prefixes.dat   SPOG.dat
+     * GOSP.idn  GSPO.idn      nodes.dat    OSP.idn   POS.idn        prefixIdx.dat  SPOG.idn
+     * GPOS.dat  journal.jrnl  OSP.dat      POS.dat   prefix2id.dat  prefixIdx.idn  SPO.idn
+     * GPOS.idn  node2id.dat   OSPG.dat     POSG.dat  prefix2id.idn  SPO.dat
+     */
+
+    /* StoreParams
+     * primaryIndexTriples    dft:SPO
+     * tripleIndexes          dft:[SPO, POS, OSP]
+     *
+     * primaryIndexQuads      dft:GSPO
+     * quadIndexes            dft:[GSPO, GPOS, GOSP, POSG, OSPG, SPOG]
+     *
+     * primaryIndexPrefix     dft:GPU
+     * prefixIndexes          dft:[GPU]
+     * -- Actual name of GPU index.
+     * indexPrefix            dft:prefixIdx BPT
+     *
+     * -- NodeTable
+     * indexNode2Id           dft:node2id
+     * indexId2Node           dft:nodes
+     *
+     * -- NodeTable
+     * prefixNode2Id          dft:prefix2id BPT *
+     * prefixId2Node          dft:prefixes DAT *
+     */
+
+    /**
+     * Test to see is a location is either empty (and a fresh TDB1 database can be
+     * created there) or has looks like it is an existing TDB1 database. See
+     * {@link #checkTDB1(Location)} for a test that the location is a valid, existing
+     * database.
+     */
+    public static boolean isTDB1(String pathname) {
+        return isTDB1(Location.create(pathname));
+    }
+
+    /**
+     * Test to see is a location is either empty (and a fresh TDB1 database can be
+     * created there) or has looks like it is an existing TDB1 database. See
+     * {@link #checkTDB1(Location)} for a test that the location is a valid, existing
+     * database.
+     */
+    public static boolean isTDB1(Location location) {
+        if ( location.isMem() )
+            return true;
+
+        if ( ! location.exists() )
+            return false;
+        if ( isEmpty(location) )
+            return true;
+
+        return isTDB1_Storage(location);
+    }
+
+    /** Quick check for TDB1. Does not check for validity. */
+    private static boolean isTDB1_Storage(Location location) {
+        // Very occasionally, deleting the journal makes sense.
+        //return exists(location, Names.journalFileBase, Names.extJournal);
+        // Test for triples primary index
+        StoreParams params = getStoreParams(location);
+        return exists(location, params.getPrimaryIndexTriples(), Names.bptExtTree, Names.bptExtRecords);
+    }
+
+    /**
+     * Check all files exist for a TDB1 database, or the area is empty (and so a new
+     * database can be created in the location). Throw an exception if a file
+     * is missing.
+     */
+    public static void checkTDB1(String pathname) {
+        checkTDB1(Location.create(pathname));
+    }
+
+    /**
+     * Check all files exist for a TDB1 database, or the area is empty (and so a new
+     * database can be created in the location). Throw an exception if a file
+     * is missing.
+     */
+    public static void checkTDB1(Location location) {
+        if ( location.isMem() )
+            return;
+
+        if ( isEmpty(location) )
+            return;
+
+        if ( ! isTDB1(location) )
+            throw new TDBException("Not a TDB1 location: "+location);
+
+        // Places for StoreParams: location or default
+        StoreParams params = getStoreParams(location);
+
+        validate(location, Names.journalFileBase, Names.extJournal);
+
+        // Check for indexes
+        containsIndex(params.getPrimaryIndexTriples(), params.getTripleIndexes());
+        validateBPT(location, params.getTripleIndexes());
+
+        containsIndex(params.getPrimaryIndexQuads(), params.getTripleIndexes());
+        validateBPT(location, params.getQuadIndexes());
+
+        // prefixes. GPU
+        containsIndex(params.getPrimaryIndexPrefix(), params.getPrefixIndexes());
+        // GPOU is not in files "GPU" -- validateBPT(location, params.getPrefixIndexes());
+        // Filename of GPU.
+        validateBPT(location, params.getIndexPrefix());
+
+        //---- Node tables.
+        validateBPT(location, params.getIndexNode2Id());
+        validateDAT(location, params.getIndexId2Node());
+
+        validateBPT(location, params.getPrefixNode2Id());
+        validateDAT(location, params.getPrefixId2Node());
+    }
+
+    // Places for StoreParams: location or default
+    private static StoreParams getStoreParams(Location location) {
+        StoreParams params = StoreParamsCodec.read(location);
+        if ( params == null )
+            params = StoreParams.getDftStoreParams();
+        return params;
+    }
+
+    private static void validateDAT(Location location, String file) {
+        validate(location, file, Names.extNodeData);
+    }
+
+    private static void containsIndex(String primaryIdx, String[] tripleIndexes) {
+        if ( Arrays.stream(tripleIndexes).findFirst().isPresent() )
+            return;
+        List<String> list = Arrays.asList(tripleIndexes);
+        throw new TDBException("Missing primary in index list: "+primaryIdx+" "+list);
+
+    }
+
+    private static void validateBPT(Location location, String[] indexes) {
+        for ( String idx : indexes )
+            validateBPT(location, idx);
+    }
+
+    private static void validateBPT(Location location, String index) {
+        validate(location, index, Names.bptExtTree, Names.bptExtRecords);
+    }
+
+    private static void validate(Location location, String basename, String ... exts) {
+        if ( exts.length == 0 ) {
+            String fn = location.absolute(basename);
+            if ( location.exists(basename) )
+                good(fn);
+            else
+                throw missingFile(fn);
+            return;
+        }
+
+        for ( String ext : exts ) {
+            String fn = location.absolute(basename, ext);
+            if ( location.exists(basename, ext) )
+                good(location, basename, ext);
+            else
+                throw missingFile(fn);
+        }
+    }
+
+    private static boolean isEmpty(Location location) {
+        Path path = Paths.get(location.getDirectoryPath());
+        try(DirectoryStream<Path> entries = Files.newDirectoryStream(path)) {
+            return ! entries.iterator().hasNext();
+        } catch(IOException ex) {
+            IO.exception(ex);
+            return false;
+        }
+    }
+
+    private static boolean exists(Location location, String baseName, String ... exts) {
+        if ( exts.length == 0 )
+            return location.exists(baseName);
+
+        for ( String ext : exts ) {
+            if ( ! location.exists(baseName, ext) )
+                return false;
+        }
+        return true;
+    }
+
+    private static RuntimeException missingFile(String filename) {
+        return new TDBException("No such file: "+filename);
+    }
+
+    private static void good(Location location, String basename, String ext) {}
+    private static void good(String filename) {}
+}
+
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/system/spot/SpotTDB2.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/system/spot/SpotTDB2.java
new file mode 100644
index 0000000..5e16493
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/system/spot/SpotTDB2.java
@@ -0,0 +1,259 @@
+/*
+ * 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.system.spot;
+
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.jena.atlas.io.IO;
+import org.apache.jena.dboe.base.file.Location;
+import org.apache.jena.dboe.sys.Names;
+import org.apache.jena.tdb2.TDBException;
+import org.apache.jena.tdb2.params.StoreParams;
+import org.apache.jena.tdb2.params.StoreParamsCodec;
+import org.apache.jena.tdb2.sys.DatabaseOps;
+import org.apache.jena.tdb2.sys.IOX;
+import org.apache.jena.tdb2.sys.Util;
+
+class SpotTDB2 {
+
+    /* TDB2 layout
+     * top:
+     *
+     * Data-0001/  tdb.lock
+     *
+     * Data-0001:
+     * GOSP.bpt  GPU.dat       nodes.dat       OSPG.dat  POSG.idn           SPO.bpt
+     * GOSP.dat  GPU.idn       nodes-data.bdf  OSPG.idn  POS.idn            SPO.dat
+     * GOSP.idn  GSPO.bpt      nodes-data.obj  OSP.idn   prefixes.bpt       SPOG.bpt
+     * GPOS.bpt  GSPO.dat      nodes.idn       POS.bpt   prefixes.dat       SPOG.dat
+     * GPOS.dat  GSPO.idn      OSP.bpt         POS.dat   prefixes-data.bdf  SPOG.idn
+     * GPOS.idn  journal.jrnl  OSP.dat         POSG.bpt  prefixes-data.obj  SPO.idn
+     * GPU.bpt   nodes.bpt     OSPG.bpt        POSG.dat  prefixes.idn       tdb.lock
+     */
+//
+//    // TEMP
+//    // These are in DatabaseOps
+//    private static final String dbPrefix     = "Data";
+//    private static final String SEP          = "-";
+
+    public static boolean isTDB2(String pathname) {
+        return isTDB2(Location.create(pathname));
+    }
+
+    /**
+     * Test to see is a location is either empty (and a fresh TDB2 database can be
+     * created there) or has looks like it is an existing TDB2 database.
+     * See {@link #checkTDB2(Location)} for a test that the location is a valid, existing
+     * database.
+     */
+    public static boolean isTDB2(Location location) {
+        if ( location.isMem() )
+            return true;
+        if ( ! location.exists() )
+            return false;
+        if ( isEmpty(location) )
+            return true;
+        // Look for Data-*
+        Path db = storageDir(location);
+        if ( db == null )
+            // Uninitialized?
+            // Or TDB1?
+            return ! SpotTDB1.isTDB1(location.getDirectoryPath());
+        // Validate storage.
+        Location storageLocation = IOX.asLocation(db);
+        return isTDB2_Storage(storageLocation);
+    }
+
+    /** Quick check for TDB2. Does not check for validity. */
+    private static boolean isTDB2_Storage(Location location) {
+        // Journal check.
+        //return exists(location, Names.journalFileBase, Names.extJournal);
+        // Test for triples primary index
+        StoreParams params = getStoreParams(location);
+        return exists(location, params.getPrimaryIndexTriples(),
+            Names.extBptTree, Names.extBptRecords, Names.extBptState);
+    }
+
+    public static void checkTDB2(String pathname) {
+        checkTDB2(Location.create(pathname));
+    }
+
+    public static void checkTDB2(Location location) {
+        if ( location.isMem() )
+            return;
+        if ( isEmpty(location) )
+            return;
+        Path db = storageDir(location);
+        if ( db == null )
+            // Uninitialized
+            return ;
+        // Storage. Easier to work in "Location".
+        Location locStorage = Location.create(db);
+        checkStorageArea(locStorage);
+    }
+
+    /** Return the current active database area within a database directory. */
+    private static Path storageDir(Location location) {
+        // Database directory
+        Path path = IOX.asPath(location);
+        // Storage directory in database directory.
+        Path db = findLocation(path, DatabaseOps.dbPrefix);
+        return db;
+    }
+
+    private static Path findLocation(Path directory, String namebase) {
+        if ( ! Files.exists(directory) )
+            return null;
+        // In-order, low to high.
+        List<Path> maybe = IOX.scanForDirByPattern(directory, namebase, DatabaseOps.SEP);
+        return Util.getLastOrNull(maybe);
+    }
+
+    // Places for StoreParams: location or default
+    private static StoreParams getStoreParams(Location location) {
+        StoreParams params = StoreParamsCodec.read(location);
+        if ( params == null )
+            params = StoreParams.getDftStoreParams();
+        return params;
+    }
+
+    /**
+     * Check all files exist for a TDB2 database, or the area is empty (and so a new
+     * database can be created in the location). Throw {@link TDBException} is a file
+     * is missing.
+     */
+    private static void checkStorageArea(Location location) {
+
+        if ( location.isMem() )
+            return;
+
+        if ( isEmpty(location) )
+            return;
+
+        // Journal, fixed name.
+        validate(location, Names.journalFileBase, Names.extJournal);
+
+        // Places for StoreParams: location or default
+        StoreParams params = getStoreParams(location);
+
+        // Check for indexes
+        containsIndex(params.getPrimaryIndexTriples(), params.getTripleIndexes());
+        validateBPT(location, params.getTripleIndexes());
+
+        containsIndex(params.getPrimaryIndexQuads(), params.getTripleIndexes());
+        validateBPT(location, params.getQuadIndexes());
+
+        // prefixes. GPU
+        containsIndex(params.getPrimaryIndexPrefix(), params.getPrefixIndexes());
+        // GPU is not in files "GPU"
+        // Filename of GPU.
+        validateBPT(location, params.getPrefixIndexes());
+
+        //---- Node tables.
+        /*
+         * nodes.bpt
+         * nodes.dat
+         * nodes-data.bdf
+         * nodes-data.obj
+         */
+        validateBPT(location, params.getNodeTableBaseName());
+        validateDAT(location, params.getNodeTableBaseName()+"-data");
+
+        /*
+         * prefixes...
+         */
+        // XXX validateBPT(location, params.getPrefixTableBaseName());
+        validateBPT(location, params.getPrefixTableBaseName());
+        validateDAT(location, params.getPrefixTableBaseName()+"-data");
+
+    }
+
+    private static void validateDAT(Location location, String file) {
+        validate(location, file, Names.extObjNodeData, Names.extBdfState);
+    }
+
+    private static void containsIndex(String primaryIdx, String[] tripleIndexes) {
+        if ( Arrays.stream(tripleIndexes).findFirst().isPresent() )
+            return;
+        List<String> list = Arrays.asList(tripleIndexes);
+        throw new TDBException("Missing primary in index list: "+primaryIdx+" "+list);
+    }
+
+    private static void validateBPT(Location location, String[] indexes) {
+        for ( String idx : indexes )
+            validateBPT(location, idx);
+    }
+
+    private static void validateBPT(Location location, String index) {
+        validate(location, index, Names.extBptTree, Names.extBptRecords, Names.extBptState);
+    }
+
+    private static void validate(Location location, String basename, String ... exts) {
+        if ( exts.length == 0 ) {
+            String fn = location.absolute(basename);
+            if ( location.exists(basename) )
+                good(fn);
+            else
+                throw missingFile(fn);
+            return;
+        }
+
+        for ( String ext : exts ) {
+            String fn = location.absolute(basename, ext);
+            if ( location.exists(basename, ext) )
+                good(location, basename, ext);
+            else
+                throw missingFile(fn);
+        }
+    }
+
+    private static boolean isEmpty(Location location) {
+        Path path = Paths.get(location.getDirectoryPath());
+        try(DirectoryStream<Path> entries = Files.newDirectoryStream(path)) {
+            return ! entries.iterator().hasNext();
+        } catch(IOException ex) {
+            IO.exception(ex);
+            return false;
+        }
+    }
+
+    private static boolean exists(Location location, String baseName, String ... exts) {
+        if ( exts.length == 0 )
+            return location.exists(baseName);
+
+        for ( String ext : exts ) {
+            if ( ! location.exists(baseName, ext) ) return false;
+        }
+        return true;
+    }
+
+    private static RuntimeException missingFile(String filename) {
+        return new TDBException("No such file: "+filename);
+    }
+
+    private static void good(Location location, String basename, String ext) {}
+    private static void good(String filename) {}
+}
+
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/system/spot/TDBOps.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/system/spot/TDBOps.java
new file mode 100644
index 0000000..cd16491
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/system/spot/TDBOps.java
@@ -0,0 +1,59 @@
+/*
+ * 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.system.spot;
+
+/** Operations related to TDB1 and TDB2 */
+public class TDBOps {
+    /**
+     * Test to see is a location appears to be a TDB1 database, or a database can be
+     * created at the pathname.
+     */
+    public static boolean isTDB1(String pathname) {
+        return SpotTDB1.isTDB1(pathname);
+    }
+
+    /**
+     * Check whether a location is a TDB1 database, performing some validation
+     * checks. Throw an exception if invalid. This check passes if a database can be
+     * created at the location (i.e. it is empty or does not exist yet).
+     * Validation is of the file structure, not the database contents.
+     */
+    public static void checkTDB1(String pathname) {
+        SpotTDB1.checkTDB1(pathname);
+    }
+
+    /**
+     * Test to see is a location appears to be a TDB2 database, or a database can be
+     * created at the pathname.
+     */
+    public static boolean isTDB2(String pathname) {
+        return SpotTDB2.isTDB2(pathname);
+    }
+
+    /**
+     * Check whether a location is a TDB2 database, performing some validation
+     * checks. Throw an exception if invalid. This check passes if a database can be
+     * created at the location (i.e. it is empty or does not exist yet).
+     * Validation is of the file structure, not the database contents.
+     */
+    public static void checkTDB2(String pathname) {
+        SpotTDB2.checkTDB2(pathname);
+    }
+}
+
diff --git a/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/cmds/DSGSetup.java b/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/cmds/DSGSetup.java
new file mode 100644
index 0000000..b4cb255
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/cmds/DSGSetup.java
@@ -0,0 +1,67 @@
+/*
+ * 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.main.cmds;
+
+import jena.cmd.CmdException;
+import org.apache.jena.atlas.io.IO;
+import org.apache.jena.fuseki.system.spot.TDBOps;
+import org.apache.jena.tdb.TDBFactory;
+import org.apache.jena.tdb2.DatabaseMgr;
+
+/*package*/ class DSGSetup {
+
+    /**
+     * Given a path name and a preference of TDB1/TDB2 for new databases, return
+     * details of the setup to use.
+     */
+    /*package*/ static void setupTDB(String directory, boolean useTDB2, ServerConfig serverConfig) {
+        if ( ! IO.exists(directory) )
+            throw new CmdException("Does not exist: " + directory);
+        if ( ! IO.isDirectory(directory) )
+            throw new CmdException("Not a directory: " + directory);
+
+        if ( IO.isEmptyDirectory(directory) ) {
+            if ( useTDB2 )
+                setupTDB2(directory, serverConfig);
+            else
+                setupTDB1(directory, serverConfig);
+            return;
+        }
+
+        // Exists, not empty or does not exist
+        if ( TDBOps.isTDB1(directory) ) {
+            setupTDB1(directory, serverConfig);
+            return;
+        } else if ( TDBOps.isTDB2(directory) ) {
+            setupTDB2(directory, serverConfig);
+            return;
+        } else
+            throw new CmdException("Directory not a database: " + directory);
+    }
+
+    private static void setupTDB1(String directory, ServerConfig serverConfig) {
+        serverConfig.datasetDescription = "TDB1 dataset: location="+directory;
+        serverConfig.dsg = TDBFactory.createDatasetGraph(directory);
+    }
+
+    private static void setupTDB2(String directory, ServerConfig serverConfig) {
+        serverConfig.datasetDescription = "TDB2 dataset: location="+directory;
+        serverConfig.dsg = DatabaseMgr.connectDatasetGraph(directory);
+    }
+}
diff --git a/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/cmds/FusekiMain.java b/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/cmds/FusekiMain.java
index 11b4545..d1aeb1e 100644
--- a/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/cmds/FusekiMain.java
+++ b/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/cmds/FusekiMain.java
@@ -248,7 +248,7 @@ public class FusekiMain extends CmdARQ {
 
         // Which TDB to use to create a command line TDB database.
         useTDB2 = contains(argTDB2mode);
-        String tag = useTDB2 ? "TDB2" : "TDB";
+        String tag = useTDB2 ? "TDB2" : "TDB1";
 
         if ( allowEmpty ) {
             serverConfig.empty = true;
@@ -317,12 +317,8 @@ public class FusekiMain extends CmdARQ {
         }
 
         if ( contains(argTDB) ) {
-            String dir = getValue(argTDB);
-            serverConfig.datasetDescription = tag+" dataset: "+dir;
-            serverConfig.dsg =
-                useTDB2
-                ? DatabaseMgr.connectDatasetGraph(dir)
-                    : TDBFactory.createDatasetGraph(dir);
+            String directory = getValue(argTDB);
+            DSGSetup.setupTDB(directory, useTDB2, serverConfig);
         }
 
         if ( contains(ModAssembler.assemblerDescDecl) ) {
diff --git a/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiInitialConfig.java b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiArgs.java
similarity index 91%
rename from jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiInitialConfig.java
rename to jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiArgs.java
index 71c4894..bae04cc 100644
--- a/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiInitialConfig.java
+++ b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiArgs.java
@@ -25,7 +25,7 @@ import org.apache.jena.fuseki.Fuseki;
 import org.apache.jena.sparql.core.DatasetGraph;
 
 /** Dataset setup (command line, config file) for a dataset (or several if config file) */
-public class FusekiInitialConfig {
+public class FusekiArgs {
     public boolean quiet = false;
     public boolean verbose = Fuseki.verboseLogging;
 
@@ -33,9 +33,9 @@ public class FusekiInitialConfig {
     // through the command line processing should not allow --conf and a templated /dataset.
 
     // Label for dataset setup (command line).
-    public String datasetDescription  = null;
+    public String    datasetDescription  = null;
     // Either this ... command line ...
-    public String    argTemplateFile  = null;              // Command list args --mem, --loc, --memtdb
+    public String    templateFile     = null;              // Command list args --mem, --loc, --memtdb
     public String    datasetPath      = null;              // Dataset name on the command line.
     public boolean   allowUpdate      = false;             // Command line --update.
     // Special case - prebuilt dataset.  Uses datasetPath.
@@ -49,10 +49,10 @@ public class FusekiInitialConfig {
     // Additional information.
     public Map<String,String> params  = new HashMap<>();
 
-    public FusekiInitialConfig() {}
+    public FusekiArgs() {}
 
     public void reset() {
-        argTemplateFile  = null;
+        templateFile  = null;
         datasetPath = null;
         allowUpdate = false;
         dsg = null;
diff --git a/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiCmd.java b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiCmd.java
index 1c0c785..028bb55 100644
--- a/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiCmd.java
+++ b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/FusekiCmd.java
@@ -114,7 +114,7 @@ public class FusekiCmd {
             jettyServerConfig.verboseLogging = false;
         }
 
-        private final FusekiInitialConfig cmdLineConfig  = new FusekiInitialConfig();
+        private final FusekiArgs cmdLine = new FusekiArgs();
         private boolean useTDB2;
 
         public FusekiCmdInner(String... argv) {
@@ -169,8 +169,8 @@ public class FusekiCmd {
                 jettyServerConfig.verboseLogging = true;
                 // Output is still at level INFO (currently)
             }
-            cmdLineConfig.quiet = super.isQuiet();
-            cmdLineConfig.verbose = super.isVerbose();
+            cmdLine.quiet = super.isQuiet();
+            cmdLine.verbose = super.isVerbose();
 
             // Any final tinkering with FUSEKI_HOME and FUSEKI_BASE, e.g. arguments like --home, --base, then ....
             FusekiEnv.resetEnvironment();
@@ -178,8 +178,8 @@ public class FusekiCmd {
             Logger log = Fuseki.serverLog;
 
             if ( contains(argFusekiConfig) ) {
-                cmdLineConfig.fusekiCmdLineConfigFile = getValue(argFusekiConfig);
-                cmdLineConfig.datasetDescription = "Configuration: "+cmdLineConfig.fusekiCmdLineConfigFile;
+                cmdLine.fusekiCmdLineConfigFile = getValue(argFusekiConfig);
+                cmdLine.datasetDescription = "Configuration: "+cmdLine.fusekiCmdLineConfigFile;
             }
 
             ArgDecl assemblerDescDecl = new ArgDecl(ArgDecl.HasValue, "desc", "dataset");
@@ -199,7 +199,7 @@ public class FusekiCmd {
             if ( contains(argMemTDB) )
                 x++;
 
-            if ( cmdLineConfig.fusekiCmdLineConfigFile != null ) {
+            if ( cmdLine.fusekiCmdLineConfigFile != null ) {
                 if ( x >= 1 )
                     throw new CmdException("Dataset specified on the command line but a configuration file also given.");
             } else {
@@ -212,7 +212,7 @@ public class FusekiCmd {
             if ( cmdlineConfigPresent && getPositional().size() == 0 )
                 throw new CmdException("Missing service name");
 
-            if ( cmdLineConfig.fusekiCmdLineConfigFile != null && getPositional().size() > 0 )
+            if ( cmdLine.fusekiCmdLineConfigFile != null && getPositional().size() > 0 )
                 throw new CmdException("Service name will come from --conf; no command line service name allowed");
 
 
@@ -222,35 +222,35 @@ public class FusekiCmd {
             if ( cmdlineConfigPresent && getPositional().size() > 1 )
                 throw new CmdException("Multiple dataset path names given");
 
-            if ( ! cmdlineConfigPresent && cmdLineConfig.fusekiCmdLineConfigFile == null ) {
+            if ( ! cmdlineConfigPresent && cmdLine.fusekiCmdLineConfigFile == null ) {
                 // Turn command line argument into an absolute file name.
                 FusekiEnv.setEnvironment();
                 Path cfg = FusekiEnv.FUSEKI_BASE.resolve(FusekiWebapp.DFT_CONFIG).toAbsolutePath();
                 if ( Files.exists(cfg) )
-                    cmdLineConfig.fusekiServerConfigFile = cfg.toString();
+                    cmdLine.fusekiServerConfigFile = cfg.toString();
             }
 
             // Which TDB to use to create a command line TDB database.
             useTDB2 = contains(argTDB2mode);
 
-            cmdLineConfig.allowUpdate = contains(argUpdate);
+            cmdLine.allowUpdate = contains(argUpdate);
 
             if ( contains(argMem) ) {
-                cmdLineConfig.datasetDescription = "in-memory";
+                cmdLine.datasetDescription = "in-memory";
                 // Only one setup should be called by the test above but to be safe
                 // and in case of future changes, clear the configuration.
-                cmdLineConfig.reset();
-                cmdLineConfig.argTemplateFile = Template.templateTIM_MemFN;
+                cmdLine.reset();
+                cmdLine.templateFile = Template.templateTIM_MemFN;
                 // Always allow.
-                cmdLineConfig.allowUpdate = true;
+                cmdLine.allowUpdate = true;
             }
 
             if ( contains(argFile) ) {
                 List<String> filenames = getValues(argFile);
                 // Directly populate the dataset.
-                cmdLineConfig.reset();
-                cmdLineConfig.dsg = DatasetGraphFactory.createTxnMem();
-                cmdLineConfig.datasetDescription = "in-memory, with files loaded";
+                cmdLine.reset();
+                cmdLine.dsg = DatasetGraphFactory.createTxnMem();
+                cmdLine.datasetDescription = "in-memory, with files loaded";
                 for ( String filename : filenames ) {
                     String pathname = filename;
                     if ( filename.startsWith("file:") )
@@ -260,10 +260,10 @@ public class FusekiCmd {
                     Lang language = RDFLanguages.filenameToLang(filename);
                     if ( language == null )
                         throw new CmdException("Can't guess language for file: " + filename);
-                    Txn.executeWrite(cmdLineConfig.dsg, ()->{
+                    Txn.executeWrite(cmdLine.dsg, ()->{
                         try {
                             log.info("Dataset: in-memory: load file: " + filename);
-                            RDFDataMgr.read(cmdLineConfig.dsg, filename);
+                            RDFDataMgr.read(cmdLine.dsg, filename);
                         } catch (RiotException ex) {
                             throw new CmdException("Failed to load file: " + filename);
                         }
@@ -272,40 +272,37 @@ public class FusekiCmd {
             }
 
             if ( contains(argMemTDB) ) {
-                //log.info("TDB dataset: in-memory");
-                cmdLineConfig.reset();
-                cmdLineConfig.argTemplateFile = useTDB2 ? Template.templateTDB2_MemFN : Template.templateTDB1_MemFN;
-                cmdLineConfig.params.put(Template.DIR, Names.memName);
+                cmdLine.reset();
+                cmdLine.templateFile = useTDB2 ? Template.templateTDB2_MemFN : Template.templateTDB1_MemFN;
+                cmdLine.params.put(Template.DIR, Names.memName);
                 // Always allow.
-                cmdLineConfig.allowUpdate = true;
-                cmdLineConfig.datasetDescription = useTDB2 ? "TDB2 dataset (in-memory)" : "TDB dataset (in-memory)";
+                cmdLine.allowUpdate = true;
+                cmdLine.datasetDescription = useTDB2 ? "TDB2 dataset (in-memory)" : "TDB1 dataset (in-memory)";
             }
 
             if ( contains(argTDB) ) {
-                cmdLineConfig.reset();
-                cmdLineConfig.argTemplateFile =
-                    useTDB2 ? Template.templateTDB2_DirFN : Template.templateTDB1_DirFN;
                 String dir = getValue(argTDB);
-                cmdLineConfig.params.put(Template.DIR, dir);
-                cmdLineConfig.datasetDescription = useTDB2 ? "TDB2 dataset: "+dir : "TDB dataset: "+dir;
+                cmdLine.params.put(Template.DIR, dir);
+                cmdLine.reset();
+                WebappDSGSetup.setup(dir, useTDB2, cmdLine);
             }
 
             // Otherwise
             if ( contains(assemblerDescDecl) ) {
-                cmdLineConfig.datasetDescription = "Assembler: "+ modDataset.getAssemblerFile();
+                cmdLine.datasetDescription = "Assembler: "+ modDataset.getAssemblerFile();
                 // Need to add service details.
                 Dataset ds = modDataset.createDataset();
                 //cmdLineDataset.dsg = ds.asDatasetGraph();
             }
 
             if ( cmdlineConfigPresent ) {
-                cmdLineConfig.datasetPath = getPositionalArg(0);
-                if ( cmdLineConfig.datasetPath.length() > 0 && !cmdLineConfig.datasetPath.startsWith("/") )
-                    throw new CmdException("Dataset path name must begin with a /: " + cmdLineConfig.datasetPath);
-                if ( ! cmdLineConfig.allowUpdate )
-                    Fuseki.serverLog.info("Running in read-only mode for "+cmdLineConfig.datasetPath);
+                cmdLine.datasetPath = getPositionalArg(0);
+                if ( cmdLine.datasetPath.length() > 0 && !cmdLine.datasetPath.startsWith("/") )
+                    throw new CmdException("Dataset path name must begin with a /: " + cmdLine.datasetPath);
+                if ( ! cmdLine.allowUpdate )
+                    Fuseki.serverLog.info("Running in read-only mode for "+cmdLine.datasetPath);
                 // Include the dataset name as NAME for any templates.
-                cmdLineConfig.params.put(Template.NAME,  cmdLineConfig.datasetPath);
+                cmdLine.params.put(Template.NAME,  cmdLine.datasetPath);
             }
 
             // ---- Jetty server
@@ -360,7 +357,7 @@ public class FusekiCmd {
         @Override
         protected void exec() {
             try {
-                runFuseki(cmdLineConfig, jettyServerConfig);
+                runFuseki(cmdLine, jettyServerConfig);
             } catch (FusekiException ex) {
                 throw new TerminationException(1);
             }
@@ -373,7 +370,7 @@ public class FusekiCmd {
     }
 
     /** Configure and run a Fuseki server - this function does not return except for error starting up */
-    public static void runFuseki(FusekiInitialConfig serverConfig, JettyServerConfig jettyConfig) {
+    public static void runFuseki(FusekiArgs serverConfig, JettyServerConfig jettyConfig) {
         FusekiServerListener.initialSetup = serverConfig;
         JettyFusekiWebapp.initializeServer(jettyConfig);
         JettyFusekiWebapp.instance.start();
diff --git a/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/WebappDSGSetup.java b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/WebappDSGSetup.java
new file mode 100644
index 0000000..7217d2c
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/cmd/WebappDSGSetup.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.fuseki.cmd;
+
+import jena.cmd.CmdException;
+import org.apache.jena.atlas.io.IO;
+import org.apache.jena.fuseki.mgt.Template;
+import org.apache.jena.fuseki.system.spot.TDBOps;
+
+/*package*/ class WebappDSGSetup {
+
+    /**
+     * Given a path name and a preference of TDB1/TDB2 for new databases, return
+     * details of the setup to use.
+     */
+    /*package*/ static void setup(String directory, boolean useTDB2, FusekiArgs cmdLineConfig) {
+        if ( ! IO.exists(directory) )
+            throw new CmdException("Does not exist: " + directory);
+        if ( ! IO.isDirectory(directory) )
+            throw new CmdException("Not a directory: " + directory);
+
+        if ( IO.isEmptyDirectory(directory) ) {
+            if ( useTDB2 )
+                setupTDB2(directory, cmdLineConfig);
+            else
+                setupTDB1(directory, cmdLineConfig);
+            return;
+        }
+
+        // Exists, not empty or does not exist
+        if ( TDBOps.isTDB1(directory) ) {
+            setupTDB1(directory, cmdLineConfig);
+            return;
+        } else if ( TDBOps.isTDB2(directory) ) {
+            setupTDB2(directory, cmdLineConfig);
+            return;
+        } else
+            throw new CmdException("Directory not a database: " + directory);
+    }
+
+    private static void setupTDB1(String directory, FusekiArgs cmdLineConfig) {
+        cmdLineConfig.params.put(Template.DIR, directory);
+        cmdLineConfig.templateFile = Template.templateTDB1_DirFN;
+        cmdLineConfig.datasetDescription = "TDB1 dataset: location="+directory;
+    }
+
+    private static void setupTDB2(String directory, FusekiArgs cmdLineConfig) {
+        cmdLineConfig.params.put(Template.DIR, directory);
+        cmdLineConfig.templateFile = Template.templateTDB2_DirFN;
+        cmdLineConfig.datasetDescription = "TDB2 dataset: location="+directory;
+    }
+
+}
diff --git a/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/webapp/FusekiServerListener.java b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/webapp/FusekiServerListener.java
index c741361..a704f8b 100644
--- a/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/webapp/FusekiServerListener.java
+++ b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/webapp/FusekiServerListener.java
@@ -24,7 +24,7 @@ import javax.servlet.ServletContextListener;
 
 import org.apache.jena.fuseki.Fuseki;
 import org.apache.jena.fuseki.FusekiException;
-import org.apache.jena.fuseki.cmd.FusekiInitialConfig;
+import org.apache.jena.fuseki.cmd.FusekiArgs;
 import org.apache.jena.fuseki.metrics.MetricsProviderRegistry;
 import org.apache.jena.fuseki.server.DataAccessPointRegistry;
 import org.apache.jena.fuseki.server.FusekiInfo;
@@ -44,7 +44,7 @@ public class FusekiServerListener implements ServletContextListener {
 
     public FusekiServerListener() { }
 
-    public static FusekiInitialConfig initialSetup = null;
+    public static FusekiArgs initialSetup = null;
 
     private boolean initialized = false;
 
@@ -84,7 +84,7 @@ public class FusekiServerListener implements ServletContextListener {
             // The command line code sets initialSetup.
             // In a non-command line startup, initialSetup is null.
             if ( initialSetup == null ) {
-                initialSetup = new FusekiInitialConfig();
+                initialSetup = new FusekiArgs();
                 String cfg = FusekiEnv.FUSEKI_BASE.resolve(FusekiWebapp.DFT_CONFIG).toAbsolutePath().toString();
                 initialSetup.fusekiServerConfigFile = cfg;
             }
diff --git a/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/webapp/FusekiWebapp.java b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/webapp/FusekiWebapp.java
index ca5fc5e..fd963ed 100644
--- a/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/webapp/FusekiWebapp.java
+++ b/jena-fuseki2/jena-fuseki-webapp/src/main/java/org/apache/jena/fuseki/webapp/FusekiWebapp.java
@@ -35,14 +35,18 @@ import jena.cmd.CmdException;
 import org.apache.jena.atlas.io.IO;
 import org.apache.jena.atlas.lib.FileOps;
 import org.apache.jena.atlas.lib.InternalErrorException;
+import org.apache.jena.dboe.sys.Names;
 import org.apache.jena.fuseki.Fuseki;
 import org.apache.jena.fuseki.FusekiConfigException;
 import org.apache.jena.fuseki.build.DatasetDescriptionMap;
 import org.apache.jena.fuseki.build.FusekiConfig;
-import org.apache.jena.fuseki.cmd.FusekiInitialConfig;
+import org.apache.jena.fuseki.cmd.FusekiArgs;
 import org.apache.jena.fuseki.mgt.Template;
 import org.apache.jena.fuseki.mgt.TemplateFunctions;
-import org.apache.jena.fuseki.server.*;
+import org.apache.jena.fuseki.server.DataAccessPoint;
+import org.apache.jena.fuseki.server.DataAccessPointRegistry;
+import org.apache.jena.fuseki.server.DataService;
+import org.apache.jena.fuseki.server.FusekiVocab;
 import org.apache.jena.fuseki.servlets.HttpAction;
 import org.apache.jena.fuseki.servlets.ServletOps;
 import org.apache.jena.rdf.model.*;
@@ -51,7 +55,6 @@ import org.apache.jena.riot.RDFDataMgr;
 import org.apache.jena.riot.RDFLanguages;
 import org.apache.jena.sparql.core.DatasetGraph;
 import org.apache.jena.sparql.core.assembler.AssemblerUtils;
-import org.apache.jena.tdb.sys.Names;
 
 public class FusekiWebapp
 {
@@ -212,7 +215,7 @@ public class FusekiWebapp
         }
     }
 
-    public static void initializeDataAccessPoints(DataAccessPointRegistry registry, FusekiInitialConfig initialSetup, String configDir) {
+    public static void initializeDataAccessPoints(DataAccessPointRegistry registry, FusekiArgs initialSetup, String configDir) {
         List<DataAccessPoint> configFileDBs = initServerConfiguration(initialSetup);
         List<DataAccessPoint> directoryDBs =  FusekiConfig.readConfigurationDirectory(configDir);
         List<DataAccessPoint> systemDBs =     FusekiConfig.readSystemDatabase(SystemState.getDataset());
@@ -225,7 +228,7 @@ public class FusekiWebapp
         datapoints.forEach(registry::register);
     }
 
-    private static List<DataAccessPoint> initServerConfiguration(FusekiInitialConfig params) {
+    private static List<DataAccessPoint> initServerConfiguration(FusekiArgs params) {
         // Has a side effect of global context setting
         // when processing a config file.
         // Compatibility.
@@ -245,8 +248,8 @@ public class FusekiWebapp
         else if ( params.dsg != null ) {
             DataAccessPoint dap = datasetDefaultConfiguration(params.datasetPath, params.dsg, params.allowUpdate);
             datasets.add(dap);
-        } else if ( params.argTemplateFile != null ) {
-            DataAccessPoint dap = configFromTemplate(params.argTemplateFile, params.datasetPath, params.allowUpdate, params.params);
+        } else if ( params.templateFile != null ) {
+            DataAccessPoint dap = configFromTemplate(params.templateFile, params.datasetPath, params.allowUpdate, params.params);
             datasets.add(dap);
         }
         // No datasets is valid.
@@ -284,13 +287,8 @@ public class FusekiWebapp
         Fuseki.configLog.info("Template file: " + templateFile);
         String dir = params.get(Template.DIR);
         if ( dir != null ) {
-            if ( Objects.equals(dir, Names.memName) ) {
-                Fuseki.configLog.info("TDB dataset: in-memory");
-            } else {
-                if ( !FileOps.exists(dir) )
-                    throw new CmdException("Directory not found: " + dir);
-                Fuseki.configLog.info("TDB dataset: directory=" + dir);
-            }
+            if ( ! Objects.equals(dir, Names.memName) && !FileOps.exists(dir) )
+                throw new CmdException("Directory not found: " + dir);
         }
         //-- Logging
 
diff --git a/jena-fuseki2/jena-fuseki-webapp/src/test/java/org/apache/jena/fuseki/ServerCtl.java b/jena-fuseki2/jena-fuseki-webapp/src/test/java/org/apache/jena/fuseki/ServerCtl.java
index 985b54c..32f5b5b 100644
--- a/jena-fuseki2/jena-fuseki-webapp/src/test/java/org/apache/jena/fuseki/ServerCtl.java
+++ b/jena-fuseki2/jena-fuseki-webapp/src/test/java/org/apache/jena/fuseki/ServerCtl.java
@@ -30,7 +30,7 @@ import org.apache.http.client.HttpClient;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.jena.atlas.io.IO;
 import org.apache.jena.atlas.lib.FileOps;
-import org.apache.jena.fuseki.cmd.FusekiInitialConfig;
+import org.apache.jena.fuseki.cmd.FusekiArgs;
 import org.apache.jena.fuseki.cmd.JettyFusekiWebapp;
 import org.apache.jena.fuseki.jetty.JettyServerConfig;
 import org.apache.jena.fuseki.system.FusekiNetLib;
@@ -253,7 +253,7 @@ public class ServerCtl {
         SystemState.location = Location.mem();
         SystemState.init$();
 
-        FusekiInitialConfig params = new FusekiInitialConfig();
+        FusekiArgs params = new FusekiArgs();
         dsgTesting = DatasetGraphFactory.createTxnMem();
         params.dsg = dsgTesting;
         params.datasetPath = datasetPath;