You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by an...@apache.org on 2013/11/26 15:11:50 UTC

svn commit: r1545667 - in /jena/branches/jena-fuseki-new-ui: ./ src-dev/dev/ src/main/java/org/apache/jena/fuseki/mgt/ src/main/java/org/apache/jena/fuseki/server/ src/main/java/org/apache/jena/fuseki/servlets/

Author: andy
Date: Tue Nov 26 14:11:50 2013
New Revision: 1545667

URL: http://svn.apache.org/r1545667
Log:
Add dataset to running server : POST /$/datasets with a body of RDF assembler for the service+dataset

Added:
    jena/branches/jena-fuseki-new-ui/config2.ttl
    jena/branches/jena-fuseki-new-ui/post-ds   (with props)
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/DatasetsCollectionServlet.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/ActionLib.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberFilter.java
Removed:
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/ActionDataset.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/PageNames.java
Modified:
    jena/branches/jena-fuseki-new-ui/src-dev/dev/DevFuseki.java
    jena/branches/jena-fuseki-new-ui/src-dev/dev/RunFuseki.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/ManagementServer.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/MgtFunctions.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/DatasetRef.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/FusekiConfig.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/SPARQLServer.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_REST_R.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_ServletBase.java
    jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberServlet.java

Added: jena/branches/jena-fuseki-new-ui/config2.ttl
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/config2.ttl?rev=1545667&view=auto
==============================================================================
--- jena/branches/jena-fuseki-new-ui/config2.ttl (added)
+++ jena/branches/jena-fuseki-new-ui/config2.ttl Tue Nov 26 14:11:50 2013
@@ -0,0 +1,35 @@
+@prefix :        <#> .
+@prefix fuseki:  <http://jena.apache.org/fuseki#> .
+@prefix rdf:     <http://www.w3.org/1999/02/22-rdf-syntax-ns#> .
+
+@prefix rdfs:    <http://www.w3.org/2000/01/rdf-schema#> .
+@prefix tdb:     <http://jena.hpl.hp.com/2008/tdb#> .
+@prefix ja:      <http://jena.hpl.hp.com/2005/11/Assembler#> .
+
+## ## ## Always.
+## ## @prefix tdb:     <http://jena.hpl.hp.com/2008/tdb#> .
+## ## [] ja:loadClass "com.hp.hpl.jena.tdb.TDB" .
+## ## tdb:DatasetTDB  rdfs:subClassOf  ja:RDFDataset .
+## ## tdb:GraphTDB    rdfs:subClassOf  ja:Model .
+## ## ##
+
+<#service_tdb> rdf:type fuseki:Service ;
+    rdfs:label                      "TDB Service" ;
+    fuseki:name                     "NEW" ;
+    fuseki:serviceQuery             "query" ;
+    fuseki:serviceQuery             "sparql" ;
+    fuseki:serviceUpdate            "update" ;
+    fuseki:serviceUpload            "upload" ;
+    fuseki:serviceReadWriteGraphStore      "data" ;
+    # A separate read-only graph store endpoint:
+    fuseki:serviceReadGraphStore       "get" ;
+    fuseki:dataset           <#tdb_dataset> ;
+    .
+
+<#tdb_dataset> rdf:type      tdb:DatasetTDB ;
+    tdb:location "--mem--" ;
+##     # Query timeout on this dataset (milliseconds)
+    ja:context [ ja:cxtName "arq:queryTimeout" ;  ja:cxtValue "1000" ] ;
+##     # Default graph for query is the (read-only) union of all named graphs.
+    tdb:unionDefaultGraph true ;
+   .

Added: jena/branches/jena-fuseki-new-ui/post-ds
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/post-ds?rev=1545667&view=auto
==============================================================================
--- jena/branches/jena-fuseki-new-ui/post-ds (added)
+++ jena/branches/jena-fuseki-new-ui/post-ds Tue Nov 26 14:11:50 2013
@@ -0,0 +1,4 @@
+#!/bin/bash
+
+curl --data-binary @config2.ttl --header 'Content-type: text/turtle' \
+     http://localhost:3030/$/datasets
\ No newline at end of file

Propchange: jena/branches/jena-fuseki-new-ui/post-ds
------------------------------------------------------------------------------
    svn:executable = *

Modified: jena/branches/jena-fuseki-new-ui/src-dev/dev/DevFuseki.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src-dev/dev/DevFuseki.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src-dev/dev/DevFuseki.java (original)
+++ jena/branches/jena-fuseki-new-ui/src-dev/dev/DevFuseki.java Tue Nov 26 14:11:50 2013
@@ -18,42 +18,27 @@
 
 package dev;
 
