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

[2/6] jena git commit: JENA-1623: Endpoint access control lists

JENA-1623: Endpoint access control lists


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

Branch: refs/heads/master
Commit: ca7839362f3ce720e6e457e00d84b2c3a55168b3
Parents: ba6b878
Author: Andy Seaborne <an...@apache.org>
Authored: Mon Nov 19 23:58:51 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Nov 20 20:22:51 2018 +0000

----------------------------------------------------------------------
 .../jena/riot/system/StreamRDFWriter.java       |  12 +-
 .../jena/fuseki/access/DataAccessCtl.java       |   2 +-
 .../org/apache/jena/fuseki/access/Users.java    |  35 -----
 .../java/org/apache/jena/fuseki/auth/Auth.java  | 102 ++++++++++++
 .../org/apache/jena/fuseki/auth/AuthPolicy.java |  39 +++++
 .../apache/jena/fuseki/auth/AuthPolicyList.java |  73 +++++++++
 .../jena/fuseki/auth/RequestAuthorization.java  |  60 +++++++
 .../java/org/apache/jena/fuseki/auth/Users.java |  33 ++++
 .../jena/fuseki/build/FusekiBuildLib.java       |   1 +
 .../apache/jena/fuseki/build/FusekiBuilder.java |  43 +++--
 .../apache/jena/fuseki/build/FusekiConfig.java  |   5 +-
 .../jena/fuseki/build/RequestAuthorization.java | 115 --------------
 .../org/apache/jena/fuseki/ctl/ActionCtl.java   |   2 +-
 .../org/apache/jena/fuseki/ctl/ActionStats.java |   2 +-
 .../apache/jena/fuseki/ctl/JsonDescription.java |   2 +-
 .../jena/fuseki/jetty/FusekiErrorHandler.java   |  12 +-
 .../fuseki/server/DataAccessPointRegistry.java  |   2 +-
 .../apache/jena/fuseki/server/DataService.java  |  16 +-
 .../org/apache/jena/fuseki/server/Endpoint.java |  73 +++++----
 .../apache/jena/fuseki/server/FusekiInfo.java   |   2 +-
 .../jena/fuseki/servlets/ActionService.java     |  85 ++++++++--
 .../apache/jena/fuseki/servlets/AuthFilter.java |   2 +-
 .../jena/fuseki/servlets/FusekiFilter.java      |   2 +-
 .../apache/jena/fuseki/servlets/HttpAction.java |  20 ++-
 .../jena/fuseki/servlets/ServiceRouter.java     | 157 ++++++++-----------
 .../apache/jena/fuseki/servlets/ServletOps.java |   9 +-
 .../test/java/org/apache/jena/fuseki/Dummy.java |   4 +
 .../apache/jena/fuseki/main/FusekiServer.java   |  37 +++--
 .../jena/fuseki/main/cmds/FusekiMain.java       |   2 +-
 .../jena/fuseki/main/TestFusekiTestServer.java  |   3 +-
 .../fuseki/main/access/TS_SecurityFuseki.java   |   2 +-
 .../jena/fuseki/main/access/TestAuthorized.java | 111 +++++++++++++
 .../main/access/TestAuthorizedRequest.java      | 110 -------------
 .../main/access/TestPasswordServices.java       |   7 +-
 jena-fuseki2/jena-fuseki-webapp/fuseki-dev      |   2 +-
 35 files changed, 715 insertions(+), 469 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-arq/src/main/java/org/apache/jena/riot/system/StreamRDFWriter.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/system/StreamRDFWriter.java b/jena-arq/src/main/java/org/apache/jena/riot/system/StreamRDFWriter.java
index e01237c..7fcd875 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/system/StreamRDFWriter.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/system/StreamRDFWriter.java
@@ -40,7 +40,9 @@ import org.apache.jena.sparql.core.Quad ;
 
 /** Write RDF in a streaming fashion.
  *  {@link RDFDataMgr} operations do not provide this guarantee.
- *  See {@link  RDFWriterRegistry} for general purpose writers.
+ *  See {@link RDFWriterRegistry} for general purpose writers. 
+ *  {@link StreamRDFWriter} returns the same writer as {@link RDFWriterRegistry}
+ *  if the {@link RDFFormat} is a streaming format. 
  *  
  * @see RDFDataMgr 
  * @see RDFWriterRegistry 
@@ -152,11 +154,11 @@ public class StreamRDFWriter {
         register(RDFFormat.RDFNULL,         streamWriterFactoryNull) ;
     }
 
-    /** Get a StreamRDF destination that will output in syntax <tt>Lang</tt>
+    /** Get a StreamRDF destination that will output in syntax {@code Lang}
      *  and is guaranteed to do so in a scaling, streaming fashion.    
      * @param output OutputStream
      * @param lang   The syntax 
-     * @return       StreamRDF
+     * @return       StreamRDF, or null if Lang does not have a streaming format.
      * @see StreamOps#graphToStream
      * @see StreamOps#datasetToStream
      */
@@ -165,11 +167,11 @@ public class StreamRDFWriter {
         return getWriterStream(output, fmt) ;
     }
 