+import java.util.ArrayList ;
+import java.util.List ;
+
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.fuseki.FusekiConfigException ;
+import org.apache.jena.fuseki.server.DatasetRef ;
+import org.apache.jena.fuseki.server.FusekiConfig ;
+import org.apache.jena.riot.RDFDataMgr ;
+import org.slf4j.Logger ;
+import org.slf4j.LoggerFactory ;
+
+import com.hp.hpl.jena.query.* ;
+import com.hp.hpl.jena.rdf.model.Model ;
+import com.hp.hpl.jena.rdf.model.RDFNode ;
+import com.hp.hpl.jena.rdf.model.Resource ;
+import com.hp.hpl.jena.update.UpdateAction ;
+import com.hp.hpl.jena.update.UpdateFactory ;
+import com.hp.hpl.jena.update.UpdateRequest ;
+
 public class DevFuseki
 {
-    // SPARQL_QueryDataset.vialidate query -- but FROM etc is important for TDB. 
-    // DatasetDescription from protocol
-    //   createDataset from the ARQ.
-    //   SPARQL_QueryGeneral.datasetFromDescription
-    
-    // Config:
-    //   fuseki:name ==> fuseki:serviceName of fuseki:endpointBase 
-    //   rdfs:label for log files.
-    
-    // sparql.jsp needs to switch on presence and name of service endpoints.
-    // --accept for to soh for construct queries (check can get CONSTRUCT in TTL).
-    
-    // application/json for application/sparql-results+json. 
-    // application/xml for application/sparql-results+xml. 
-    
-    // LimitingGraph, LimitingBulkUpdateHandler --> change to use a limiting Sink<>
-    // Finish: SPARQL_QueryGeneral
-    //    Parse errors and etc need to be passed out.
-    // --jetty-config documentation
-    
-    // Rework arguments.
-    //   Explicit install pages.
-    //   Pages for read-only.
-    
-    // RDF/XML_ABBREV in ResponseModel
-    // Config Jetty from a file?
-    //   Alternatibe way to run Fuseki
-    
-	// Flint?
-	// Pages for publish mode.
-
-	// Multiple Accept headers
-    // WebContent and ContentType clean up.
-    
 	// SOH default to not needing 'default'
 	// More error handling.
 
@@ -62,55 +47,6 @@ public class DevFuseki
     
     //soh : --accept application/turtle for CONSTRUCT queries.
     
-    // Direct naming.
-    // Use absence/presence of a query string to switch.
-    
-    // sparql.jsp ==> if no dataset, go to choosing page or error page linking to.
-    
-    // Better handling of bad URI name for a graph http:/example/X => stacktrace.
-    
-    // FUSEKI_ROOT
-
-    // Documentation:
-    //   Plan documentation.
-    //   MIME types supported
-    // Include a data file in the distribution.
-    // curl as commandline
-    
-    // Dataset servers - bulk loader.
-    // Access the bulk loader via web. [later]
-    
-    // Structure pages, different static content servers
-    // /Main - index.html = fuseki.html
-    // /validate
-    // /admin
-    
-    // Server-local graph naming
-    
-    // Plain text error pages
-    
-    // Deploy to sparql.org -- need query form for read-only mode.
-    
-    // + LARQ
-    
-    // Testing:
-    //   No file -> error.
-    
-    // Remove from package conneg - use code from ARQ/Atlas.
-    // TypedStream TypedInoputStrea, TypedOutputStream, MediaType, MediaRange
-    
-    // Bundle tdb scripts with Fuseki.
-
-    // ParserFor - share between SPARQL_REST and SPARQL_Upload
-    // UploadTo dataset (TriG, N-Quads)
-   
-    // populate forms with prefixes (later)
-    
-    // Tests
-    //   TestProtocol (HTTP update, query, update), inc status codes.
-    //   SPARQL Query servlet / SPARQL Update servlet
-    //   TestContentNegotiation - is coveage enough?
-    
     // ?? Slug header:
     // http://bitworking.org/projects/atom/rfc5023.html#rfc.section.9.7
     
@@ -118,32 +54,111 @@ public class DevFuseki
     
     // Authentication
     
-    // SOH
-    //   Refactor into body/no_body send & body/no_body receive
-    // All:
-    // -v --help --accept --user/--password ( or --auth user:pass) 
-    // Drop --service.
-    // Local config file - read to get service settings. 
-    
-    //   --accept line/shortname : s-get, s-query
-    //   Basic authentication: --user --password
-    
-    // Argument names: --service naming seems inconsistent.
-    
-    // Plug-ins:
-    //   Dataset (query, Update), HttpInternalIF?
-    //   "Connection"
-    // Locking => transaction support (via default model?)
-    //   HttpAction.beginRead() etc.
-    
-    // Java clients:
-    //   DatasetAccessor : don't serialise to byte[] and then send. 
-    //   DatasetAccessor : check existence of endpoint. 
-
     // Content-Length: SHOULD
     //   Transfer-Encoding: identity
     // "chunked" encoding
     // gzip
     
-    // Code examples
+    private static Logger log = LoggerFactory.getLogger("Devel") ;
+    
+    public static void main(String ... argv) {
+        Model m = read("config2.ttl") ;
+        // One rdf:type fuseki:Service
+        
+        ResultSet rs = query("SELECT * { ?service rdf:type fuseki:Service }", m) ; 
+        List<Resource> services = new ArrayList<Resource>() ;
+        for ( ; rs.hasNext() ; )
+        {
+            QuerySolution soln = rs.next() ;
+            Resource svc = soln.getResource("service") ;
+            services.add(svc) ;
+        }
+        
+        if ( services.size() == 0 ) {
+            log.error("No services found") ;
+            throw new FusekiConfigException() ;
+        }
+        if ( services.size() > 1 ) {
+            log.error("Multiple services found") ;
+            throw new FusekiConfigException() ;
+        }
+
+        Resource service = services.get(0) ;
+        DatasetRef sd = FusekiConfig.processService(service) ;
+        sd.init() ;
+        System.out.println("DONE");
+    }
+
+    private static Model read(String filename) {
+
+        Model m = RDFDataMgr.loadModel(filename) ;
+        String x1 = StrUtils.strjoinNL
+            ( "PREFIX tdb: <http://jena.hpl.hp.com/2008/tdb#>" ,
+              "PREFIX ja:  <http://jena.hpl.hp.com/2005/11/Assembler#>", 
+              "INSERT                    { [] ja:loadClass 'com.hp.hpl.jena.tdb.TDB' }",
+              "WHERE { FILTER NOT EXISTS { [] ja:loadClass 'com.hp.hpl.jena.tdb.TDB' } }"
+             ) ;
+        String x2 = StrUtils.strjoinNL
+            ("PREFIX tdb: <http://jena.hpl.hp.com/2008/tdb#>" ,
+             "PREFIX ja:  <http://jena.hpl.hp.com/2005/11/Assembler#>",
+             "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
+             "INSERT DATA {",
+             "   tdb:DatasetTDB  rdfs:subClassOf  ja:RDFDataset .",
+             "   tdb:GraphTDB    rdfs:subClassOf  ja:Model .",
+             "}" 
+             ) ;
+        execute(m, x1) ;
+        execute(m, x2) ;
+        return m ;
+        
+    }
+
+    private static void execute(Model m, String x) {
+        UpdateRequest req = UpdateFactory.create(x) ;
+        UpdateAction.execute(req, m);
+    }
+    
+    // Copies from original FusekiConfig
+    
+    private static String prefixes = StrUtils.strjoinNL(
+    "PREFIX fuseki: <http://jena.apache.org/fuseki#>" ,
+    "PREFIX rdf:    <http://www.w3.org/1999/02/22-rdf-syntax-ns#>",
+    "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+    "PREFIX tdb:    <http://jena.hpl.hp.com/2008/tdb#>",
+    "PREFIX list:   <http://jena.hpl.hp.com/ARQ/list#>",
+    "PREFIX list:   <http://jena.hpl.hp.com/ARQ/list#>",
+    "PREFIX xsd:     <http://www.w3.org/2001/XMLSchema#>",
+    "PREFIX apf:     <http://jena.hpl.hp.com/ARQ/property#>", 
+    "PREFIX afn:     <http://jena.hpl.hp.com/ARQ/function#>" ,
+    "") ;
+    
+    private static ResultSet query(String string, Model m)
+    {
+        return query(string, m, null, null) ;
+    }
+
+    private static ResultSet query(String string, Model m, String varName, RDFNode value)
+    {
+        Query query = QueryFactory.create(prefixes+string) ;
+        QuerySolutionMap initValues = null ;
+        if ( varName != null )
+            initValues = querySolution(varName, value) ;
+        QueryExecution qExec = QueryExecutionFactory.create(query, m, initValues) ;
+        ResultSet rs = ResultSetFactory.copyResults(qExec.execSelect()) ;
+        qExec.close() ;
+        return rs ;
+    }
+    
+    private static QuerySolutionMap querySolution(String varName, RDFNode value)
+    {
+        QuerySolutionMap qsm = new QuerySolutionMap() ;
+        querySolution(qsm, varName, value) ;
+        return qsm ;
+    }
+    
+    private static QuerySolutionMap querySolution(QuerySolutionMap qsm, String varName, RDFNode value)
+    {
+        qsm.add(varName, value) ;
+        return qsm ;
+    }
 }

Modified: jena/branches/jena-fuseki-new-ui/src-dev/dev/RunFuseki.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src-dev/dev/RunFuseki.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src-dev/dev/RunFuseki.java (original)
+++ jena/branches/jena-fuseki-new-ui/src-dev/dev/RunFuseki.java Tue Nov 26 14:11:50 2013
@@ -61,7 +61,7 @@ public class RunFuseki
         
         FusekiCmd.main(
                      //  "-v",
-                     "--update", "--mem", "--mgtPort=0", "/ds"
+                     "--uber", "--update", "--mem", "--mgtPort=0", "/ds"
                      //"--update", "--loc="+tmpdir+"DB", "--set=tdb:unionDefaultGraph=true", "/ds"
                      //"--update", "--mem", "/ds"
 

Added: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/DatasetsCollectionServlet.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/DatasetsCollectionServlet.java?rev=1545667&view=auto
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/DatasetsCollectionServlet.java (added)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/DatasetsCollectionServlet.java Tue Nov 26 14:11:50 2013
@@ -0,0 +1,309 @@
+/**
+ * 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.mgt;
+
+import java.io.IOException ;
+import java.io.InputStream ;
+import java.util.List ;
+
+import javax.servlet.ServletException ;
+import javax.servlet.http.HttpServletRequest ;
+import javax.servlet.http.HttpServletResponse ;
+
+import org.apache.jena.atlas.io.IO ;
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.atlas.web.ContentType ;
+import org.apache.jena.fuseki.FusekiConfigException ;
+import org.apache.jena.fuseki.FusekiLib ;
+import org.apache.jena.fuseki.server.DatasetRef ;
+import org.apache.jena.fuseki.server.DatasetRegistry ;
+import org.apache.jena.fuseki.server.FusekiConfig ;
+import org.apache.jena.fuseki.server.SPARQLServer ;
+import org.apache.jena.fuseki.servlets.ActionErrorException ;
+import org.apache.jena.fuseki.servlets.ServletBase ;
+import org.apache.jena.riot.* ;
+import org.apache.jena.riot.lang.LangRIOT ;
+import org.apache.jena.riot.system.ErrorHandler ;
+import org.apache.jena.riot.system.ErrorHandlerFactory ;
+import org.apache.jena.riot.system.StreamRDF ;
+import org.apache.jena.riot.system.StreamRDFLib ;
+
+import com.hp.hpl.jena.query.* ;
+import com.hp.hpl.jena.rdf.model.* ;
+import com.hp.hpl.jena.shared.PrefixMapping ;
+import com.hp.hpl.jena.update.UpdateAction ;
+import com.hp.hpl.jena.update.UpdateFactory ;
+import com.hp.hpl.jena.update.UpdateRequest ;
+import com.hp.hpl.jena.vocabulary.RDF ;
+import com.hp.hpl.jena.vocabulary.RDFS ;
+
+public class DatasetsCollectionServlet extends ServletBase {
+    
+    public DatasetsCollectionServlet() {}
+    
+    // Move doCommon from SPARQL_ServletBase to between ServiceBase and SPARQL_ServletBase??
+    
+    @Override
+    protected void doPost(HttpServletRequest request, HttpServletResponse response) {
+        // POST a new description.
+        try {
+            execPost(request, response) ;
+        } catch (ActionErrorException ex) {
+            if ( ex.exception != null )
+                ex.exception.printStackTrace(System.err) ;
+            // XXX Log message done by printResponse in a moment.
+            if ( ex.message != null )
+                responseSendError(response, ex.rc, ex.message) ;
+            else
+                responseSendError(response, ex.rc) ;
+        } 
+    }
+        
+    @Override
+    protected void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException { 
+        super.doGet(request, response); 
+        
+        
+        
+    }
+
+    protected void execPost(HttpServletRequest request, HttpServletResponse response) {
+        Model m = ModelFactory.createDefaultModel() ;
+        StreamRDF dest = StreamRDFLib.graph(m.getGraph()) ;
+        bodyAsGraph(request, dest) ;
+        Resource t = m.createResource("http://jena.apache.org/fuseki#Service") ;
+        List<Resource> services = getByType(t, m) ; 
+            
+//        ResultSet rs = query("SELECT * { ?service rdf:type fuseki:Service }", m) ; 
+//        List<Resource> services = new ArrayList<Resource>() ;
+//        for ( ; rs.hasNext() ; )
+//        {
+//            QuerySolution soln = rs.next() ;
+//            Resource svc = soln.getResource("service") ;
+//            services.add(svc) ;
+//        }
+        
+        if ( services.size() == 0 ) {
+            log.error("No services found") ;
+            throw new FusekiConfigException() ;
+        }
+        if ( services.size() > 1 ) {
+            log.error("Multiple services found") ;
+            throw new FusekiConfigException() ;
+        }
+
+        // Test name.
+        
+        Resource service = services.get(0) ;
+        String name = ((Literal)getOne(service, "fuseki:name")).getLexicalForm() ;
+        //log.info("name = "+name); 
+        DatasetRef dsDesc = FusekiConfig.processService(service) ;
+        String datasetPath = dsDesc.name ;
+        if ( DatasetRegistry.get().isRegistered(datasetPath) )
+            // Remove?
+            errorBadRequest("Already registered: "+name);
+        SPARQLServer.registerDataset(datasetPath, dsDesc) ;
+    }
+
+//    @Override
+//    protected void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException
+//    {
+//        
+//    }
+    
+    
+    // XXX Merge with SPARQL_REST_RW.incomingData
+    
+    protected static ErrorHandler errorHandler = ErrorHandlerFactory.errorHandlerStd(log) ;
+
+    private static void bodyAsGraph(HttpServletRequest request, StreamRDF dest) {
+        String base = wholeRequestURL(request) ;
+        ContentType ct = FusekiLib.getContentType(request) ;
+        Lang lang = WebContent.contentTypeToLang(ct.getContentType()) ;
+        if ( lang == null ) {
+            errorBadRequest("Unknown content type for triples: " + ct) ;
+            return ;
+        }
+        InputStream input = null ;
+        try { input = request.getInputStream() ; } 
+        catch (IOException ex) { IO.exception(ex) ; }
+
+        int len = request.getContentLength() ;
+//        if ( verbose ) {
+//            if ( len >= 0 )
+//                log.info(format("[%d]   Body: Content-Length=%d, Content-Type=%s, Charset=%s => %s", action.id, len,
+//                                ct.getContentType(), ct.getCharset(), lang.getName())) ;
+//            else
+//                log.info(format("[%d]   Body: Content-Type=%s, Charset=%s => %s", action.id, ct.getContentType(),
+//                                ct.getCharset(), lang.getName())) ;
+//        }
+
+        parse(dest, input, lang, base) ;
+    }
+
+    public static void parse(StreamRDF dest, InputStream input, Lang lang, String base) {
+        // Need to adjust the error handler.
+//        try { RDFDataMgr.parse(dest, input, base, lang) ; }
+//        catch (RiotException ex) { errorBadRequest("Parse error: "+ex.getMessage()) ; }
+        LangRIOT parser = RiotReader.createParser(input, lang, base, dest) ;
+        parser.getProfile().setHandler(errorHandler) ;
+        try { parser.parse() ; } 
+        catch (RiotException ex) { errorBadRequest("Parse error: "+ex.getMessage()) ; }
+    }
+    
+    private static Model read(String filename) {
+
+        Model m = RDFDataMgr.loadModel(filename) ;
+        String x1 = StrUtils.strjoinNL
+            ( "PREFIX tdb: <http://jena.hpl.hp.com/2008/tdb#>" ,
+              "PREFIX ja:  <http://jena.hpl.hp.com/2005/11/Assembler#>", 
+              "INSERT                    { [] ja:loadClass 'com.hp.hpl.jena.tdb.TDB' }",
+              "WHERE { FILTER NOT EXISTS { [] ja:loadClass 'com.hp.hpl.jena.tdb.TDB' } }"
+             ) ;
+        String x2 = StrUtils.strjoinNL
+            ("PREFIX tdb: <http://jena.hpl.hp.com/2008/tdb#>" ,
+             "PREFIX ja:  <http://jena.hpl.hp.com/2005/11/Assembler#>",
+             "PREFIX rdfs:    <http://www.w3.org/2000/01/rdf-schema#>",
+             "INSERT DATA {",
+             "   tdb:DatasetTDB  rdfs:subClassOf  ja:RDFDataset .",
+             "   tdb:GraphTDB    rdfs:subClassOf  ja:Model .",
+             "}" 
+             ) ;
+        execute(m, x1) ;
+        execute(m, x2) ;
+        return m ;
+        
+    }
+
+    private static void execute(Model m, String x) {
+        UpdateRequest req = UpdateFactory.create(x) ;
+        UpdateAction.execute(req, m);
+    }
+    
+    // Temprary -- XXX Copies from original FusekiConfig
+    
+    private static String prefixes = StrUtils.strjoinNL(
+    "PREFIX fuseki: <http://jena.apache.org/fuseki#>" ,
+    "PREFIX rdf:    <http://www.w3.org/1999/02/22-rdf-syntax-ns#>",
+    "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+    "PREFIX tdb:    <http://jena.hpl.hp.com/2008/tdb#>",
+    "PREFIX list:   <http://jena.hpl.hp.com/ARQ/list#>",
+    "PREFIX list:   <http://jena.hpl.hp.com/ARQ/list#>",
+    "PREFIX xsd:     <http://www.w3.org/2001/XMLSchema#>",
+    "PREFIX apf:     <http://jena.hpl.hp.com/ARQ/property#>", 
+    "PREFIX afn:     <http://jena.hpl.hp.com/ARQ/function#>" ,
+    "") ;
+    
+    private static ResultSet query(String string, Model m)
+    {
+        return query(string, m, null, null) ;
+    }
+
+    private static ResultSet query(String string, Model m, String varName, RDFNode value)
+    {
+        Query query = QueryFactory.create(prefixes+string) ;
+        QuerySolutionMap initValues = null ;
+        if ( varName != null )
+            initValues = querySolution(varName, value) ;
+        QueryExecution qExec = QueryExecutionFactory.create(query, m, initValues) ;
+        ResultSet rs = ResultSetFactory.copyResults(qExec.execSelect()) ;
+        qExec.close() ;
+        return rs ;
+    }
+    
+    private static QuerySolutionMap querySolution(String varName, RDFNode value)
+    {
+        QuerySolutionMap qsm = new QuerySolutionMap() ;
+        querySolution(qsm, varName, value) ;
+        return qsm ;
+    }
+    
+    private static QuerySolutionMap querySolution(QuerySolutionMap qsm, String varName, RDFNode value)
+    {
+        qsm.add(varName, value) ;
+        return qsm ;
+    }
+    
+    private static RDFNode getOne(Resource svc, String property)
+    {
+        String ln = property.substring(property.indexOf(':')+1) ;
+        ResultSet rs = query("SELECT * { ?svc "+property+" ?x}", svc.getModel(), "svc", svc) ;
+        if ( ! rs.hasNext() )
+            throw new FusekiConfigException("No "+ln+" for service "+nodeLabel(svc)) ;
+        RDFNode x = rs.next().get("x") ;
+        if ( rs.hasNext() )
+            throw new FusekiConfigException("Multiple "+ln+" for service "+nodeLabel(svc)) ;
+        return x ;
+    }
+    
+    private static List<Resource> getByType(Resource type, Model m)
+    {
+        ResIterator rIter = m.listSubjectsWithProperty(RDF.type, type) ;
+        return Iter.toList(rIter) ;
+    }
+    
+    
+    // Node presentation
+    private static String nodeLabel(RDFNode n)
+    {
+        if ( n == null )
+            return "<null>" ;
+        if ( n instanceof Resource )
+            return strForResource((Resource)n) ;
+        
+        Literal lit = (Literal)n ;
+        return lit.getLexicalForm() ;
+    }
+    private static String strForResource(Resource r) { return strForResource(r, r.getModel()) ; }
+    
+    private static String strForResource(Resource r, PrefixMapping pm)
+    {
+        if ( r == null )
+            return "NULL ";
+        if ( r.hasProperty(RDFS.label))
+        {
+            RDFNode n = r.getProperty(RDFS.label).getObject() ;
+            if ( n instanceof Literal )
+                return ((Literal)n).getString() ;
+        }
+        
+        if ( r.isAnon() )
+            return "<<blank node>>" ;
+
+        if ( pm == null )
+            pm = r.getModel() ;
+
+        return strForURI(r.getURI(), pm ) ;
+    }
+    
+    private static String strForURI(String uri, PrefixMapping pm)
+    {
+        if ( pm != null )
+        {
+            String x = pm.shortForm(uri) ;
+            
+            if ( ! x.equals(uri) )
+                return x ;
+        }
+        return "<"+uri+">" ;
+    }
+
+}
+

Modified: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/ManagementServer.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/ManagementServer.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/ManagementServer.java (original)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/ManagementServer.java Tue Nov 26 14:11:50 2013
@@ -78,6 +78,7 @@ public class ManagementServer
             throw new FusekiException("Base URI does nto start with a '/'") ; 
         addServlet(context, new MgtCmdServlet(),    base+"mgt") ;
         addServlet(context, new StatsServlet(),     base+"stats") ;
+        addServlet(context, new DatasetsCollectionServlet(),     base+"datasets") ;
     }
 
     // SHARE

Modified: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/MgtFunctions.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/MgtFunctions.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/MgtFunctions.java (original)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/mgt/MgtFunctions.java Tue Nov 26 14:11:50 2013
@@ -74,12 +74,6 @@ public class MgtFunctions
         return Iter.toList(DatasetRegistry.get().keys()) ;
     }
 
-    /** Return name of */  
-    public static String actionDataset(HttpServletRequest request)
-    {
-        return PageNames.actionDatasetNames ;
-    }
-
     // Service name getters ...
     
     /** Return a SPARQL query service name for the dataset */

Modified: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/DatasetRef.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/DatasetRef.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/DatasetRef.java (original)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/DatasetRef.java Tue Nov 26 14:11:50 2013
@@ -238,4 +238,15 @@ public class DatasetRef implements Datas
         cs.add(CounterName.GSPoptionsGood) ;
         cs.add(CounterName.GSPoptionsBad) ;
     }