-    /** Get a StreamRDF destination that will output in syntax <tt>RDFFormat</tt>
+    /** Get a StreamRDF destination that will output in syntax {@code RDFFormat}
      *  and is guaranteed to do so in a scaling, streaming fashion.    
      * @param output OutputStream
      * @param format  The syntax (as an {@link RDFFormat}) 
-     * @return       StreamRDF
+     * @return       StreamRDF, or null if format is not registered for streaming.
      * @see StreamOps#graphToStream
      * @see StreamOps#datasetToStream
      */

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-access/src/main/java/org/apache/jena/fuseki/access/DataAccessCtl.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-access/src/main/java/org/apache/jena/fuseki/access/DataAccessCtl.java b/jena-fuseki2/jena-fuseki-access/src/main/java/org/apache/jena/fuseki/access/DataAccessCtl.java
index b97aeec..d49e2ab 100644
--- a/jena-fuseki2/jena-fuseki-access/src/main/java/org/apache/jena/fuseki/access/DataAccessCtl.java
+++ b/jena-fuseki2/jena-fuseki-access/src/main/java/org/apache/jena/fuseki/access/DataAccessCtl.java
@@ -54,7 +54,7 @@ public class DataAccessCtl {
     public static final Symbol   symAuthorizationService    = Symbol.create(VocabSecurity.getURI() + "authService");
 
     /** Get the user from the servlet context via {@link HttpServletRequest#getRemoteUser} */ 
-    public static final Function<HttpAction, String> requestUserServlet = (action)->action.request.getRemoteUser();
+    public static final Function<HttpAction, String> requestUserServlet = (action)->action.getUser();
 
     /**
      * Get the user from {@code ?user} query string parameter. Use carefully; for situations where the user name has

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-access/src/main/java/org/apache/jena/fuseki/access/Users.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-access/src/main/java/org/apache/jena/fuseki/access/Users.java b/jena-fuseki2/jena-fuseki-access/src/main/java/org/apache/jena/fuseki/access/Users.java
deleted file mode 100644
index 62e2bab..0000000
--- a/jena-fuseki2/jena-fuseki-access/src/main/java/org/apache/jena/fuseki/access/Users.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.fuseki.access;
-
-public class Users {
-    
-    
-    /**
-     * Reserved user role name: Name of the user role for any authenticated user of the system.
-     * In the servlet API, this equates to {@code getRemoteUser() != null}.
-     */
-    public static String UserAnyLoggedIn = "*" ; 
-
-    /** 
-     * Reserved user role name: Name of the user role for any authenticated user of the system
-     * In the servlet API, this includes {@code getRemoteUser() != null}.
-     */
-    public static String UserAny = "_" ; 
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/Auth.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/Auth.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/Auth.java
new file mode 100644
index 0000000..2e61129
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/Auth.java
@@ -0,0 +1,102 @@
+/*
+ * 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.auth;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Objects;
+
+import org.apache.jena.fuseki.Fuseki;
+import org.apache.jena.fuseki.FusekiConfigException;
+
+/** Authorization Policies.
+ * See {@link Users} for special user names.
+ */
+public class Auth {
+    /** Any authenticated user. */
+    public static AuthPolicy ANY_USER = (user) -> user != null;
+
+    /** Any user, whether authenticated or not. */
+    public static AuthPolicy ANY_ANON = (user) -> true;
+
+    /** Never allow. */
+    public static AuthPolicy DENY     = (user) -> false;
+
+    /** A policy that allows specific users (convenience wrapped for {@link #policyAllowSpecific(Collection)}). */ 
+    public static AuthPolicy policyAllowSpecific(String... allowedUsers) {
+        return Auth.policyAllowSpecific(Arrays.asList(allowedUsers));
+    }
+
+    /** 
+     * A policy that allows specific users.
+     * <ul>
+     * <li>If any user is {@linkplain Users#UserAnyLoggedIn}, then this policy is the same as {@linkplain #ANY_USER}.
+     * <li>If any user is {@linkplain Users#UserAnyAnon}, then this policy is the same as {@linkplain #ANY_ANON}.
+     * </ul>
+     */ 
+    public static AuthPolicy policyAllowSpecific(Collection<String> allowedUsers) {
+        Objects.requireNonNull(allowedUsers, "allowedUsers");
+        if ( allowedUsers.contains(Users.UserAnyLoggedIn) ) {
+            if ( allowedUsers.size() > 1 )
+                Fuseki.configLog.warn("Both 'any user' and a list of users given");
+            return ANY_USER;
+        }
+        if ( allowedUsers.contains(Users.UserAnyAnon) ) {
+            if ( allowedUsers.size() > 1 )
+                Fuseki.configLog.warn("Both 'anon user' and a list of users given");
+            return ANY_ANON;
+        }
+
+        if ( allowedUsers.stream().anyMatch(Objects::isNull) )
+            throw new FusekiConfigException("null user found : "+allowedUsers);  
+        return new RequestAuthorization(allowedUsers);
+    }
+
+    /**
+     * Test whether a user (principal) is allowed by a authorization policy.  
+     * The policy can be null, meaning no restrictions, and the function returns true.
+     * {@code user} maybe null, meaning unauthenticated and any policy must deal with this. 
+     * @param user
+     * @param policy
+     * @return boolean True if the policy is null or allows the user.
+     */
+    public static boolean allow(String user, AuthPolicy policy) {
+        if ( policy == null )
+            return true;
+        return policy.isAllowed(user);
+    }
+    
+    /**
+     * Test whether a user (principal) is allowed by a authorization policy
+     * and perform an action if the policy does not allow the user.
+     * The action can throw an exception.
+     * Additional, return true/false - see {@link #allow(String, AuthPolicy)}.
+     * The policy can be null, meaning no restrictions, and the function returns true.
+     * {@code user} maybe null, meaning unauthenticated and any policy must deal with this. 
+     * @param user
+     * @param policy
+     * @param notAllowed Runnable to execute if the policy does not allow the user.
+     */
+    public static boolean allow(String user, AuthPolicy policy, Runnable notAllowed) {
+        if ( allow(user, policy) )
+            return true;
+        notAllowed.run();
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/AuthPolicy.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/AuthPolicy.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/AuthPolicy.java
new file mode 100644
index 0000000..ef6d330
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/AuthPolicy.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.fuseki.auth;
+
+/**
+ * Policy for authorization to a resource.
+ * Assumes the user has already been authenticated.
+ */
+public interface AuthPolicy {
+    /** 
+     * Is the use authorized for this resource?
+     */
+    public boolean isAllowed(String user);
+
+    /**
+     * Is the use denied for this resource? Both {@linkplain #isAllowed} and
+     * {@linkplain #isDenied} could be false if the policy does not knwo one way of the
+     * other.
+     */
+    public default boolean isDenied(String user) {
+        return !isAllowed(user);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/AuthPolicyList.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/AuthPolicyList.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/AuthPolicyList.java
new file mode 100644
index 0000000..1e00320
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/AuthPolicyList.java
@@ -0,0 +1,73 @@
+/*
+ * 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.auth;
+
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/** 
+ * An {@link AuthPolicy} that combines a number of {@link AuthPolicy AuthPolicies}.
+ * All policies must authorize access for this policy to allow access.
+ */ 
+public class AuthPolicyList implements AuthPolicy {
+
+    // Thread safe.
+    // Use a 
+    private final Queue<AuthPolicy> policies = new ConcurrentLinkedQueue<>();
+    
+    /**
+     * Merge {@link AuthPolicy AuthPolicies}, returning a combination of the two if both are non-null.
+     * If either is null, return the other.
+     * If both null, return null.
+     */
+    public static AuthPolicy merge(AuthPolicy policy1, AuthPolicy policy2) {
+        if ( policy1 == null )
+            return policy2 ;
+        if ( policy2 == null )
+            return policy1;
+        if ( policy1 instanceof AuthPolicyList) {
+            AuthPolicyList x = new AuthPolicyList((AuthPolicyList)policy1);
+            x.add(policy2);
+            return x;
+        }
+        AuthPolicyList x = new AuthPolicyList();
+        x.add(policy1);
+        x.add(policy2);
+        return x;
+    }
+    
+    private AuthPolicyList(AuthPolicyList other) { 
+        policies.addAll(other.policies);
+    }
+    
+    public AuthPolicyList() { }
+    
+    public void add(AuthPolicy policy) {
+        policies.add(policy);
+    }
+
+    @Override
+    public boolean isAllowed(String user) {
+        for ( AuthPolicy policy : policies ) {
+            if ( ! policy.isAllowed(user) )
+                return false;
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/RequestAuthorization.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/RequestAuthorization.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/RequestAuthorization.java
new file mode 100644
index 0000000..2a5d11a
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/RequestAuthorization.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.fuseki.auth;
+
+import java.util.*;
+
+/**
+ * Policy for allowing users to execute a request. 
+ * Assumes the user has been authenticated.
+ */
+class RequestAuthorization implements AuthPolicy {
+
+    private final Set<String>  allowedUsers;
+
+    /*package*/ RequestAuthorization(Collection<String> allowed) {
+        this.allowedUsers = (allowed == null) ? Collections.emptySet() : new HashSet<>(allowed);
+    }
+    
+    @Override
+    public boolean isAllowed(String user) {
+        if ( user == null )
+            return false;
+        if ( contains(allowedUsers, user) )
+            return true;
+        return false;
+    }
+
+    @Override
+    public boolean isDenied(String user) {
+        return !isAllowed(user);
+    }
+
+    static <T> boolean isNullOrEmpty(Collection<T> collection) {
+        if ( collection == null )
+            return true;
+        return collection.isEmpty(); 
+    }
+    
+    static <T> boolean contains(Collection<T> collection, T obj) {
+        if ( collection == null )
+            return false;
+        return collection.contains(obj); 
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/Users.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/Users.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/Users.java
new file mode 100644
index 0000000..7b6c5fd
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/auth/Users.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.fuseki.auth;
+
+public class Users {
+    /**
+     * Reserved user role name: Name of the user role for any authenticated user of the system.
+     * In the servlet API, this equates to {@code getRemoteUser() != null}.
+     */
+    public static String UserAnyLoggedIn = "*"; 
+
+    /** 
+     * Reserved user role name: Name of the user role for any authenticated user of the system
+     * In the servlet API, this includes {@code getRemoteUser() == null}
+     */
+    public static String UserAnyAnon = "_" ; 
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuildLib.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuildLib.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuildLib.java
index 5bad1ce..81a0563 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuildLib.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuildLib.java
@@ -118,6 +118,7 @@ public class FusekiBuildLib {
                 RDFList list = n.as(RDFList.class);
                 results.addAll(list.asJavaList());
             } catch (JenaException x) {
+                // Not a list.
                 results.add(n);
             }
         });

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuilder.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuilder.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuilder.java
index a6fdd8a..bfcb3a5 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuilder.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiBuilder.java
@@ -26,13 +26,13 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.jena.fuseki.FusekiConfigException;
-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.Operation ;
+import org.apache.jena.fuseki.auth.Auth;
+import org.apache.jena.fuseki.auth.AuthPolicy;
+import org.apache.jena.fuseki.server.*;
 import org.apache.jena.graph.Node;
 import org.apache.jena.query.QuerySolution ;
 import org.apache.jena.query.ResultSet ;
+import org.apache.jena.rdf.model.Literal;
 import org.apache.jena.rdf.model.Property ;
 import org.apache.jena.rdf.model.RDFNode;
 import org.apache.jena.rdf.model.Resource ;
@@ -45,7 +45,7 @@ import org.apache.jena.sparql.core.DatasetGraph ;
  */
 public class FusekiBuilder
 {
-    /** Build a DataService starting at Resource svc, with the standard (default) set of services */
+    /** Build a DataService starting at Resource svc, with the standard (default) set of services. */
     public static DataService buildDataServiceStd(DatasetGraph dsg, boolean allowUpdate) {
         DataService dataService = new DataService(dsg) ;
         populateStdServices(dataService, allowUpdate);
@@ -70,7 +70,12 @@ public class FusekiBuilder
 
     /** Add an operation to a {@link DataService} with a given endpoint name */
     public static void addServiceEP(DataService dataService, Operation operation, String endpointName) {
-        dataService.addEndpoint(operation, endpointName) ; 
+        dataService.addEndpoint(operation, endpointName) ;
+    }
+
+    /** Add an operation to a {@link DataService} with a given endpoint name */
+    public static void addServiceEP(DataService dataService, Operation operation, String endpointName, AuthPolicy requestAuth) {
+        dataService.addEndpoint(operation, endpointName, requestAuth) ;
     }
 
     public static void addServiceEP(DataService dataService, Operation operation, Resource svc, Property property) {
@@ -78,8 +83,24 @@ public class FusekiBuilder
         ResultSet rs = FusekiBuildLib.query("SELECT * { ?svc " + p + " ?ep}", svc.getModel(), "svc", svc) ;
         for ( ; rs.hasNext() ; ) {
             QuerySolution soln = rs.next() ;
-            String epName = soln.getLiteral("ep").getLexicalForm() ;
-            addServiceEP(dataService, operation, epName); 
+            AuthPolicy requestAuth = null;
+            RDFNode ep = soln.get("ep");
+            String epName = null;
+            if ( ep.isLiteral() )
+                epName = soln.getLiteral("ep").getLexicalForm() ;
+            else if ( ep.isResource() ) {
+                Resource r = (Resource)ep;
+                try {
+                    // [AuthAll]
+                    // [ fuseki:name "" ; fuseki:allowedUsers ( "" "" ) ]
+                    Resource x = r.getProperty(FusekiVocab.pAllowedUsers).getResource();
+                    requestAuth = FusekiBuilder.allowedUsers(x);
+                    epName = ((Literal)r.getProperty(FusekiVocab.pServiceName)).getLexicalForm();
+                } catch(Exception x) {}                
+            } else {
+                throw new FusekiConfigException("Unrecognized: "+ep);
+            }
+            addServiceEP(dataService, operation, epName, requestAuth); 
             //log.info("  " + operation.name + " = " + dataAccessPoint.getName() + "/" + epName) ;
         }
     }
@@ -112,13 +133,13 @@ public class FusekiBuilder
         dataAccessPoints.remove(name);
     }
 
-    /** Get the allowed users on some resources.
+    /** Get the allowed users on a resource.
      *  Returns null if the resource is null or if there were no settings. 
      *  
      * @param resource
      * @return RequestAuthorization
      */
-    public static RequestAuthorization allowedUsers(Resource resource) {
+    public static AuthPolicy allowedUsers(Resource resource) {
         if ( resource == null )
             return null;
         Collection<RDFNode> allowedUsers = FusekiBuildLib.getAll(resource, "fu:"+pAllowedUsers.getLocalName());
@@ -140,7 +161,7 @@ public class FusekiBuilder
             .map(RDFNode::asNode)
             .map(Node::getLiteralLexicalForm)
             .collect(toList());
-        return RequestAuthorization.policyAllowSpecific(userNames);
+        return Auth.policyAllowSpecific(userNames);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiConfig.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiConfig.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiConfig.java
index 5dc34e3..c42207b 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiConfig.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/FusekiConfig.java
@@ -39,6 +39,7 @@ import org.apache.jena.atlas.lib.StrUtils ;
 import org.apache.jena.datatypes.xsd.XSDDatatype;
 import org.apache.jena.fuseki.Fuseki ;
 import org.apache.jena.fuseki.FusekiConfigException ;
+import org.apache.jena.fuseki.auth.AuthPolicy;
 import org.apache.jena.fuseki.server.*;
 import org.apache.jena.query.Dataset ;
 import org.apache.jena.query.QuerySolution ;
@@ -286,8 +287,8 @@ public class FusekiConfig {
         String name = object.getLexicalForm() ;
         name = DataAccessPoint.canonical(name) ;
         DataService dataService = buildDataService(svc, dsDescMap) ;
-        RequestAuthorization allowedUsers = FusekiBuilder.allowedUsers(svc);
-        dataService.setAllowedUsers(allowedUsers);
+        AuthPolicy allowedUsers = FusekiBuilder.allowedUsers(svc);
+        dataService.setAuthPolicy(allowedUsers);
         DataAccessPoint dataAccess = new DataAccessPoint(name, dataService) ;
         return dataAccess ;
     }

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/RequestAuthorization.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/RequestAuthorization.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/RequestAuthorization.java
deleted file mode 100644
index 16a1616..0000000
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/build/RequestAuthorization.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.fuseki.build;
-
-import java.util.*;
-
-import org.apache.jena.fuseki.Fuseki;
-
-/**
- * Policy for allowing users to execute a request. Assumes the user has been
- * authenticated.
- */
-public class RequestAuthorization {
-
-    private static String ANY_USER  = "*";
-    private final Set<String>        allowedUsers;
-    private final boolean            allowAllUsers;
-    private final boolean            allowAnon;
-
-    /** Allow specific users */ 
-    public static RequestAuthorization policyAllowSpecific(String... allowedUsers) {
-        return new RequestAuthorization(Arrays.asList(allowedUsers), false, false);
-    }
-
-    /** Allow specific users */ 
-    public static RequestAuthorization policyAllowSpecific(Collection<String> allowedUsers) {
-        return new RequestAuthorization(allowedUsers, false, false);
-    }
-
-    /** Allow authenticated (logged in) user. */ 
-    public static RequestAuthorization policyAllowAuthenticated() {
-        return new RequestAuthorization(null, true, false);
-    }
-
-    /** Allow without authentication */ 
-    public static RequestAuthorization policyAllowAnon() {
-        return new RequestAuthorization(null, true, true);
-    }
-
-    /** Allow without authentication */ 
-    public static RequestAuthorization policyNoAccess() {
-        return new RequestAuthorization(Collections.emptySet(), false, false);
-    }
-
-    public RequestAuthorization(Collection<String> allowed, final boolean allowAllUsers, final boolean allowAnon) {
-        // -- anon.
-        if ( allowAnon ) {
-            if ( !isNullOrEmpty(allowed) ) {
-                //warn
-            }
-            this.allowAnon = true;
-            this.allowAllUsers = true;
-            this.allowedUsers = Collections.emptySet();
-            return;
-        }
-        this.allowAnon = false;
-        
-        // -- "any user"
-        if ( allowAllUsers || contains(allowed, ANY_USER) ) {
-            if ( allowed != null && allowed.size() > 1 )
-                Fuseki.configLog.warn("Both 'any user' and a list of users given");
-            this.allowAllUsers = true;
-            this.allowedUsers = Collections.emptySet();
-            return ;
-        }
-
-        // -- List of users
-        this.allowedUsers = (allowed == null) ? Collections.emptySet() : new HashSet<>(allowed);
-        this.allowAllUsers = false;
-    }
-    
-    public boolean isAllowed(String user) {
-        if ( allowAnon )
-            return true;
-        if ( user == null )
-            return false;
-        if ( allowAllUsers )
-            return true;
-        if ( contains(allowedUsers, user) )
-            return true;
-        return false;
-    }
-
-    public boolean isDenied(String user) {
-        return !isAllowed(user);
-    }
-
-    static <T> boolean isNullOrEmpty(Collection<T> collection) {
-        if ( collection == null )
-            return true;
-        return collection.isEmpty(); 
-    }
-    
-    static <T> boolean contains(Collection<T> collection, T obj) {
-        if ( collection == null )
-            return false;
-        return collection.contains(obj); 
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionCtl.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionCtl.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionCtl.java
index 756d948..9ed6293 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionCtl.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionCtl.java
@@ -49,7 +49,7 @@ public abstract class ActionCtl extends ActionBase {
         }
         
         action.setControlRequest(dataAccessPoint, datasetUri) ;
-        action.setEndpoint(null, null) ;   // No operation or service name.
+        action.setEndpoint(null) ;   // No operation or service name.
         executeAction(action) ;
     }
 

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionStats.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionStats.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionStats.java
index f9ab32f..f5bec6b 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionStats.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/ActionStats.java
@@ -102,7 +102,7 @@ public class ActionStats extends ActionContainerItem
             
             for ( Endpoint endpoint : endpoints ) {
                 // Endpoint names are unique for a given service.
-                builder.key(endpoint.getEndpoint()) ;
+                builder.key(endpoint.getName()) ;
                 builder.startObject() ;
                 
                 operationCounters(builder, endpoint);

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/JsonDescription.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/JsonDescription.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/JsonDescription.java
index d4d4ad5..e74b0a4 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/JsonDescription.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/ctl/JsonDescription.java
@@ -64,7 +64,7 @@ public class JsonDescription {
         builder.key(ServerConst.srvEndpoints) ;
         builder.startArray() ;
         for ( Endpoint endpoint : endpoints )
-            builder.value(endpoint.getEndpoint()) ;
+            builder.value(endpoint.getName()) ;
         builder.finishArray() ;
 
         builder.finishObject() ;

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/jetty/FusekiErrorHandler.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/jetty/FusekiErrorHandler.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/jetty/FusekiErrorHandler.java
index 5ec3882..84990cc 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/jetty/FusekiErrorHandler.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/jetty/FusekiErrorHandler.java
@@ -26,7 +26,6 @@ import javax.servlet.http.HttpServletRequest ;
 import javax.servlet.http.HttpServletResponse ;
 
 import org.apache.jena.atlas.io.IO ;
-import org.apache.jena.fuseki.Fuseki ;
 import org.apache.jena.fuseki.servlets.ServletOps ;
 import org.apache.jena.web.HttpSC ;
 import org.eclipse.jetty.http.HttpMethod ;
@@ -58,19 +57,10 @@ public class FusekiErrorHandler extends ErrorHandler
         
         ByteArrayOutputStream bytes = new ByteArrayOutputStream(1024) ;
         try ( Writer writer = IO.asUTF8(bytes) ) {
-            String reason=(response instanceof Response)?((Response)response).getReason():null;
+            String reason = (response instanceof Response) ? ((Response)response).getReason() : null;
             handleErrorPage(request, writer, response.getStatus(), reason) ;
-
-            if ( ! Fuseki.VERSION.equalsIgnoreCase("development") &&
-                 ! Fuseki.VERSION.equals("${project.version}") )
-            {
-                writer.write("\n") ;
-                writer.write("\n") ;
-                writer.write(format("Fuseki - version %s (Build date: %s)\n", Fuseki.VERSION, Fuseki.BUILD_DATE)) ;
-            }
             writer.flush();
             response.setContentLength(bytes.size()) ;
-            // Copy :-(
             response.getOutputStream().write(bytes.toByteArray()) ;
         }
     }

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataAccessPointRegistry.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataAccessPointRegistry.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataAccessPointRegistry.java
index 4c07c69..f85453d 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataAccessPointRegistry.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataAccessPointRegistry.java
@@ -53,7 +53,7 @@ public class DataAccessPointRegistry extends Registry<String, DataAccessPoint>
             System.out.printf("  (key=%s, ref=%s)\n", k, ref.getName()) ;
             ref.getDataService().getOperations().forEach((op)->{
                 ref.getDataService().getEndpoints(op).forEach(ep->{
-                    System.out.printf("     %s : %s\n", op, ep.getEndpoint()) ;
+                    System.out.printf("     %s : %s\n", op, ep.getName()) ;
                 });
             });
         }) ;

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataService.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataService.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataService.java
index 1236e24..8f02877 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataService.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/DataService.java
@@ -29,7 +29,7 @@ import org.apache.jena.ext.com.google.common.collect.ArrayListMultimap;
 import org.apache.jena.ext.com.google.common.collect.ListMultimap;
 import org.apache.jena.fuseki.Fuseki;
 import org.apache.jena.fuseki.FusekiException;
-import org.apache.jena.fuseki.build.RequestAuthorization;
+import org.apache.jena.fuseki.auth.AuthPolicy;
 import org.apache.jena.query.TxnType;
 import org.apache.jena.query.text.DatasetGraphText;
 import org.apache.jena.sparql.core.DatasetGraph;
@@ -39,7 +39,7 @@ public class DataService {
 
     private ListMultimap<Operation, Endpoint> operations  = ArrayListMultimap.create();
     private Map<String, Endpoint> endpoints               = new HashMap<>();
-    private RequestAuthorization requestAuth              = null;
+    private AuthPolicy authPolicy                         = null;
 
     /**
      * Record which {@link DataAccessPoint DataAccessPoints} this {@code DataService} is
@@ -96,7 +96,11 @@ public class DataService {
     }
     
     public void addEndpoint(Operation operation, String endpointName) {
-        Endpoint endpoint = new Endpoint(operation, endpointName);
+        addEndpoint(operation, endpointName, null);
+    }
+    
+    public void addEndpoint(Operation operation, String endpointName, AuthPolicy authPolicy) {
+        Endpoint endpoint = new Endpoint(operation, endpointName, authPolicy);
         endpoints.put(endpointName, endpoint);
         operations.put(operation, endpoint);
     }
@@ -215,10 +219,10 @@ public class DataService {
             dataset.close();
     }
 
-    public void setAllowedUsers(RequestAuthorization allowedUsers) { this.requestAuth = allowedUsers; }
+    public void setAuthPolicy(AuthPolicy authPolicy) { this.authPolicy = authPolicy; }
     
-    /** Returning null implies no access control */
-    public RequestAuthorization allowedUsers() { return requestAuth; }
+    /** Returning null implies no authorization control */
+    public AuthPolicy authPolicy() { return authPolicy; }
 
 }
 

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/Endpoint.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/Endpoint.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/Endpoint.java
index c54309d..9b95a78 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/Endpoint.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/Endpoint.java
@@ -18,51 +18,66 @@
 
 package org.apache.jena.fuseki.server;
 
-import org.apache.jena.atlas.lib.InternalErrorException ;
+import java.util.Objects;
 
+import org.apache.jena.atlas.lib.InternalErrorException;
+import org.apache.jena.fuseki.auth.AuthPolicy;
+
+/*
+ * An {@code Endpoint} is an instance of an {@link Operation} within a {@link DataService} and has counters.
+ * An {@code Endpoint} may have a name which is a path component. 
+ */
 public class Endpoint implements Counters {
-    
-    public final Operation operation ;
-    public final String endpointName ;
+
+    public final Operation   operation;
+    public final String      endpointName;
+    private final AuthPolicy authPolicy;
     // Endpoint-level counters.
-    private final CounterSet counters           = new CounterSet() ;
+    private final CounterSet counters = new CounterSet();
 
-    public Endpoint(Operation operation, String endpointName) {
-        this.operation = operation ;
+    public Endpoint(Operation operation, String endpointName, AuthPolicy requestAuth) {
+        this.operation = Objects.requireNonNull(operation, "operation");
         if ( operation == null )
-            throw new InternalErrorException("operation is null") ;
-        this.endpointName = endpointName ;
+            throw new InternalErrorException("operation is null");
+        this.endpointName = Objects.requireNonNull(endpointName, "endpointName");
+        this.authPolicy = requestAuth;
         // Standard counters - there may be others
-        counters.add(CounterName.Requests) ;
-        counters.add(CounterName.RequestsGood) ;
-        counters.add(CounterName.RequestsBad) ;
+        counters.add(CounterName.Requests);
+        counters.add(CounterName.RequestsGood);
+        counters.add(CounterName.RequestsBad);
     }
 
     @Override
-    public  CounterSet getCounters()    { return counters ; }
+    public CounterSet getCounters() {
+        return counters;
+    }
 
-    //@Override
-    public Operation getOperation()     { return operation ; }
-    
-    //@Override
-    public boolean isType(Operation operation) { 
-        return operation.equals(operation) ;
+    public Operation getOperation() {
+        return operation;
     }
 
-    public String getEndpoint()         { return endpointName ; }
-    
-    //@Override 
-    public long getRequests() { 
-        return counters.value(CounterName.Requests) ;
+    public boolean isType(Operation operation) {
+        return operation.equals(operation);
     }
-    //@Override
+
+    public String getName() {
+        return endpointName;
+    }
+
+    public AuthPolicy getAuthPolicy() {
+        return authPolicy;
+    }
+
+    public long getRequests() {
+        return counters.value(CounterName.Requests);
+    }
+
     public long getRequestsGood() {
-        return counters.value(CounterName.RequestsGood) ;
+        return counters.value(CounterName.RequestsGood);
     }
-    //@Override
+
     public long getRequestsBad() {
-        return counters.value(CounterName.RequestsBad) ;
+        return counters.value(CounterName.RequestsBad);
     }
 
 }
-

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/server/FusekiInfo.java
----------------------------------------------------------------------
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 7a5643a..8312d1a 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
@@ -74,7 +74,7 @@ public class FusekiInfo {
             DataService dSrv = dap.getDataService();
             dSrv.getOperations().forEach((op)->{
                 dSrv.getEndpoints(op).forEach(ep-> {
-                    String x = ep.getEndpoint();
+                    String x = ep.getName();
                     if ( x.isEmpty() )
                         x = "quads";
                     endpoints.add(x);   

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ActionService.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ActionService.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ActionService.java
index 933faf6..eb4e540 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ActionService.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ActionService.java
@@ -24,11 +24,14 @@ import static org.apache.jena.fuseki.server.CounterName.RequestsBad;
 import static org.apache.jena.fuseki.server.CounterName.RequestsGood;
 
 import java.io.IOException;
+import java.util.Collection;
 
 import javax.servlet.ServletException;
 
 import org.apache.jena.atlas.RuntimeIOException;
+import org.apache.jena.atlas.lib.InternalErrorException;
 import org.apache.jena.fuseki.Fuseki;
+import org.apache.jena.fuseki.auth.Auth;
 import org.apache.jena.fuseki.server.*;
 import org.apache.jena.query.QueryCancelledException;
 import org.apache.jena.web.HttpSC;
@@ -50,6 +53,7 @@ public abstract class ActionService extends ActionBase {
      * executes the HTTP Action life cycle.
      */
     @Override
+    final
     protected void execCommonWorker(HttpAction action) {
         DataAccessPoint dataAccessPoint;
         DataService dSrv;
@@ -63,12 +67,7 @@ public abstract class ActionService extends ActionBase {
                 return;
             }
             dSrv = dataAccessPoint.getDataService();
-            
-            if ( dSrv.allowedUsers() != null ) {
-                String user = action.request.getRemoteUser();
-                if ( ! dSrv.allowedUsers().isAllowed(user) )
-                    ServletOps.errorForbidden();
-            }
+
             if ( !dSrv.isAcceptingRequests() ) {
                 ServletOps.error(HttpSC.SERVICE_UNAVAILABLE_503, "Dataset not currently active");
                 return;
@@ -81,42 +80,96 @@ public abstract class ActionService extends ActionBase {
         }
 
         action.setRequest(dataAccessPoint, dSrv);
+        // Endpoint Name is "" for GSP or quads.
+        // Endpoint name is not "", but unknown for GSP direct naming (which is usually disabled).
         String endpointName = mapRequestToOperation(action, dataAccessPoint);
-
+        
         // ServiceRouter dispatch
         Operation operation = null;
         if ( !endpointName.isEmpty() ) {
             operation = chooseOperation(action, dSrv, endpointName);
             if ( operation == null )
-                ServletOps.errorBadRequest(format("dataset=%s, service=%s", dataAccessPoint.getName(), endpointName));
-
+                if ( ! Fuseki.GSP_DIRECT_NAMING ) 
+                    ServletOps.errorBadRequest(format("dataset=%s, service=%s", dataAccessPoint.getName(), endpointName));
+                else
+                    throw new InternalErrorException("Inconsistent: GSP_DIRECT_NAMING but no operation");
         } else {
+            // Endpoint ""
             operation = chooseOperation(action, dSrv);
             if ( operation == null )
                 ServletOps.errorBadRequest(format("dataset=%s", dataAccessPoint.getName()));
         }
 
+        // ---- Auth checking.
+        // -- Server-level auhtorization.
+        // Checking was carried out by servlet filter AuthFilter.
+        // Need to check Data service and endpoint authorization policies.
+        String user = action.getUser();
+        // -- Data service level authorization
+        if ( dSrv.authPolicy() != null ) {
+            if ( ! dSrv.authPolicy().isAllowed(user) )
+                ServletOps.errorForbidden();
+        }
+        
+        // -- Endpoint level authorization
+        // Make sure all contribute authentication.
+        if ( action.getEndpoint() != null ) {
+            // Specific endpoint chosen.
+            Auth.allow(user, action.getEndpoint().getAuthPolicy(), ServletOps::errorForbidden);
+        } else {
+            // No Endpoint name given; there may be several endpoints for the operation.
+            // authorization is the AND of all endpoints.
+            Collection<Endpoint> x = getEndpoints(dSrv, operation);
+            if ( x.isEmpty() )
+                throw new InternalErrorException("Inconsistent: no endpoints for "+operation);
+            x.forEach(ep->{
+                Auth.allow(user, ep.getAuthPolicy(), ServletOps::errorForbidden);
+            });
+        }
+        // ---- End auth checking.
+
         ActionService handler = action.getServiceDispatchRegistry().findHandler(operation);
         if ( handler == null )
             ServletOps.errorBadRequest(format("dataset=%s: op=%s", dataAccessPoint.getName(), operation.getName()));
-        // XXX -- replace action.setEndpoint
-        Endpoint ep = dSrv.getEndpoint(endpointName);
-        action.setEndpoint(ep, endpointName);
         handler.executeLifecycle(action);
         return;
     }
 
-    // Overridden by the ServiceRouter.
-    protected Operation chooseOperation(HttpAction action, DataService dataService, String serviceName) {
+    // Find the endpoints for an operation.
+    // This is GSP_R/GSP_RW and Quads_R/Quads_RW aware.
+    // If asked for GSP_R and there are no endpoints for GSP_R, try GSP_RW.
+    // Ditto Quads_R -> Quads_RW.
+    private Collection<Endpoint> getEndpoints(DataService dSrv, Operation operation) {
+        Collection<Endpoint> x = dSrv.getEndpoints(operation);
+        if ( x == null || x.isEmpty() ) {
+            if ( operation == Operation.GSP_R )
+                x = dSrv.getEndpoints(Operation.GSP_RW);
+            else if ( operation == Operation.Quads_R )
+                x = dSrv.getEndpoints(Operation.Quads_RW);
+        }
+        return x;
+    }
+    
+    /**
+     * Return the operation that corresponds to the endpoint name for a given data service.
+     * Side effect: This operation should set the selected endpoint in the HttpAction
+     * if this operation is determined to be a specific endpoint.
+     */
+    protected Operation chooseOperation(HttpAction action, DataService dataService, String endpointName) {
+        // Overridden by the ServiceRouter.
         // This default implementation is plain service name to operation based on the
         // DataService as would be used by operation servlets bound by web.xml
         // except Fuseki can add and delete mapping while running.
-        Endpoint ep = dataService.getEndpoint(serviceName);
+        Endpoint ep = dataService.getEndpoint(endpointName);
         Operation operation = ep.getOperation();
+        action.setEndpoint(ep);
         return operation;
     }
 
-    // Overridden by the ServiceRouter.
+    /**
+     * Return the operation that corresponds to the request when there is no endpoint name. 
+     * This operation does not set the selected endpoint in the HttpAction.
+     */
     protected Operation chooseOperation(HttpAction action, DataService dataService) {
         // No default implementation for directly bound services operation servlets.
         return null;

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/AuthFilter.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/AuthFilter.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/AuthFilter.java
index fba3b50..37cc824 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/AuthFilter.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/AuthFilter.java
@@ -35,7 +35,7 @@ import org.eclipse.jetty.security.SecurityHandler;
  * filter chain.
  * <p>
  * Either the user from {@link HttpServletRequest#getRemoteUser() getRemoteUser} is null,
- * no authentication, or it has been validated. Failed authentication wil have been
+ * no authentication, or it has been validated. Failed authentication will have been
  * handled and rejected by the {@link SecurityHandler security handler} before they get to
  * the filter chain.
  */

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/FusekiFilter.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/FusekiFilter.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/FusekiFilter.java
index ad2188e..ccd174c 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/FusekiFilter.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/FusekiFilter.java
@@ -35,7 +35,7 @@ import org.slf4j.Logger ;
  */
 public class FusekiFilter implements Filter {
     private static Logger log = Fuseki.serverLog ;
-    private static ServiceRouter routerServlet = new ServiceRouter.AccessByConfig() ;
+    private static ServiceRouter routerServlet = new ServiceRouter() ;
     
     @Override
     public void init(FilterConfig filterConfig) {

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java
index 0436c5e..8757527 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java
@@ -57,7 +57,7 @@ public class HttpAction
     
     // -- Valid only for operational actions (e.g. SPARQL).
     
-    public  String          endpointName    = null ;        // Endpoint name srv was found under 
+    public  String          xendpointName    = null ;        // Endpoint name srv was found under 
     public  Endpoint        endpoint        = null ;
     private Transactional   transactional   = null ;
     private boolean         isTransactional = false ;
@@ -199,6 +199,16 @@ public class HttpAction
         return context ;
     }
 
+    /** Return the authenticated user this {@code HttpAction}.
+     * Return null for no authenticated user.
+     */
+    public String getUser() {
+        if ( request == null )
+            return null;
+        return request.getRemoteUser();
+        //Same as: return request.getUserPrincipal().getName();
+    }
+
     /**
      * Return the "Transactional" for this HttpAction.
      */
@@ -250,12 +260,10 @@ public class HttpAction
     }
     
     /** Set the endpoint and endpoint name that this is an action for. 
-     * @param srvRef {@link Endpoint}
-     * @param endpointName
+     * @param endpoint {@link Endpoint}
      */
-    public void setEndpoint(Endpoint srvRef, String endpointName) {
-        this.endpoint = srvRef ; 
-        this.endpointName = endpointName ;
+    public void setEndpoint(Endpoint endpoint) {
+        this.endpoint = endpoint ; 
     }
     
     /** Get the endpoint for the action (may be null) . */

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServiceRouter.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServiceRouter.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServiceRouter.java
index 8ec9cd1..6b8c279 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServiceRouter.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServiceRouter.java
@@ -52,89 +52,8 @@ import org.apache.jena.riot.web.HttpNames;
  * It work in conjunction with {@link ActionService#execCommonWorker} to decide where to
  * route requests.
  */
-public abstract class ServiceRouter extends ActionService {
-    protected abstract boolean allowQuery(HttpAction action);
-
-    protected abstract boolean allowUpdate(HttpAction action);
-
-    protected abstract boolean allowGSP_R(HttpAction action);
-
-    protected abstract boolean allowGSP_RW(HttpAction action);
-
-    protected abstract boolean allowQuads_R(HttpAction action);
-
-    protected abstract boolean allowQuads_RW(HttpAction action);
-
-//    public static class ReadOnly extends ServiceRouterServlet {
-//        public ReadOnly() { super() ; }
-//        @Override protected boolean allowQuery(HttpAction action) { return true ; }
-//        @Override protected boolean allowUpdate(HttpAction action) { return false ; }
-//        @Override protected boolean allowGSP_R(HttpAction action) { return true ; }
-//        @Override protected boolean allowGSP_RW(HttpAction action) { return false ; }
-//        @Override protected boolean allowQuads_R(HttpAction action) { return true ; }
-//        @Override protected boolean allowQuads_RW(HttpAction action) { return false ; }
-//    }
-//
-//    public static class ReadWrite extends ServiceRouterServlet {
-//        public ReadWrite() { super() ; }
-//        @Override protected boolean allowQuery(HttpAction action) { return true ; }
-//        @Override protected boolean allowUpdate(HttpAction action) { return true ; }
-//        @Override protected boolean allowGSP_R(HttpAction action) { return true ; }
-//        @Override protected boolean allowGSP_RW(HttpAction action) { return true ; }
-//        @Override protected boolean allowQuads_R(HttpAction action) { return true ; }
-//        @Override protected boolean allowQuads_RW(HttpAction action) { return true ; }
-//    }
-
-    public static class AccessByConfig extends ServiceRouter {
-        public AccessByConfig() {
-            super();
-        }
-
-        @Override
-        protected boolean allowQuery(HttpAction action) {
-            return isEnabled(action, Operation.Query);
-        }
-
-        @Override
-        protected boolean allowUpdate(HttpAction action) {
-            return isEnabled(action, Operation.Update);
-        }
-
-        @Override
-        protected boolean allowGSP_R(HttpAction action) {
-            return isEnabled(action, Operation.GSP_R) || isEnabled(action, Operation.GSP_RW);
-        }
-
-        @Override
-        protected boolean allowGSP_RW(HttpAction action) {
-            return isEnabled(action, Operation.GSP_RW);
-        }
-
-        @Override
-        protected boolean allowQuads_R(HttpAction action) {
-            return isEnabled(action, Operation.Quads_R) || isEnabled(action, Operation.Quads_RW);
-        }
-
-        @Override
-        protected boolean allowQuads_RW(HttpAction action) {
-            return isEnabled(action, Operation.Quads_RW);
-        }
-
-        /**
-         * Test whether there is a configuration that allows this action as the operation
-         * given. Ignores the operation in the action which is set due to parsing - it
-         * might be "quads" which is the generic operation when just the dataset is
-         * specified.
-         */
-        private boolean isEnabled(HttpAction action, Operation operation) {
-            // Disregard the operation name of the action
-            DataService dSrv = action.getDataService();
-            if ( dSrv == null )
-                return false;
-            return !dSrv.getEndpoints(operation).isEmpty();
-        }
-    }
-
+public class ServiceRouter extends ActionService {
+    
     public ServiceRouter() {
         super();
     }
@@ -165,14 +84,19 @@ public abstract class ServiceRouter extends ActionService {
      * {@link Fuseki#GSP_DIRECT_NAMING}.
      */
     @Override
-    protected Operation chooseOperation(HttpAction action, DataService dataService, String serviceName) {
-        // Check enabled happens here. 
-        // Must be enabled by configuration to be in the lookup.
-        Endpoint ep = dataService.getEndpoint(serviceName);
+    protected Operation chooseOperation(HttpAction action, DataService dataService, String endpointName) {
+        // Default implementation in ActionService:
+//        Endpoint ep = dataService.getEndpoint(endpointName);
+//        Operation operation = ep.getOperation();
+//        action.setEndpoint(ep);
+        
+        Endpoint ep = dataService.getEndpoint(endpointName);
         if ( ep != null ) {
             Operation operation = ep.getOperation();
-            if ( operation != null ) {
-                // If GSP, no params means Quads operation.
+            action.setEndpoint(ep);
+            if ( operation != null ) { 
+                // Can this be null?
+                // If a GSP operation, then no params means Quads operation.
                 if ( operation.equals(Operation.GSP_R) || operation.equals(Operation.GSP_RW) ) {
                     // Look for special case. Quads on the GSP service endpoint.
                     boolean hasParamGraph = action.request.getParameter(HttpNames.paramGraph) != null;
@@ -186,12 +110,14 @@ public abstract class ServiceRouter extends ActionService {
                 }
                 return operation;
             }
+            System.err.printf("Notice: endpoint %s but no operation", endpointName);
         }
 
+        // No endpoint.
         // There is a trailing part - unrecognized service name ==> GSP direct naming.
         if ( !Fuseki.GSP_DIRECT_NAMING )
             ServletOps.errorNotFound(
-                "Not found: dataset='" + printName(action.getDataAccessPoint().getName()) + "' service='" + printName(serviceName) + "'");
+                "Not found: dataset='" + printName(action.getDataAccessPoint().getName()) + "' endpoint='" + printName(endpointName) + "'");
         // GSP Direct naming - the servlets handle direct and indirct naming.
         return gspOperation(action, action.request);
     }
@@ -207,9 +133,8 @@ public abstract class ServiceRouter extends ActionService {
      * <li>Content type</li>
      * </ul>
      */
-    @Override
+    @Override final
     protected Operation chooseOperation(HttpAction action, DataService dataService) {
-        Endpoint ep = dataService.getEndpoint("");
         HttpServletRequest request = action.getRequest();
 
         // ---- Dispatch based on HttpParams : Query, Update, GSP.
@@ -351,8 +276,8 @@ public abstract class ServiceRouter extends ActionService {
         boolean isHEAD = method.equals(HttpNames.METHOD_HEAD);
         return isGET || isHEAD;
     }
-
-    private String printName(String x) {
+    
+    private static String printName(String x) {
         if ( x.startsWith("/") )
             return x.substring(1);
         return x;
@@ -388,4 +313,48 @@ public abstract class ServiceRouter extends ActionService {
     protected void doDelete(HttpServletRequest request, HttpServletResponse response) {
         doCommon(request, response);
     }
+
+    // Check whether an operation is allowed by the setup.
+    // This is used when the operation/endpoint is not named directly
+    // e.g. http://host:port/dataset?query= 
+    //   is implicitly a call on 
+    // http://host:port/dataset/sparql?query=
+    
+    protected boolean allowQuery(HttpAction action) {
+        return isEnabled(action, Operation.Query);
+    }
+
+    protected boolean allowUpdate(HttpAction action) {
+        return isEnabled(action, Operation.Update);
+    }
+
+    protected boolean allowGSP_R(HttpAction action) {
+        return isEnabled(action, Operation.GSP_R) || isEnabled(action, Operation.GSP_RW);
+    }
+
+    protected boolean allowGSP_RW(HttpAction action) {
+        return isEnabled(action, Operation.GSP_RW);
+    }
+
+    protected boolean allowQuads_R(HttpAction action) {
+        return isEnabled(action, Operation.Quads_R) || isEnabled(action, Operation.Quads_RW);
+    }
+
+    protected boolean allowQuads_RW(HttpAction action) {
+        return isEnabled(action, Operation.Quads_RW);
+    }
+
+    /**
+     * Test whether there is a configuration that allows this action as the operation
+     * given. Ignores the operation in the action which is set due to parsing - it
+     * might be "quads" which is the generic operation when just the dataset is
+     * specified.
+     */
+    private boolean isEnabled(HttpAction action, Operation operation) {
+        // Disregard the operation name of the action.
+        DataService dSrv = action.getDataService();
+        if ( dSrv == null )
+            return false;
+        return !dSrv.getEndpoints(operation).isEmpty();
+    }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServletOps.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServletOps.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServletOps.java
index 0fa6add..c9f9029 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServletOps.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/ServletOps.java
@@ -24,6 +24,7 @@ import java.io.PrintWriter ;
 import javax.servlet.ServletOutputStream ;
 import javax.servlet.http.HttpServletResponse ;
 
+import org.apache.jena.atlas.io.IndentedWriter;
 import org.apache.jena.atlas.json.JSON ;
 import org.apache.jena.atlas.json.JsonValue ;
 import org.apache.jena.fuseki.system.UploadDetails;
@@ -193,8 +194,12 @@ public class ServletOps {
             ServletOutputStream out = response.getOutputStream() ;
             response.setContentType(WebContent.contentTypeJSON);
             response.setCharacterEncoding(WebContent.charsetUTF8) ;
-            JSON.write(out, v) ;
-            out.println() ; 
+            
+            IndentedWriter iOut = new IndentedWriter(out) ;
+            JSON.write(iOut, v) ;
+            // Make sure we end with a newline.
+            iOut.ensureStartOfLine();
+            iOut.flush() ;
             out.flush() ;
         } catch (IOException ex) { ServletOps.errorOccurred(ex) ; }
     }

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-core/src/test/java/org/apache/jena/fuseki/Dummy.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/test/java/org/apache/jena/fuseki/Dummy.java b/jena-fuseki2/jena-fuseki-core/src/test/java/org/apache/jena/fuseki/Dummy.java
index 50255c0..0969d76 100644
--- a/jena-fuseki2/jena-fuseki-core/src/test/java/org/apache/jena/fuseki/Dummy.java
+++ b/jena-fuseki2/jena-fuseki-core/src/test/java/org/apache/jena/fuseki/Dummy.java
@@ -18,6 +18,10 @@
 
 package org.apache.jena.fuseki;
 
+/** 
+ * Tests in jena-fuseki-main and jena-fuseki-webapp
+ * because so many rely on having a server to run.
+ */  
 public class Dummy {
 
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/FusekiServer.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/FusekiServer.java b/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/FusekiServer.java
index 4f63c84..3d298e8 100644
--- a/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/FusekiServer.java
+++ b/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/FusekiServer.java
@@ -34,9 +34,10 @@ import org.apache.jena.fuseki.Fuseki;
 import org.apache.jena.fuseki.FusekiConfigException;
 import org.apache.jena.fuseki.FusekiException;
 import org.apache.jena.fuseki.access.DataAccessCtl;
+import org.apache.jena.fuseki.auth.Auth;
+import org.apache.jena.fuseki.auth.AuthPolicy;
 import org.apache.jena.fuseki.build.FusekiBuilder;
 import org.apache.jena.fuseki.build.FusekiConfig;
-import org.apache.jena.fuseki.build.RequestAuthorization;
 import org.apache.jena.fuseki.ctl.ActionPing;
 import org.apache.jena.fuseki.ctl.ActionStats;
 import org.apache.jena.fuseki.jetty.FusekiErrorHandler1;
@@ -235,7 +236,7 @@ public class FusekiServer {
         private boolean                  verbose            = false;
         private boolean                  withStats          = false;
         private boolean                  withPing           = false;
-        private RequestAuthorization     serverAllowedUsers = null;
+        private AuthPolicy               serverAuth         = null;
         private String                   passwordFile       = null;
         private String                   realm              = null;
         private AuthScheme               authScheme             = null;
@@ -457,7 +458,8 @@ public class FusekiServer {
             Model model = AssemblerUtils.readAssemblerFile(filename);
 
             Resource server = FusekiConfig.findServer(model);
-            processServerLevel(server);
+            // [AuthAll]
+            processServerLevel(server); 
             
             // Process server and services, whether via server ja:services or, if absent, by finding by type.
             // Side effect - sets global context.
@@ -486,9 +488,20 @@ public class FusekiServer {
             String authStr = GraphUtils.getAsStringValue(server, FusekiVocab.pAuth);
             authScheme = AuthScheme.scheme(authStr);
             
-            serverAllowedUsers = FusekiBuilder.allowedUsers(server);
+            serverAuth = FusekiBuilder.allowedUsers(server);
         }
 
+        /** Process password file, auth and rela settings on the server description. **/
+        private void processAuthentication(Resource server) {
+            passwordFile = GraphUtils.getAsStringValue(server, FusekiVocab.pPasswordFile);
+            if ( passwordFile != null )
+                passwordFile(passwordFile);
+            realm = GraphUtils.getAsStringValue(server, FusekiVocab.pRealm);
+            if ( realm == null )
+                realm = "TripleStore"; 
+            realm(realm);
+        }
+        
         /**
          * Choose the HTTP authentication scheme. 
          */
@@ -656,19 +669,19 @@ public class FusekiServer {
                     .map(name-> dataAccessPoints.get(name).getDataService().getDataset())
                     .anyMatch(DataAccessCtl::isAccessControlled);
 
-            hasAllowedUsers = ( serverAllowedUsers != null );
+            hasAllowedUsers = ( serverAuth != null );
             if ( ! hasAllowedUsers ) {
                 // Any datasets with allowedUsers?
                 hasAllowedUsers =
                     dataAccessPoints.keys().stream()
                         .map(name-> dataAccessPoints.get(name).getDataService())
-                        .anyMatch(dSvc->dSvc.allowedUsers() != null);
+                        .anyMatch(dSvc->dSvc.authPolicy() != null);
             }
             
             // If only a password file given, and nothing else, set the server to allowedUsers="*" (must log in).  
             if ( passwordFile != null && ! hasAllowedUsers ) {
                 hasAllowedUsers = true;
-                serverAllowedUsers = RequestAuthorization.policyAllowAuthenticated();
+                serverAuth = Auth.ANY_USER;
             }
         }
         
@@ -729,12 +742,12 @@ public class FusekiServer {
             // -- Access control
             if ( securityHandler != null && securityHandler instanceof ConstraintSecurityHandler ) {
                 ConstraintSecurityHandler csh = (ConstraintSecurityHandler)securityHandler;
-                if ( serverAllowedUsers != null )
+                if ( serverAuth != null )
                     JettyLib.addPathConstraint(csh, "/*");
                 else {
                     DataAccessPointRegistry.get(cxt).forEach((name, dap)-> {
                         DatasetGraph dsg = dap.getDataService().getDataset();
-                        if ( dap.getDataService().allowedUsers() != null ) {
+                        if ( dap.getDataService().authPolicy() != null ) {
                             // Not need if whole server ACL'ed.
                             JettyLib.addPathConstraint(csh, DataAccessPoint.canonical(name));
                             JettyLib.addPathConstraint(csh, DataAccessPoint.canonical(name)+"/*");
@@ -756,7 +769,7 @@ public class FusekiServer {
             context.setContextPath(contextPath);
             if ( securityHandler != null ) {
                 context.setSecurityHandler(securityHandler);
-                if ( serverAllowedUsers != null ) {
+                if ( serverAuth != null ) {
                     ConstraintSecurityHandler csh = (ConstraintSecurityHandler)securityHandler;
                     JettyLib.addPathConstraint(csh, "/*");
                 }
@@ -768,8 +781,8 @@ public class FusekiServer {
         private void servletsAndFilters(ServletContextHandler context) {
             // Fuseki dataset services filter
             // First in chain.
-            if ( serverAllowedUsers != null ) {
-                Predicate<String> auth = serverAllowedUsers::isAllowed; 
+            if ( serverAuth != null ) {
+                Predicate<String> auth = serverAuth::isAllowed; 
                 AuthFilter authFilter = new AuthFilter(auth);
                 addFilter(context, "/*", authFilter);
                 //JettyLib.addPathConstraint(null, contextPath);

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-main/src/main/java/org/apache/jena/fuseki/main/cmds/FusekiMain.java
----------------------------------------------------------------------
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 e1505b1..f6ba99b 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
@@ -547,7 +547,7 @@ public class FusekiMain extends CmdARQ {
                 DataService dSrv = dap.getDataService();
                 dSrv.getOperations().forEach((op)->{
                     dSrv.getEndpoints(op).forEach(ep-> {
-                        String x = ep.getEndpoint();
+                        String x = ep.getName();
                         if ( x.isEmpty() )
                             x = "quads";
                         endpoints.add(x);   

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/TestFusekiTestServer.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/TestFusekiTestServer.java b/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/TestFusekiTestServer.java
index 8637d36..ebed98f 100644
--- a/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/TestFusekiTestServer.java
+++ b/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/TestFusekiTestServer.java
@@ -53,7 +53,8 @@ public class TestFusekiTestServer {
         // No auth set - should work.
         try ( TypedInputStream in = HttpOp.execHttpGet(FusekiTestServer.urlDataset(), "*/*") ) {}
         catch (HttpException ex) {
-            Assert.assertTrue(ex.getResponseCode() == HttpSC.FORBIDDEN_403 || ex.getResponseCode() == HttpSC.UNAUTHORIZED_401 );
+            if ( ex.getResponseCode() == HttpSC.FORBIDDEN_403 || ex.getResponseCode() == HttpSC.UNAUTHORIZED_401 ) 
+                return;
             throw ex;
         }
     }

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/access/TS_SecurityFuseki.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/access/TS_SecurityFuseki.java b/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/access/TS_SecurityFuseki.java
index d34023c..e344d60 100644
--- a/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/access/TS_SecurityFuseki.java
+++ b/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/access/TS_SecurityFuseki.java
@@ -26,7 +26,7 @@ import org.junit.runners.Suite;
 
 @RunWith(Suite.class)
 @Suite.SuiteClasses( {
-    TestAuthorizedRequest.class
+    TestAuthorized.class
     , TestSecurityFilterFuseki.class
     , TestFusekiSecurityAssemblerSeparate.class
     , TestFusekiSecurityAssemblerShared.class

http://git-wip-us.apache.org/repos/asf/jena/blob/ca783936/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/access/TestAuthorized.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/access/TestAuthorized.java b/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/access/TestAuthorized.java
new file mode 100644
index 0000000..0562c49
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-main/src/test/java/org/apache/jena/fuseki/main/access/TestAuthorized.java
@@ -0,0 +1,111 @@
+/*
+ * 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.access;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.jena.fuseki.auth.Auth;
+import org.apache.jena.fuseki.auth.AuthPolicy;
+import org.apache.jena.fuseki.build.FusekiBuilder;
+import org.apache.jena.rdf.model.Model;
+import org.apache.jena.rdf.model.Resource;
+import org.apache.jena.riot.RDFDataMgr;
+import org.junit.Test;
+
+public class TestAuthorized {
+    
+    static Model model = RDFDataMgr.loadModel("testing/Access/allowedUsers.ttl");
+    
+    @Test public void auth_anon() {
+        AuthPolicy auth = Auth.ANY_ANON;
+        assertTrue(auth.isAllowed(null));
+        assertTrue(auth.isAllowed("user1"));
+    }
+    
+    @Test public void auth_anyLoggedIn_1() {
+        AuthPolicy auth = Auth.ANY_USER;
+        assertFalse(auth.isAllowed(null));
+        assertTrue(auth.isAllowed("user1"));
+    }
+    
+    @Test public void auth_anyLoggedIn_2() {
+        AuthPolicy auth = Auth.policyAllowSpecific("*");
+        assertFalse(auth.isAllowed(null));
+        assertTrue(auth.isAllowed("user1"));
+    }
+
+    @Test public void auth_noOne() {
+        AuthPolicy auth = Auth.DENY;
+        assertFalse(auth.isAllowed(null));
+        assertFalse(auth.isAllowed("user1"));
+    }
+
+
+    @Test public void auth_user_1() {
+        AuthPolicy auth = Auth.policyAllowSpecific("user1", "user2");
+        assertFalse(auth.isAllowed(null));
+        assertTrue(auth.isAllowed("user1"));
+        assertTrue(auth.isAllowed("user2"));
+        assertFalse(auth.isAllowed("user3"));
+    }
+    
+    @Test public void auth_parse_no_info_1() {
+        Resource r = model.createResource("http://example/notInData");
+        AuthPolicy auth = FusekiBuilder.allowedUsers(r);
+        assertNull(auth);
+    }
+
+    @Test public void auth_parse_no_info_2() {
+        Resource r = model.createResource("http://example/none");
+        AuthPolicy auth = FusekiBuilder.allowedUsers(r);
+        assertNull(auth);
+    }
+
+    @Test public void auth_parse_1() {
+        Resource r = model.createResource("http://example/r1");
+        AuthPolicy auth = FusekiBuilder.allowedUsers(r);
+        assertNotNull(auth);
+        assertFalse(auth.isAllowed(null));
+        assertTrue(auth.isAllowed("user1"));
+        assertTrue(auth.isAllowed("user2"));
+        assertFalse(auth.isAllowed("user3"));
+    }
+    
+    @Test public void auth_parse_2() {
+        Resource r = model.createResource("http://example/r2");
+        AuthPolicy auth = FusekiBuilder.allowedUsers(r);
+        assertNotNull(auth);
+        assertFalse(auth.isAllowed(null));
+        assertTrue(auth.isAllowed("user1"));
+        assertTrue(auth.isAllowed("user2"));
+        assertFalse(auth.isAllowed("user3"));
+    }
+    
+    @Test public void auth_parse_loggedIn() {
+        Resource r = model.createResource("http://example/rLoggedIn");
+        AuthPolicy auth = FusekiBuilder.allowedUsers(r);
+        assertNotNull(auth);
+        assertFalse(auth.isAllowed(null));
+        assertTrue(auth.isAllowed("user1"));
+        assertTrue(auth.isAllowed("user3"));
+    }
+}