+    
+    public static String canocialDatasetPath(String datasetPath) {
+        if ( datasetPath.equals("/") )
+            datasetPath = "" ;
+        else
+            if ( !datasetPath.startsWith("/") )
+                datasetPath = "/" + datasetPath ;
+        if ( datasetPath.endsWith("/") )
+            datasetPath = datasetPath.substring(0, datasetPath.length() - 1) ;
+        return datasetPath ;
+    }
 }

Modified: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/FusekiConfig.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/FusekiConfig.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/FusekiConfig.java (original)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/FusekiConfig.java Tue Nov 26 14:11:50 2013
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.jena.fuseki.server;
+package org.apache.jena.fuseki.server ;
 
 import java.lang.reflect.Method ;
 import java.util.ArrayList ;
@@ -59,49 +59,46 @@ import com.hp.hpl.jena.util.FileManager 
 import com.hp.hpl.jena.vocabulary.RDF ;
 import com.hp.hpl.jena.vocabulary.RDFS ;
 
-public class FusekiConfig
-{
-    static { Fuseki.init(); }
+public class FusekiConfig {
+    static {
+        Fuseki.init() ;
+    }
 
     // The datastructure that captures a servers configuration.
-    
+
     // Server port
-    int port ;
+    int                   port ;
     // Management command port - -1 for none.
-    int mgtPort ;           
-    List<DatasetRef> datasets = null ;
-    
-    
-    private static Logger log = Fuseki.configLog ;
-    
-    private static String prefixes = StrUtils.strjoinNL(
-    "PREFIX fu:     <http://jena.apache.org/fuseki#>" ,
-    "PREFIX rdf:    <http://www.w3.org/1999/02/22-rdf-syntax-ns#>",
-    "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
-    "PREFIX tdb:    <http://jena.hpl.hp.com/2008/tdb#>",
-    "PREFIX list:   <http://jena.hpl.hp.com/ARQ/list#>",
-    "PREFIX list:   <http://jena.hpl.hp.com/ARQ/list#>",
-    "PREFIX xsd:     <http://www.w3.org/2001/XMLSchema#>",
-    "PREFIX apf:     <http://jena.hpl.hp.com/ARQ/property#>", 
-    "PREFIX afn:     <http://jena.hpl.hp.com/ARQ/function#>" ,
-    "") ;
-    
-    public static ServerConfig defaultConfiguration(String datasetPath, DatasetGraph dsg, boolean allowUpdate, boolean listenLocal)
-    {
+    int                   mgtPort ;
+    List<DatasetRef>      datasets = null ;
+
+    private static Logger log      = Fuseki.configLog ;
+
+    private static String prefixes = StrUtils.strjoinNL
+        ("PREFIX fu:     <http://jena.apache.org/fuseki#>",
+         "PREFIX rdf:    <http://www.w3.org/1999/02/22-rdf-syntax-ns#>",
+         "PREFIX rdfs:   <http://www.w3.org/2000/01/rdf-schema#>",
+         "PREFIX tdb:    <http://jena.hpl.hp.com/2008/tdb#>",
+         "PREFIX list:   <http://jena.hpl.hp.com/ARQ/list#>",
+         "PREFIX list:   <http://jena.hpl.hp.com/ARQ/list#>",
+         "PREFIX xsd:     <http://www.w3.org/2001/XMLSchema#>",
+         "PREFIX apf:     <http://jena.hpl.hp.com/ARQ/property#>",
+         "PREFIX afn:     <http://jena.hpl.hp.com/ARQ/function#>", "") ;
+
+    public static ServerConfig defaultConfiguration(String datasetPath, DatasetGraph dsg, boolean allowUpdate,
+                                                    boolean listenLocal) {
         DatasetRef dbDesc = new DatasetRef() ;
-        dbDesc.name = datasetPath ;
+        dbDesc.name = DatasetRef.canocialDatasetPath(datasetPath) ;
         dbDesc.dataset = dsg ;
         dbDesc.query.endpoints.add(HttpNames.ServiceQuery) ;
         dbDesc.query.endpoints.add(HttpNames.ServiceQueryAlt) ;
 
-        if ( allowUpdate )
-        {
+        if ( allowUpdate ) {
             dbDesc.update.endpoints.add(HttpNames.ServiceUpdate) ;
             dbDesc.upload.endpoints.add(HttpNames.ServiceUpload) ;
             dbDesc.readWriteGraphStore.endpoints.add(HttpNames.ServiceData) ;
             dbDesc.allowDatasetUpdate = true ;
-        }
-        else
+        } else
             dbDesc.readGraphStore.endpoints.add(HttpNames.ServiceData) ;
         ServerConfig config = new ServerConfig() ;
         config.datasets = Arrays.asList(dbDesc) ;
@@ -115,12 +112,12 @@ public class FusekiConfig
         config.verboseLogging = false ;
         return config ;
     }
-    
-    public static ServerConfig configure(String filename)
-    {
+
+    public static ServerConfig configure(String filename) {
         // Be absolutely sure everything has initialized.
-        // Some initialization registers assemblers and sets abbreviation vocabulary. 
-        ARQ.init();
+        // Some initialization registers assemblers and sets abbreviation
+        // vocabulary.
+        ARQ.init() ;
         TDB.init() ;
         Fuseki.init() ;
         Model m = FileManager.get().loadModel(filename) ;
@@ -128,29 +125,30 @@ public class FusekiConfig
         // Find one server.
         List<Resource> servers = getByType(FusekiVocab.tServer, m) ;
         if ( servers.size() == 0 )
-            throw new FusekiConfigException("No server found (no resource with type "+strForResource(FusekiVocab.tServer)) ;
+            throw new FusekiConfigException("No server found (no resource with type "
+                                            + strForResource(FusekiVocab.tServer)) ;
         if ( servers.size() > 1 )
-            throw new FusekiConfigException(servers.size()+" servers found (must be exactly one in a configuration file)") ;
-        
-        // ---- Server 
+            throw new FusekiConfigException(servers.size()
+                                            + " servers found (must be exactly one in a configuration file)") ;
+
+        // ---- Server
         Resource server = servers.get(0) ;
         processServer(server) ;
 
         // ---- Services
-        ResultSet rs = query("SELECT * { ?s fu:services [ list:member ?member ] }", m) ; 
-        if ( ! rs.hasNext() )
+        ResultSet rs = query("SELECT * { ?s fu:services [ list:member ?member ] }", m) ;
+        if ( !rs.hasNext() )
             log.warn("No services found") ;
-        
-        List<DatasetRef> services =  new ArrayList<DatasetRef>() ; 
-        
-        for ( ; rs.hasNext() ; )
-        {
+
+        List<DatasetRef> services = new ArrayList<DatasetRef>() ;
+
+        for ( ; rs.hasNext() ; ) {
             QuerySolution soln = rs.next() ;
             Resource svc = soln.getResource("member") ;
             DatasetRef sd = processService(svc) ;
             services.add(sd) ;
         }
-        
+
         // TODO Properties for the other fields.
         ServerConfig config = new ServerConfig() ;
         config.datasets = services ;
@@ -163,146 +161,141 @@ public class FusekiConfig
         config.verboseLogging = false ;
         return config ;
     }
-    
 
-    // DatasetRef used where there isn't a real Dataset e.g. the SPARQL processor.  
-    
-    private static DatasetRef noDataset      = new DatasetRef() ;
-    private static DatasetGraph dummyDSG        = new DatasetGraphReadOnly(DatasetGraphFactory.createMemFixed()) ;
+    // DatasetRef used where there isn't a real Dataset e.g. the SPARQL
+    // processor.
+
+    private static DatasetRef   noDataset = new DatasetRef() ;
+    private static DatasetGraph dummyDSG  = new DatasetGraphReadOnly(DatasetGraphFactory.createMemFixed()) ;
     static {
         noDataset.name = "" ;
         noDataset.dataset = dummyDSG ;
         noDataset.query.endpoints.add(HttpNames.ServiceQuery) ;
         noDataset.query.endpoints.add(HttpNames.ServiceQueryAlt) ;
         noDataset.allowDatasetUpdate = false ;
-        noDataset.init();
-        // Don't register it.  
-        // This is used as a placeholder and shoudl not be found by "all datasets"  
+        noDataset.init() ;
+        // Don't register it.
+        // This is used as a placeholder and shoudl not be found by
+        // "all datasets"
         // DatasetRegistry.get().put("", noDataset) ;
     }
 
-    /** Return the DatasetRef (read-only) for when there is no dataset, just a SPARQL Query processor */ 
-    public static DatasetRef serviceOnlyDatasetRef() { return noDataset ; }
+    /**
+     * Return the DatasetRef (read-only) for when there is no dataset, just a
+     * SPARQL Query processor
+     */
+    public static DatasetRef serviceOnlyDatasetRef() {
+        return noDataset ;
+    }
 
-    private static void processServer(Resource server)
-    {
+    private static void processServer(Resource server) {
         // Global, currently.
         AssemblerUtils.setContext(server, Fuseki.getContext()) ;
-        
+
         StmtIterator sIter = server.listProperties(JA.loadClass) ;
-        for( ; sIter.hasNext(); )
-        {
+        for ( ; sIter.hasNext() ; ) {
             Statement s = sIter.nextStatement() ;
             RDFNode rn = s.getObject() ;
             String className = null ;
-            if ( rn instanceof Resource )
-            {
+            if ( rn instanceof Resource ) {
                 String uri = ((Resource)rn).getURI() ;
-                if ( uri == null )
-                {
+                if ( uri == null ) {
                     log.warn("Blank node for class to load") ;
                     continue ;
                 }
                 String javaScheme = "java:" ;
-                if ( ! uri.startsWith(javaScheme) )
-                {
-                    log.warn("Class to load is not 'java:': "+uri) ;
+                if ( !uri.startsWith(javaScheme) ) {
+                    log.warn("Class to load is not 'java:': " + uri) ;
                     continue ;
                 }
                 className = uri.substring(javaScheme.length()) ;
             }
             if ( rn instanceof Literal )
-                className = ((Literal)rn).getLexicalForm() ; 
-            /*Loader.*/loadAndInit(className) ;
+                className = ((Literal)rn).getLexicalForm() ;
+            /* Loader. */loadAndInit(className) ;
         }
         // ----
     }
 
-    private static void loadAndInit(String className)
-    {
+    private static void loadAndInit(String className) {
         try {
-            Class<?> classObj = Class.forName(className);
-            log.info("Loaded "+className) ;
-            Method initMethod = classObj.getMethod("init");
-            initMethod.invoke(null);
-        } catch (ClassNotFoundException ex)
-        {
-            log.warn("Class not found: "+className);
-        } 
-        catch (Exception e)         { throw new FusekiConfigException(e) ; }
+            Class<? > classObj = Class.forName(className) ;
+            log.info("Loaded " + className) ;
+            Method initMethod = classObj.getMethod("init") ;
+            initMethod.invoke(null) ;
+        }
+        catch (ClassNotFoundException ex) {
+            log.warn("Class not found: " + className) ;
+        }
+        catch (Exception e) {
+            throw new FusekiConfigException(e) ;
+        }
     }
 
-    private static DatasetRef processService(Resource svc)
-    {
-        log.info("Service: "+nodeLabel(svc)) ;
+    public static DatasetRef processService(Resource svc) {
+        log.info("Service: " + nodeLabel(svc)) ;
         DatasetRef sDesc = new DatasetRef() ;
         sDesc.name = ((Literal)getOne(svc, "fu:name")).getLexicalForm() ;
-        log.info("  name = "+sDesc.name) ;
+        sDesc.name = DatasetRef.canocialDatasetPath(sDesc.name) ;
+        log.info("  name = " + sDesc.name) ;
 
-        addServiceEP("query", sDesc.name, sDesc.query, svc, "fu:serviceQuery") ; 
-        addServiceEP("update", sDesc.name, sDesc.update, svc, "fu:serviceUpdate") ; 
+        addServiceEP("query", sDesc.name, sDesc.query, svc, "fu:serviceQuery") ;
+        addServiceEP("update", sDesc.name, sDesc.update, svc, "fu:serviceUpdate") ;
         addServiceEP("upload", sDesc.name, sDesc.upload, svc, "fu:serviceUpload") ;
         addServiceEP("graphStore(RW)", sDesc.name, sDesc.readWriteGraphStore, svc, "fu:serviceReadWriteGraphStore") ;
         addServiceEP("graphStore(R)", sDesc.name, sDesc.readGraphStore, svc, "fu:serviceReadGraphStore") ;
         // Extract timeout overriding configuration if present.
-        if (svc.hasProperty(FusekiVocab.pAllowTimeoutOverride)) {
-            sDesc.allowTimeoutOverride = svc.getProperty(FusekiVocab.pAllowTimeoutOverride).getObject().asLiteral().getBoolean();
-            if (svc.hasProperty(FusekiVocab.pMaximumTimeoutOverride)) {
-                sDesc.maximumTimeoutOverride = (int) (svc.getProperty(FusekiVocab.pMaximumTimeoutOverride).getObject().asLiteral().getFloat() * 1000);
+        if ( svc.hasProperty(FusekiVocab.pAllowTimeoutOverride) ) {
+            sDesc.allowTimeoutOverride = svc.getProperty(FusekiVocab.pAllowTimeoutOverride).getObject().asLiteral().getBoolean() ;
+            if ( svc.hasProperty(FusekiVocab.pMaximumTimeoutOverride) ) {
+                sDesc.maximumTimeoutOverride = (int)(svc.getProperty(FusekiVocab.pMaximumTimeoutOverride).getObject().asLiteral().getFloat() * 1000) ;
             }
         }
-        
+
         Resource datasetDesc = ((Resource)getOne(svc, "fu:dataset")) ;
 
         // Check if it is in the model.
-        if ( ! datasetDesc.hasProperty(RDF.type) )
-            throw new FusekiConfigException("No rdf:type for dataset "+nodeLabel(datasetDesc)) ;
-        
-        Dataset ds = (Dataset)Assembler.general.open(datasetDesc)  ;
-        sDesc.dataset = ds.asDatasetGraph() ; 
+        if ( !datasetDesc.hasProperty(RDF.type) )
+            throw new FusekiConfigException("No rdf:type for dataset " + nodeLabel(datasetDesc)) ;
+
+        Dataset ds = (Dataset)Assembler.general.open(datasetDesc) ;
+        sDesc.dataset = ds.asDatasetGraph() ;
         return sDesc ;
-        
+
     }
-    
-    private static RDFNode getOne(Resource svc, String property)
-    {
-        String ln = property.substring(property.indexOf(':')+1) ;
-        ResultSet rs = query("SELECT * { ?svc "+property+" ?x}", svc.getModel(), "svc", svc) ;
-        if ( ! rs.hasNext() )
-            throw new FusekiConfigException("No "+ln+" for service "+nodeLabel(svc)) ;
+
+    private static RDFNode getOne(Resource svc, String property) {
+        String ln = property.substring(property.indexOf(':') + 1) ;
+        ResultSet rs = query("SELECT * { ?svc " + property + " ?x}", svc.getModel(), "svc", svc) ;
+        if ( !rs.hasNext() )
+            throw new FusekiConfigException("No " + ln + " for service " + nodeLabel(svc)) ;
         RDFNode x = rs.next().get("x") ;
         if ( rs.hasNext() )
-            throw new FusekiConfigException("Multiple "+ln+" for service "+nodeLabel(svc)) ;
+            throw new FusekiConfigException("Multiple " + ln + " for service " + nodeLabel(svc)) ;
         return x ;
     }
-    
-    private static List<Resource> getByType(Resource type, Model m)
-    {
+
+    private static List<Resource> getByType(Resource type, Model m) {
         ResIterator rIter = m.listSubjectsWithProperty(RDF.type, type) ;
         return Iter.toList(rIter) ;
     }
 
-    private static void addServiceEP(String label, String name, ServiceRef service, Resource svc, String property)
-    {
-        ResultSet rs = query("SELECT * { ?svc "+property+" ?ep}", svc.getModel(), "svc", svc) ;
-        for ( ; rs.hasNext() ; )
-        {
+    private static void addServiceEP(String label, String name, ServiceRef service, Resource svc, String property) {
+        ResultSet rs = query("SELECT * { ?svc " + property + " ?ep}", svc.getModel(), "svc", svc) ;
+        for ( ; rs.hasNext() ; ) {
             QuerySolution soln = rs.next() ;
             String epName = soln.getLiteral("ep").getLexicalForm() ;
             service.endpoints.add(epName) ;
-            log.info("  "+label+" = /"+name+"/"+epName) ;
+            log.info("  " + label + " = /" + name + "/" + epName) ;
         }
     }
 
-
-    private static ResultSet query(String string, Model m)
-    {
+    private static ResultSet query(String string, Model m) {
         return query(string, m, null, null) ;
     }
 
-    private static ResultSet query(String string, Model m, String varName, RDFNode value)
-    {
-        Query query = QueryFactory.create(prefixes+string) ;
+    private static ResultSet query(String string, Model m, String varName, RDFNode value) {
+        Query query = QueryFactory.create(prefixes + string) ;
         QuerySolutionMap initValues = null ;
         if ( varName != null )
             initValues = querySolution(varName, value) ;
@@ -311,64 +304,58 @@ public class FusekiConfig
         qExec.close() ;
         return rs ;
     }
-    
-    private static QuerySolutionMap querySolution(String varName, RDFNode value)
-    {
+
+    private static QuerySolutionMap querySolution(String varName, RDFNode value) {
         QuerySolutionMap qsm = new QuerySolutionMap() ;
         querySolution(qsm, varName, value) ;
         return qsm ;
     }
-    
-    private static QuerySolutionMap querySolution(QuerySolutionMap qsm, String varName, RDFNode value)
-    {
+
+    private static QuerySolutionMap querySolution(QuerySolutionMap qsm, String varName, RDFNode value) {
         qsm.add(varName, value) ;
         return qsm ;
     }
-    
+
     // Node presentation
-    private static String nodeLabel(RDFNode n)
-    {
+    private static String nodeLabel(RDFNode n) {
         if ( n == null )
             return "<null>" ;
         if ( n instanceof Resource )
             return strForResource((Resource)n) ;
-        
+
         Literal lit = (Literal)n ;
         return lit.getLexicalForm() ;
     }
-    
-    private static String strForResource(Resource r) { return strForResource(r, r.getModel()) ; }
-    
-    private static String strForResource(Resource r, PrefixMapping pm)
-    {
+
+    private static String strForResource(Resource r) {
+        return strForResource(r, r.getModel()) ;
+    }
+
+    private static String strForResource(Resource r, PrefixMapping pm) {
         if ( r == null )
-            return "NULL ";
-        if ( r.hasProperty(RDFS.label))
-        {
+            return "NULL " ;
+        if ( r.hasProperty(RDFS.label) ) {
             RDFNode n = r.getProperty(RDFS.label).getObject() ;
             if ( n instanceof Literal )
                 return ((Literal)n).getString() ;
         }
-        
+
         if ( r.isAnon() )
             return "<<blank node>>" ;
 
         if ( pm == null )
             pm = r.getModel() ;
 
-        return strForURI(r.getURI(), pm ) ;
+        return strForURI(r.getURI(), pm) ;
     }
-    
-    private static String strForURI(String uri, PrefixMapping pm)
-    {
-        if ( pm != null )
-        {
+
+    private static String strForURI(String uri, PrefixMapping pm) {
+        if ( pm != null ) {
             String x = pm.shortForm(uri) ;
-            
-            if ( ! x.equals(uri) )
+
+            if ( !x.equals(uri) )
                 return x ;
         }
-        return "<"+uri+">" ;
+        return "<" + uri + ">" ;
     }
 }
-

Modified: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/SPARQLServer.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/SPARQLServer.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/SPARQLServer.java (original)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/server/SPARQLServer.java Tue Nov 26 14:11:50 2013
@@ -28,11 +28,10 @@ import javax.servlet.DispatcherType ;
 import javax.servlet.http.HttpServlet ;
 
 import org.apache.jena.fuseki.Fuseki ;
+import org.apache.jena.fuseki.FusekiConfigException ;
 import org.apache.jena.fuseki.FusekiException ;
 import org.apache.jena.fuseki.HttpNames ;
-import org.apache.jena.fuseki.mgt.ActionDataset ;
 import org.apache.jena.fuseki.mgt.MgtFunctions ;
-import org.apache.jena.fuseki.mgt.PageNames ;
 import org.apache.jena.fuseki.servlets.* ;
 import org.apache.jena.fuseki.validation.DataValidator ;
 import org.apache.jena.fuseki.validation.IRIValidator ;
@@ -46,6 +45,7 @@ import org.eclipse.jetty.server.Connecto
 import org.eclipse.jetty.server.Server ;
 import org.eclipse.jetty.server.nio.BlockingChannelConnector ;
 import org.eclipse.jetty.servlet.DefaultServlet ;
+import org.eclipse.jetty.servlet.FilterHolder ;
 import org.eclipse.jetty.servlet.ServletContextHandler ;
 import org.eclipse.jetty.servlet.ServletHolder ;
 import org.eclipse.jetty.servlets.GzipFilter ;
@@ -96,9 +96,16 @@ public class SPARQLServer {
 
     private void configureDatasets(ServletContextHandler context) {
         // Build them all.
+        
         for (DatasetRef dsDesc : serverConfig.datasets)
             configureOneDataset(context, dsDesc, serverConfig.enableCompression) ;
         
+        if ( überServlet ) {
+            FilterHolder f = new FilterHolder(new SPARQL_UberFilter()) ;
+            //FilterMapping
+            EnumSet<DispatcherType> es = EnumSet.allOf(DispatcherType.class) ; 
+            context.addFilter(f, "/*", es);
+        }
     }
     
     /**
@@ -150,13 +157,15 @@ public class SPARQLServer {
         return server ;
     }
     
-    public int getPort() {
+    public int getPort() {        
+
         return server.getConnectors()[0].getPort() ;
     }
 
     /**
      * Get the datasets associated with the server.
-     * @return returns the datasets via {@link org.apache.jena.fuseki.server.ServerConfig#datasets}
+     * @return returns the datasets via {@link org.apache.jena.fuseki.server.ServerConfig#datasets        
+}
      */
     public List<DatasetRef> getDatasets() {
         return serverConfig.datasets ;
@@ -233,14 +242,13 @@ public class SPARQLServer {
 
         serverLog.debug("Pages = " + serverConfig.pages) ;
 
-        boolean installManager = true ;
         boolean installServices = true ;
 
         String validationRoot = "/validate" ;
 
         // Should all services be /_/.... or some such?
 
-        if ( installManager || installServices ) {
+        if ( installServices ) {
             // TODO Respect port.
             if ( serverConfig.pagesPort != serverConfig.port )
                 serverLog.warn("Not supported yet - pages on a different port to services") ;
@@ -252,12 +260,6 @@ public class SPARQLServer {
             addServlet(context, templateEngine, "*.tpl", false) ;
         }
 
-        if ( installManager ) {
-            // Action when control panel selects a dataset.
-            HttpServlet datasetChooser = new ActionDataset() ;
-            addServlet(context, datasetChooser, PageNames.actionDatasetNames, false) ;
-        }
-
         if ( installServices ) {
             // Validators
             HttpServlet validateQuery = new QueryValidator() ;
@@ -278,7 +280,7 @@ public class SPARQLServer {
             addServlet(context, generalQueryService, HttpNames.ServiceGeneralQuery, enableCompression) ;
         }
 
-        if ( installManager || installServices ) {
+        if ( installServices ) {
             String[] files = {"fuseki.html", "index.html"} ;
             context.setWelcomeFiles(files) ;
             addContent(context, "/", serverConfig.pages) ;
@@ -295,31 +297,17 @@ public class SPARQLServer {
     private static List<String> ListOfEmptyString = Arrays.asList("") ;
 
     private void configureOneDataset(ServletContextHandler context, DatasetRef dsDesc, boolean enableCompression) {
-        String datasetPath = dsDesc.name ;
-        if ( datasetPath.equals("/") )
-            datasetPath = "" ;
-        else
-            if ( !datasetPath.startsWith("/") )
-                datasetPath = "/" + datasetPath ;
-
-        if ( datasetPath.endsWith("/") )
-            datasetPath = datasetPath.substring(0, datasetPath.length() - 1) ;
-
-        dsDesc.init() ;
-
-        DatasetRegistry.get().put(datasetPath, dsDesc) ;
-        serverLog.info(format("Dataset path = %s", datasetPath)) ;
-
-        HttpServlet sparqlQuery = new SPARQL_QueryDataset() ;
-        HttpServlet sparqlUpdate = new SPARQL_Update() ;
-        HttpServlet sparqlUpload = new SPARQL_Upload() ;
-        HttpServlet sparqlHttpR = new SPARQL_REST_R() ;
-        HttpServlet sparqlHttpRW = new SPARQL_REST_RW() ;
-        HttpServlet sparqlDataset = new SPARQL_UberServlet.AccessByConfig() ;
-
+        
+        String datasetPath = DatasetRef.canocialDatasetPath(dsDesc.name) ;
+        registerDataset(datasetPath, dsDesc) ;
+        
         if ( !überServlet ) {
-            // If uberserver, these are unnecessary but can be used.
-            // If just means the überservlet isn't handling these operations.
+            HttpServlet sparqlQuery = new SPARQL_QueryDataset() ;
+            HttpServlet sparqlUpdate = new SPARQL_Update() ;
+            HttpServlet sparqlUpload = new SPARQL_Upload() ;
+            HttpServlet sparqlHttpR = new SPARQL_REST_R() ;
+            HttpServlet sparqlHttpRW = new SPARQL_REST_RW() ;
+
             addServlet(context, datasetPath, sparqlQuery, dsDesc.query, enableCompression) ;
             addServlet(context, datasetPath, sparqlUpdate, dsDesc.update, false) ;
             addServlet(context, datasetPath, sparqlUpload, dsDesc.upload, false) ; // No point - no results of any size.
@@ -330,21 +318,33 @@ public class SPARQLServer {
             // ListOfEmptyString, enableCompression) ;
         } else {
             // This is the servlet that analyses requests and dispatches them to
-            // the appropriate servlet.
+            // the appropriate servlet. 
             // SPARQL Query, SPARQL Update -- handles dataset?query=
             // dataset?update=
             // Graph Store Protocol (direct and indirect naming) if enabled.
             // GET/PUT/POST on the dataset itself.
-            // It also checks for a request that looks like a service request
-            // and passes it
+            // It also checks for a request that looks like a service request and passes it
             // on to the service (this takes precedence over direct naming).
-            addServlet(context, datasetPath, sparqlDataset, epDataset, enableCompression) ;
+
+            if ( false ) {
+                //XXX Filter version superceeds this.
+                HttpServlet sparqlDataset = new SPARQL_UberServlet.AccessByConfig() ;
+                addServlet(context, datasetPath, sparqlDataset, epDataset, enableCompression) ;
+            }
         }
 
         // Add JMX beans to record daatset and it's services.
         addJMX(dsDesc) ;
     }
 
+    public static void registerDataset(String datasetPath, DatasetRef dsDesc) {
+        dsDesc.init() ;
+        if ( DatasetRegistry.get().isRegistered(datasetPath) )
+            throw new FusekiConfigException("Already registered: key = "+datasetPath) ;
+        DatasetRegistry.get().put(datasetPath, dsDesc) ;
+        serverLog.info(format("Dataset path = %s", datasetPath)) ;
+    }
+    
     private static Server configServer(String jettyConfig) {
         try {
             serverLog.info("Jetty server config file = " + jettyConfig) ;

Added: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/ActionLib.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/ActionLib.java?rev=1545667&view=auto
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/ActionLib.java (added)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/ActionLib.java Tue Nov 26 14:11:50 2013
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.fuseki.servlets;
+
+import org.apache.jena.fuseki.server.DatasetRef ;
+import org.apache.jena.fuseki.server.DatasetRegistry ;
+
+/** Operations related to servlets */
+
+public class ActionLib {
+
+    /** Map request to uri in the registry.
+     * A possible implementation for mapRequestToDataset(String)
+     *  that assumes the form /dataset/service 
+     *  Returning null means no mapping found. 
+     */
+    
+    public static String mapRequestToDataset(String uri)
+    {
+        // Chop off trailing part - the service selector
+        // e.g. /dataset/sparql => /dataset 
+        int i = uri.lastIndexOf('/') ;
+        if ( i == -1 )
+            return null ;
+        if ( i == 0 )
+        {
+            // started with '/' - leave.
+            return uri ;
+        }
+        
+        return uri.substring(0, i) ;
+    }
+
+    public static String mapRequestToService(DatasetRef dsRef, String uri, String datasetURI)
+    {
+        if ( dsRef == null )
+            return "" ;
+        if ( dsRef.name.length() >= uri.length() )
+            return "" ;
+        return uri.substring(dsRef.name.length()+1) ;   // Skip the separating "/"
+        
+    }
+    
+    /** Implementation of mapRequestToDataset(String) that looks for
+     * the longest match in the registry.
+     * This includes use in direct naming GSP. 
+     */
+    public static String mapRequestToDatasetLongest$(String uri) 
+    {
+        if ( uri == null )
+            return null ;
+        
+        // This covers local, using the URI as a direct name for
+        // a graph, not just using the indirect ?graph= or ?default 
+        // forms.
+
+        String ds = null ;
+        for ( String ds2 : DatasetRegistry.get().keys() ) {
+            if ( ! uri.startsWith(ds2) )
+                continue ;
+
+            if ( ds == null )
+            {
+                ds = ds2 ;
+                continue ; 
+            }
+            if ( ds.length() < ds2.length() )
+            {
+                ds = ds2 ;
+                continue ;
+            }
+        }
+        return ds ;
+    }
+
+}
+

Modified: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_REST_R.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_REST_R.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_REST_R.java (original)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_REST_R.java Tue Nov 26 14:11:50 2013
@@ -39,7 +39,7 @@ public class SPARQL_REST_R extends SPARQ
     
     
     @Override
-    protected String mapRequestToDataset(String uri) { return mapRequestToDatasetLongest$(uri) ; } 
+    protected String mapRequestToDataset(String uri) { return ActionLib.mapRequestToDatasetLongest$(uri) ; } 
 
     @Override
     protected void doGet(HttpAction action)

Modified: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_ServletBase.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_ServletBase.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_ServletBase.java (original)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_ServletBase.java Tue Nov 26 14:11:50 2013
@@ -43,7 +43,6 @@ import com.hp.hpl.jena.sparql.util.Conte
 
 public abstract class SPARQL_ServletBase extends ServletBase
 {
-    
     protected SPARQL_ServletBase()      {   super() ; }
     
     // Common framework for handling HTTP requests
@@ -134,7 +133,7 @@ public abstract class SPARQL_ServletBase
             dsRef = FusekiConfig.serviceOnlyDatasetRef() ;
 
         action.setDataset(dsRef) ;
-        String serviceName = mapRequestToService(dsRef, uri, datasetUri) ;
+        String serviceName = ActionLib.mapRequestToService(dsRef, uri, datasetUri) ;
         ServiceRef srvRef = dsRef.getServiceRef(serviceName) ;
         action.setService(srvRef) ;
         executeAction(action) ;
@@ -185,6 +184,14 @@ public abstract class SPARQL_ServletBase
         }
     }
     
+    /** Map request to uri in the registry.
+     *  null means no mapping done (passthrough). 
+     */
+    protected String mapRequestToDataset(String uri) 
+    {
+        return ActionLib.mapRequestToDataset(uri) ;
+    }
+    
     protected static void incCounter(Counters counters, CounterName name) {
         try {
             if ( counters.getCounters().contains(name) )
@@ -270,74 +277,4 @@ public abstract class SPARQL_ServletBase
             return String.format("%,d ms", time) ;
         return String.format("%,.3f s", time/1000.0) ;
     }
-
-    /** Map request to uri in the registry.
-     *  null means no mapping done (passthrough). 
-     */
-    protected String mapRequestToDataset(String uri) 
-    {
-        return mapRequestToDataset$(uri) ;
-    }
-    
-    /** A possible implementation for mapRequestToDataset(String)
-     *  that assums the form /dataset/service 
-     */
-    
-    protected static String mapRequestToDataset$(String uri)
-    {
-        // Chop off trailing part - the service selector
-        // e.g. /dataset/sparql => /dataset 
-        int i = uri.lastIndexOf('/') ;
-        if ( i == -1 )
-            return null ;
-        if ( i == 0 )
-        {
-            // started with '/' - leave.
-            return uri ;
-        }
-        
-        return uri.substring(0, i) ;
-    }
-
-    protected String mapRequestToService(DatasetRef dsRef, String uri, String datasetURI)
-    {
-        if ( dsRef == null )
-            return "" ;
-        if ( dsRef.name.length() >= uri.length() )
-            return "" ;
-        return uri.substring(dsRef.name.length()+1) ;   // Skip the separating "/"
-        
-    }
-    
-    /** Implementation of mapRequestToDataset(String) that looks for
-     * the longest match in the registry.
-     * This includes use in direct naming GSP. 
-     */
-    protected static String mapRequestToDatasetLongest$(String uri) 
-    {
-        if ( uri == null )
-            return null ;
-        
-        // This covers local, using the URI as a direct name for
-        // a graph, not just using the indirect ?graph= or ?default 
-        // forms.
-
-        String ds = null ;
-        for ( String ds2 : DatasetRegistry.get().keys() ) {
-            if ( ! uri.startsWith(ds2) )
-                continue ;
-
-            if ( ds == null )
-            {
-                ds = ds2 ;
-                continue ; 
-            }
-            if ( ds.length() < ds2.length() )
-            {
-                ds = ds2 ;
-                continue ;
-            }
-        }
-        return ds ;
-    }
 }

Added: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberFilter.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberFilter.java?rev=1545667&view=auto
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberFilter.java (added)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberFilter.java Tue Nov 26 14:11:50 2013
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.fuseki.servlets;
+
+import java.io.IOException ;
+
+import javax.servlet.* ;
+import javax.servlet.http.HttpServletRequest ;
+import javax.servlet.http.HttpServletResponse ;
+
+import org.apache.jena.fuseki.Fuseki ;
+import org.apache.jena.fuseki.server.DatasetRegistry ;
+import org.slf4j.Logger ;
+
+/** Look at all requests and see if they match a registered dataset name; 
+ * if they do, pass down to the uber servlet, which can dispatch any request
+ * for any service. 
+ *  
+ */
+public class SPARQL_UberFilter implements Filter {
+    private static Logger log = Fuseki.requestLog ; //LoggerFactory.getLogger(SomeFilter.class) ;
+    private static SPARQL_UberServlet überServlet = new SPARQL_UberServlet.AccessByConfig() ;
+    
+    @Override
+    public void init(FilterConfig filterConfig) throws ServletException {}
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
+        try {
+            // See SPARQL_Servlet.execCommonWorker
+            HttpServletRequest req = (HttpServletRequest)request ;
+            HttpServletResponse resp = (HttpServletResponse)response ;
+
+            String uri = req.getRequestURI() ;
+            String datasetUri = ActionLib.mapRequestToDataset(uri) ;
+
+            if ( datasetUri != null ) {        
+                if ( DatasetRegistry.get().isRegistered(datasetUri) ) {
+                    // Intercept and redirect
+                    log.info("Redirect: "+uri);
+                    überServlet.doCommon(req, resp) ;
+                    return ;
+                }
+            }
+        } catch (Exception ex) {}
+        
+        // Not found - continue. 
+        chain.doFilter(request, response);
+    }
+
+    @Override
+    public void destroy() {}
+
+}
+

Modified: jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberServlet.java
URL: http://svn.apache.org/viewvc/jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberServlet.java?rev=1545667&r1=1545666&r2=1545667&view=diff
==============================================================================
--- jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberServlet.java (original)
+++ jena/branches/jena-fuseki-new-ui/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_UberServlet.java Tue Nov 26 14:11:50 2013
@@ -122,7 +122,7 @@ public abstract class SPARQL_UberServlet
     @Override
     protected String mapRequestToDataset(String uri) 
     {
-        return mapRequestToDatasetLongest$(uri) ;
+        return ActionLib.mapRequestToDatasetLongest$(uri) ;
     }