You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by ch...@apache.org on 2016/02/11 21:01:43 UTC

qpid-dispatch git commit: Add policy switch to disable ruleset processing. Add description of policyRuleset settings map contents. Add second ruleset to test conf to allow management access. Add guts of connection shutdown accounting. TODO: Need to get p

Repository: qpid-dispatch
Updated Branches:
  refs/heads/crolke-DISPATCH-188-1 95e970220 -> b9c73771a


Add policy switch to disable ruleset processing.
Add description of policyRuleset settings map contents.
Add second ruleset to test conf to allow management access.
Add guts of connection shutdown accounting.
TODO: Need to get python calls out of shutdown loop


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

Branch: refs/heads/crolke-DISPATCH-188-1
Commit: b9c73771ac3e1aeb3f162e70fbc28381a064ddf4
Parents: 95e9702
Author: Chuck Rolke <cr...@redhat.com>
Authored: Thu Feb 11 14:55:56 2016 -0500
Committer: Chuck Rolke <cr...@redhat.com>
Committed: Thu Feb 11 14:55:56 2016 -0500

----------------------------------------------------------------------
 python/qpid_dispatch/management/qdrouter.json   |   8 ++
 .../qdrouter.policyRuleset.settings.txt         | 105 +++++++++++++++++
 .../qpid_dispatch_internal/management/config.py |   2 +-
 .../policy/policy_local.py                      |  36 +++++-
 .../policy/policy_manager.py                    |  35 ++++--
 .../policy/policy_util.py                       |   3 +-
 src/policy.c                                    | 114 ++++++++++++++-----
 src/policy_private.h                            |  35 +++++-
 src/server.c                                    |   5 +-
 src/server_private.h                            |   1 +
 tests/policy-2/test-router-with-policy.json.in  |  26 ++++-
 tests/router_policy_test.py                     |  14 +--
 tests/system_tests_policy.py                    |   2 +-
 13 files changed, 331 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/python/qpid_dispatch/management/qdrouter.json
----------------------------------------------------------------------
diff --git a/python/qpid_dispatch/management/qdrouter.json b/python/qpid_dispatch/management/qdrouter.json
index 0944b3d..6162f3d 100644
--- a/python/qpid_dispatch/management/qdrouter.json
+++ b/python/qpid_dispatch/management/qdrouter.json
@@ -1012,6 +1012,13 @@
                     "required": false,
                     "create": true
                 },
+                "enableAccessRules": {
+                    "type": "boolean",
+                    "default": false,
+                    "description": "Enable user rule set connection denial.",
+                    "required": false,
+                    "create": true
+                },
                 "connectionsProcessed": {"type": "integer", "graph": true},
                 "connectionsDenied": {"type": "integer", "graph": true},
                 "connectionsCurrent": {"type": "integer", "graph": true}
@@ -1021,6 +1028,7 @@
         "policyRuleset": {
             "description": "Per application definition of the locations from which users may connect and the groups to which users belong.",
             "extends": "configurationEntity",
+	    "operations": ["CREATE"],
             "attributes": {
                 "applicationName": {
                     "type": "string",

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/python/qpid_dispatch/management/qdrouter.policyRuleset.settings.txt
----------------------------------------------------------------------
diff --git a/python/qpid_dispatch/management/qdrouter.policyRuleset.settings.txt b/python/qpid_dispatch/management/qdrouter.policyRuleset.settings.txt
new file mode 100644
index 0000000..773df7a
--- /dev/null
+++ b/python/qpid_dispatch/management/qdrouter.policyRuleset.settings.txt
@@ -0,0 +1,105 @@
+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.
+
+
+
+The schema `qdrouterd.json` is a JSON format file that defines annotations and
+entity types of the Qpid Dispatch Router management model.  The model is based
+on the AMQP management specification.
+
+Schema entity `policyRuleset` includes several attributes of type map. In the current form the management schema provides no way to define the keys and values in these attributes. These maps cannot be specified in the schema and they cannot be checked by the schema processing. 
+
+Until the schema is extended specify embedded maps this document describes the policyRuleset settings.
+
+  "policyAppSettings": {
+      "description": "For a given user group define the policy settings applied to the user's AMQP connection.",
+      "extends": "configurationEntity",
+      "attributes": {
+          "userGroupName": {
+              "type": "string",
+              "description": "The user group to which these settings apply.",
+              "required": true
+          },
+          "maxFrameSize": {
+              "type": "integer",
+              "description": "Largest frame that may be sent on this connection. Zero implies system default. (AMQP Open, max-frame-size)",
+              "default": 65536,
+              "required": false,
+              "create": true
+          },
+          "maxMessageSize": {
+              "type": "integer",
+              "description": "Largest message size supported by links created on this connection. Zero implies system default. (AMQP Attach, max-message-size)",
+              "default": 0,
+              "required": false,
+              "create": true
+          },
+          "maxSessionWindow": {
+              "type": "integer",
+              "description": "Largest incoming and outgoing window for sessions created on this connection. Zero implies system default. (AMQP Begin, incoming-window, outgoing-window)",
+              "default": 2147483647,
+              "required": false,
+              "create": true
+          },
+          "maxSessions": {
+              "type": "integer",
+              "description": "Maximum number of sessions that may be created on this connection. Zero implies system default. (AMQP Open, channel-max)",
+              "default": 10,
+              "required": false,
+              "create": true
+          },
+          "maxSenders": {
+              "type": "integer",
+              "description": "Maximum number of sending links that may be created on this connection. Zero implies system default.",
+              "default": 10,
+              "required": false,
+              "create": true
+          },
+          "maxReceivers": {
+              "type": "integer",
+              "description": "Maximum number of receiving links that may be created on this connection. Zero implies system default.",
+              "required": false,
+              "create": true
+          },
+          "allowDynamicSrc": {
+              "type": "boolean",
+              "description": "This connection is allowed to use the dynamic link source feature.",
+              "default": false,
+              "required": false,
+              "create": true
+          },
+          "allowAnonymousSender": {
+              "type": "boolean",
+              "description": "This connection is allowed to use the Anonymous Sender feature.",
+              "default": false,
+              "required": false,
+              "create": true
+          },
+          "sources": {
+              "type": "string",
+              "description": "List of Source addresses allowed when creating receiving links.",
+              "required": false,
+              "create": true
+          },
+          "targets": {
+              "type": "string",
+              "description": "List of Target addresses allowed when creating sending links.",
+              "required": false,
+              "create": true
+          }
+      }
+  }

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/python/qpid_dispatch_internal/management/config.py
----------------------------------------------------------------------
diff --git a/python/qpid_dispatch_internal/management/config.py b/python/qpid_dispatch_internal/management/config.py
index eaab7e2..8e5ef2b 100644
--- a/python/qpid_dispatch_internal/management/config.py
+++ b/python/qpid_dispatch_internal/management/config.py
@@ -164,7 +164,7 @@ def configure_dispatch(dispatch, lib_handle, filename):
     # Remaining configuration
 
     for t in "fixedAddress", "listener", "connector", "waypoint", "linkRoutePattern", \
-             "policy", "policyAccessRuleset", "policyAppSettings":
+             "policy", "policyRuleset":
         for a in config.by_type(t): configure(a)
     for e in config.entities:
         configure(e)

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/python/qpid_dispatch_internal/policy/policy_local.py
----------------------------------------------------------------------
diff --git a/python/qpid_dispatch_internal/policy/policy_local.py b/python/qpid_dispatch_internal/policy/policy_local.py
index 9a3ce4d..4f25111 100644
--- a/python/qpid_dispatch_internal/policy/policy_local.py
+++ b/python/qpid_dispatch_internal/policy/policy_local.py
@@ -394,6 +394,17 @@ class AppStats(object):
     def can_connect(self, conn_id, user, host, diags):
         return self.conn_mgr.can_connect(conn_id, user, host, diags)
 
+    def disconnect(self, conn_id, user, host):
+        self.conn_mgr.disconnect(conn_id, user, host)
+#
+#
+class ConnectionFacts:
+    def __init__(self, user, host, app, conn_name):
+        self.user = user
+        self.host = host
+        self.app = app
+        self.conn_name = conn_name
+
 #
 #
 class PolicyLocal(object):
@@ -436,7 +447,12 @@ class PolicyLocal(object):
         #  validates incoming policy and readies it for internal use
         self._policy_compiler = PolicyCompiler()
 
-
+        # _connections is a map
+        #  key : numeric connection id
+        #  val : ConnectionFacts
+        # Entries created as connection AMQP Opens arrive
+        # Entries destroyed as sockets closed
+        self._connections = {}
     #
     # Service interfaces
     #
@@ -500,7 +516,7 @@ class PolicyLocal(object):
     #
     # Runtime query interface
     #
-    def lookup_user(self, user, host, app, conn_name):
+    def lookup_user(self, user, host, app, conn_name, conn_id):
         """
         Lookup function called from C.
         Determine if a user on host accessing app through AMQP Open is allowed
@@ -511,6 +527,7 @@ class PolicyLocal(object):
         @param[in] host connection remote host numeric IP address as string
         @param[in] app application user is accessing
         @param[in] conn_name connection name used for tracking reports
+        @param[in] conn_id internal connection id
         @return settings user-group name if allowed; "" if not allowed
         """
         try:
@@ -571,6 +588,10 @@ class PolicyLocal(object):
                         "%s" % (user, host, app, diag))
                 return ""
 
+            # Record facts about this connection to use during teardown
+            facts = ConnectionFacts(user, host, app, conn_name)
+            self._connections[conn_id] = facts
+
             # Return success
             return usergroup
 
@@ -613,6 +634,17 @@ class PolicyLocal(object):
             #pdb.set_trace()
             return ""
 
+    def close_connection(self, conn_id):
+        """
+        Close the connection.
+        @param conn_id:
+        @return:
+        """
+        facts = self._connections[conn_id]
+        stats = self.statsdb[facts.app]
+        stats.disconnect(facts.conn_name, facts.user, facts.host)
+
+
     def test_load_config(self):
         ruleset_str = '["policyAccessRuleset", {"applicationName": "photoserver","maxConnections": 50,"maxConnPerUser": 5,"maxConnPerHost": 20,"userGroups": {"anonymous":       "anonymous","users":           "u1, u2","paidsubscribers": "p1, p2","test":            "zeke, ynot","admin":           "alice, bob","superuser":       "ellen"},"connectionGroups": {"Ten18":     "10.18.0.0-10.18.255.255","EllensWS":  "72.135.2.9","TheLabs":   "10.48.0.0-10.48.255.255, 192.168.100.0-192.168.100.255","localhost": "127.0.0.1, ::1","TheWorld":  "*"},"connectionIngressPolicies": {"anonymous":       "TheWorld","users":           "TheWorld","paidsubscribers": "TheWorld","test":            "TheLabs","admin":           "Ten18, TheLabs, localhost","superuser":       "EllensWS, localhost"},"connectionAllowDefault": true,'
         ruleset_str += '"settings": {'

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/python/qpid_dispatch_internal/policy/policy_manager.py
----------------------------------------------------------------------
diff --git a/python/qpid_dispatch_internal/policy/policy_manager.py b/python/qpid_dispatch_internal/policy/policy_manager.py
index 5d19cd2..20ac54c 100644
--- a/python/qpid_dispatch_internal/policy/policy_manager.py
+++ b/python/qpid_dispatch_internal/policy/policy_manager.py
@@ -76,12 +76,11 @@ class PolicyManager(object):
         @param[in] attributes: from config
         """
         self._policy_local.create_ruleset(attributes)
-        # TODO: Create stats
 
     #
     # Runtime query interface
     #
-    def lookup_user(self, user, host, app, conn_name):
+    def lookup_user(self, user, host, app, conn_name, conn_id):
         """
         Lookup function called from C.
         Determine if a user on host accessing app through AMQP Open is allowed
@@ -90,11 +89,11 @@ class PolicyManager(object):
         @param[in] user connection authId
         @param[in] host connection remote host numeric IP address as string
         @param[in] app application user is accessing
+        @param[in] conn_name connection name for accounting purposes
+        @param[in] conn_id internal connection id
         @return settings user-group name if allowed; "" if not allowed
-        # Note: the upolicy[0] output is list of group names joined with '|'.
-        TODO: handle the AccessStats
         """
-        return self._policy_local.lookup_user(user, host, app, conn_name)
+        return self._policy_local.lookup_user(user, host, app, conn_name, conn_id)
 
     def lookup_settings(self, appname, name, upolicy):
         """
@@ -105,14 +104,21 @@ class PolicyManager(object):
                     TODO: make this a c struct
         @return if allowed by policy
         # Note: the upolicy output is a non-nested dict with settings of interest
-        # TODO: figure out decent defaults for upolicy settings that are undefined
         """
         return self._policy_local.lookup_settings(appname, name, upolicy)
 
+    def close_connection(self, conn_id):
+        """
+        The connection identifed is closing. Remove it from the connection
+        accounting tables.
+        @param facts:
+        @return: none
+        """
+        self._policy_local.close_connection(conn_id)
 #
 #
 #
-def policy_lookup_user(mgr, user, host, app, conn_name):
+def policy_lookup_user(mgr, user, host, app, conn_name, conn_id):
     """
     Look up a user in the policy database
     Called by C code
@@ -123,4 +129,17 @@ def policy_lookup_user(mgr, user, host, app, conn_name):
     @param conn_name:
     @return:
     """
-    return mgr.lookup_user(user, host, app, conn_name)
+    return mgr.lookup_user(user, host, app, conn_name, conn_id)
+
+#
+#
+#
+def policy_close_connection(mgr, conn_id):
+    """
+    Close the connection.
+    Called by C code
+    @param mgr:
+    @param conn_id:
+    @return:
+    """
+    mgr.close_connection(conn_id)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/python/qpid_dispatch_internal/policy/policy_util.py
----------------------------------------------------------------------
diff --git a/python/qpid_dispatch_internal/policy/policy_util.py b/python/qpid_dispatch_internal/policy/policy_util.py
index e9b94d9..000b638 100644
--- a/python/qpid_dispatch_internal/policy/policy_util.py
+++ b/python/qpid_dispatch_internal/policy/policy_util.py
@@ -324,8 +324,7 @@ class PolicyAppConnectionMgr(object):
         assert(self.connections_active > 0)
         assert(user in self.per_user_state)
         assert(conn_id in self.per_user_state[user])
-        assert(host in self.max_per_host)
-        assert(conn_id in self.max_per_host[host])
+        assert(conn_id in self.per_host_state[host])
         self.connections_active -= 1
         self.per_user_state[user].remove(conn_id)
         self.per_host_state[host].remove(conn_id)

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/src/policy.c
----------------------------------------------------------------------
diff --git a/src/policy.c b/src/policy.c
index d3c37fb..045ab33 100644
--- a/src/policy.c
+++ b/src/policy.c
@@ -77,13 +77,16 @@ struct qd_policy_t {
                           // configured settings
     int                   max_connection_limit;
     char                 *policyDb;
+    bool                  enableAccessRules;
                           // live statistics
     int                   connections_processed;
     int                   connections_denied;
     int                   connections_current;
 };
 
-
+/** Create the policy structure
+ * @param[in] qd pointer the the qd
+ **/
 qd_policy_t *qd_policy(qd_dispatch_t *qd)
 {
     qd_policy_t *policy = NEW(qd_policy_t);
@@ -92,6 +95,7 @@ qd_policy_t *qd_policy(qd_dispatch_t *qd)
     policy->log_source           = qd_log_source("POLICY");
     policy->max_connection_limit = 0;
     policy->policyDb             = 0;
+    policy->enableAccessRules    = false;
     policy->connections_processed= 0;
     policy->connections_denied   = 0;
     policy->connections_current  = 0;
@@ -101,6 +105,9 @@ qd_policy_t *qd_policy(qd_dispatch_t *qd)
 }
 
 
+/** Free the policy structure
+ * @param[in] policy pointer to the policy
+ **/
 void qd_policy_free(qd_policy_t *policy)
 {
     if (policy->policyDb)
@@ -108,10 +115,10 @@ void qd_policy_free(qd_policy_t *policy)
     free(policy);
 }
 
-#define CHECK() if (qd_error_code()) goto error
-
 //
 //
+#define CHECK() if (qd_error_code()) goto error
+
 qd_error_t qd_entity_configure_policy(qd_policy_t *policy, qd_entity_t *entity)
 {
     policy->max_connection_limit = qd_entity_opt_long(entity, "maximumConnections", 0); CHECK();
@@ -119,6 +126,7 @@ qd_error_t qd_entity_configure_policy(qd_policy_t *policy, qd_entity_t *entity)
         return qd_error(QD_ERROR_CONFIG, "maximumConnections must be >= 0");
     policy->policyDb =
         qd_entity_opt_string(entity, "policyDb", 0); CHECK();
+    policy->enableAccessRules = qd_entity_opt_bool(entity, "enableAccessRules", false); CHECK();
     qd_log(policy->log_source, QD_LOG_INFO, "Configured maximumConnections: %d", policy->max_connection_limit);
     return QD_ERROR_NONE;
 
@@ -136,9 +144,10 @@ qd_error_t qd_register_policy_manager(qd_policy_t *policy, void *policy_manager)
     return QD_ERROR_NONE;
 }
 
-//
-//
-qd_error_t qd_entity_refresh_policy(qd_entity_t* entity, void *impl) {
+/** Update the statistics in qdrouterd.conf["policy"]
+ * @param[in] entity pointer to the policy management object
+ **/
+qd_error_t qd_entity_refresh_policy(qd_entity_t* entity, void *unused) {
     // Return global stats
     if (!qd_entity_set_long(entity, "connectionsProcessed", n_processed) &&
         !qd_entity_set_long(entity, "connectionsDenied", n_denied) &&
@@ -156,6 +165,7 @@ qd_error_t qd_entity_refresh_policy(qd_entity_t* entity, void *impl) {
 // are made and there is no AMQP channel for returning
 // error conditions.
 //
+
 bool qd_policy_socket_accept(void *context, const char *hostname)
 {
     qd_policy_t *policy = (qd_policy_t *)context;
@@ -182,16 +192,37 @@ bool qd_policy_socket_accept(void *context, const char *hostname)
 }
 
 
-void qd_policy_socket_close(void *context, const char *hostname)
+//
+//
+void qd_policy_socket_close(void *context, const qd_connection_t *conn)
 {
     qd_policy_t *policy = (qd_policy_t *)context;
 
     n_connections -= 1;
     assert (n_connections >= 0);
+    if (policy->enableAccessRules) {
+        // HACK ALERT: TODO: This should be deferred to a Python thread
+        qd_python_lock_state_t lock_state = qd_python_lock();
+        PyObject *module = PyImport_ImportModule("qpid_dispatch_internal.policy.policy_manager");
+        PyObject *close_connection = module ? PyObject_GetAttrString(module, "policy_close_connection") : NULL;
+        Py_XDECREF(module);
+        PyObject *result = close_connection ? PyObject_CallFunction(close_connection, "(OK)", 
+                                                               (PyObject *)policy->py_policy_manager, 
+                                                               conn->connection_id) : NULL;
+        Py_XDECREF(close_connection);
+        if (!result) {
+            qd_python_unlock(lock_state);
+            return;
+        }
+        Py_XDECREF(result);
+
+        qd_python_unlock(lock_state);
+
+    }
+    const char *hostname = qdpn_connector_name(conn->pn_cxtr);
     if (policy->max_connection_limit > 0) {
-        qd_log(policy->log_source, POLICY_LOG_LEVEL, "Connection '%s' closed, N=%d", hostname, n_connections);
+        qd_log(policy->log_source, POLICY_LOG_LEVEL, "Connection '%s' closed. N connections=%d", hostname, n_connections);
     }
-    qd_log(policy->log_source, POLICY_LOG_LEVEL, "Connection '%s' closed, N=%d", hostname, n_connections);  // HACK EXTRA
 }
 
 
@@ -200,50 +231,74 @@ void qd_policy_socket_close(void *context, const char *hostname)
 // An AMQP Open has been received over some connection.
 // Evaluate the connection auth and the Open fields to
 // allow or deny the Open. Denied Open attempts are
-// effected with a returned Open-Close_with_condition.
+// effected by returning Open and then Close_with_condition.
 //
+/** Look up user/host/app in python policyRuleset and give the AMQP Open
+ *  a go-no_go decision. Return false if the mechanics of calling python
+ *  fails. A policy lookup will deny the connection by returning a blank
+ *  usergroup name in the name buffer.
+ * @param[in] policy pointer to policy
+ * @param[in] username authenticated user name
+ * @param[in] hostip numeric host ip address
+ * @param[in] app application name received in remote AMQP Open.hostname
+ * @param[in] conn_name connection name for tracking
+ * @param[out] name_buf pointer to settings name buffer
+ * @param[in] name_buf_size size of settings_buf
+ **/
 bool qd_policy_open_lookup_user(
     qd_policy_t *policy,
     const char *username,
     const char *hostip,
     const char *app,
-    const char *conn_name)
+    const char *conn_name,
+    char       *name_buf,
+    int         name_buf_size,
+    uint64_t    conn_id)
 {
-    // Log the names
-    qd_log(policy->log_source, 
-           POLICY_LOG_LEVEL, 
-           "Policy AMQP Open lookup_user: %s, hostip: %s, app: %s, connection: %s", 
-           username, hostip, app, conn_name);
     qd_python_lock_state_t lock_state = qd_python_lock();
     PyObject *module = PyImport_ImportModule("qpid_dispatch_internal.policy.policy_manager");
     PyObject *lookup_user = module ? PyObject_GetAttrString(module, "policy_lookup_user") : NULL;
     Py_XDECREF(module);
-    PyObject *result = lookup_user ? PyObject_CallFunction(lookup_user, "(Ossss)", (PyObject *)policy->py_policy_manager, username, hostip, app, conn_name) : NULL;
+    PyObject *result = lookup_user ? PyObject_CallFunction(lookup_user, "(OssssK)", 
+                                                           (PyObject *)policy->py_policy_manager, 
+                                                           username, hostip, app, conn_name, conn_id) : NULL;
     Py_XDECREF(lookup_user);
-    if (!result) qd_error_py();
+    if (!result) {
+        qd_python_unlock(lock_state);
+        return false;
+    }
     const char *res_string = PyString_AsString(result);
-
-    qd_log(policy->log_source,
-           POLICY_LOG_LEVEL,
-           "Policy AMQP Open lookup_user result: '%s'", res_string);
+    strncpy(name_buf, res_string, name_buf_size);
     Py_XDECREF(result);
-    
+
     qd_python_unlock(lock_state);
 
+    qd_log(policy->log_source, 
+           POLICY_LOG_LEVEL, 
+           "Policy AMQP Open lookup_user: %s, hostip: %s, app: %s, connection: %s. Usergroup: '%s'", 
+           username, hostip, app, conn_name, name_buf);
+
     return true;
 }
 
+/** Set the error condition and close the connection.
+ * Over the wire this will send an open frame followed
+ * immediately by a close frame with the error condition.
+ * @param[in] conn proton connection being closed
+ * @param[in] cond_name condition name
+ * @param[in] cond_descr condition description
+ **/ 
 void qd_policy_private_deny_amqp_connection(pn_connection_t *conn, const char *cond_name, const char *cond_descr)
 {
-    // Set the error condition and close the connection.
-    // Over the wire this will send an open frame followed
-    // immediately by a close frame with the error condition.
     pn_condition_t * cond = pn_connection_condition(conn);
     (void) pn_condition_set_name(       cond, cond_name);
     (void) pn_condition_set_description(cond, cond_descr);
     pn_connection_close(conn);
 }
 
+
+//
+//
 void qd_policy_amqp_open(void *context, bool discard)
 {
     qd_connection_t *qd_conn = (qd_connection_t *)context;
@@ -260,8 +315,13 @@ void qd_policy_amqp_open(void *context, bool discard)
         const char *hostip = qdpn_connector_hostip(qd_conn->pn_cxtr);
         const char *app = pn_connection_remote_hostname(conn);
         const char *conn_name = qdpn_connector_name(qd_conn->pn_cxtr);
+#define SETTINGS_NAME_SIZE 256
+        char settings_name[SETTINGS_NAME_SIZE];
+        uint32_t conn_id = qd_conn->connection_id;
 
-        if ( qd_policy_open_lookup_user(policy, username, hostip, app, conn_name) ) {
+        if (!policy->enableAccessRules ||
+            (qd_policy_open_lookup_user(policy, username, hostip, app, conn_name, settings_name, SETTINGS_NAME_SIZE, conn_id) &&
+             settings_name[0])) {
             // This connection is allowed.
             if (pn_connection_state(conn) & PN_LOCAL_UNINIT)
                 pn_connection_open(conn);

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/src/policy_private.h
----------------------------------------------------------------------
diff --git a/src/policy_private.h b/src/policy_private.h
index 82ac692..8c3a391 100644
--- a/src/policy_private.h
+++ b/src/policy_private.h
@@ -32,8 +32,34 @@
 
 typedef struct qd_policy_t qd_policy_t;
 
+struct qd_policy__settings_s {
+    int  maxFrameSize;
+    int  maxMessageSize;
+    int  maxSessionWindow;
+    int  maxSessions;
+    int  maxSenders;
+    int  maxReceivers;
+    bool allowDynamicSrc;
+    bool allowAnonymousSender;
+    // char * sources; // TODO:
+    // char * targets; // TODO:
+};
+
+typedef struct qd_policy__settings_s qd_policy_settings_t;
+
+/** Configure the C policy entity from the settings in qdrouterd.conf["policy"]
+ * Called python-to-C during config processing.
+ * @param[in] policy pointer to the policy
+ * @param[in] entity pointer to the managed entity
+ * @return error or not. If error then the policy is freed.
+ **/
 qd_error_t qd_entity_configure_policy(qd_policy_t *policy, qd_entity_t *entity);
 
+/** Memorize the address of python policy_manager object.
+ * This python object gets called by C to execute user lookups 
+ * @param[in] policy pointer to the policy
+ * @param[in] policy_manager the address of the policy_manager object
+ **/
 qd_error_t qd_register_policy_manager(qd_policy_t *policy, void *policy_manager);
 
 
@@ -54,16 +80,17 @@ bool qd_policy_socket_accept(void *context, const char *hostname);
  * Release the counted connection against provisioned limits
  * 
  * @param[in] context the current policy
- * @param[in] name the connector name
+ * @param[in] conn qd_connection
  **/
-void qd_policy_socket_close(void *context, const char *hostname);
+void qd_policy_socket_close(void *context, const qd_connection_t *conn);
 
 
 /** Allow or deny an incoming connection.
  * An Open performative was received over a new connection.
  * Consult local policy to determine if this host/user is
- * allow to make this connection. The underlying proton 
- * connection is either opened or closed.
+ *  allowed to make this connection.
+ * Denied pn_connections are closed with a condition.
+ * Allowed connections are signaled through qd_connection_manager.
  * This function is called from the deferred queue.
  * @param[in] context a qd_connection_t object
  * @param[in] discard callback switch

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/src/server.c
----------------------------------------------------------------------
diff --git a/src/server.c b/src/server.c
index 569faaa..ca0eed7 100644
--- a/src/server.c
+++ b/src/server.c
@@ -266,6 +266,7 @@ static void thread_process_listeners_LH(qd_server_t *qd_server)
         ctx->link_context  = 0;
         ctx->ufd           = 0;
         ctx->connection_id = qd_server->next_connection_id++; // Increment the connection id so the next connection can use it
+        ctx->policy_settings = 0;
         DEQ_INIT(ctx->deferred_calls);
         ctx->deferred_call_lock = sys_mutex();
         ctx->event_stall  = false;
@@ -718,7 +719,7 @@ static void *thread_run(void *arg)
                 DEQ_REMOVE(qd_server->connections, ctx);
 
                 if (!ctx->connector) {
-                    qd_policy_socket_close(qd_server->qd->policy, qdpn_connector_name(cxtr));
+                    qd_policy_socket_close(qd_server->qd->policy, ctx);
                 }
 
                 qdpn_connector_free(cxtr);
@@ -817,6 +818,7 @@ static void cxtr_try_open(void *context)
     ctx->user_context = 0;
     ctx->link_context = 0;
     ctx->ufd          = 0;
+    ctx->policy_settings = 0;
 
     DEQ_INIT(ctx->deferred_calls);
     ctx->deferred_call_lock = sys_mutex();
@@ -1379,6 +1381,7 @@ qd_user_fd_t *qd_user_fd(qd_dispatch_t *qd, int fd, void *context)
     ctx->user_context = 0;
     ctx->link_context = 0;
     ctx->ufd          = ufd;
+    ctx->policy_settings = 0;
     DEQ_INIT(ctx->deferred_calls);
     ctx->deferred_call_lock = sys_mutex();
     ctx->event_stall  = false;

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/src/server_private.h
----------------------------------------------------------------------
diff --git a/src/server_private.h b/src/server_private.h
index e61da61..43706ed 100644
--- a/src/server_private.h
+++ b/src/server_private.h
@@ -99,6 +99,7 @@ struct qd_connection_t {
     void             *link_context; // Context shared by this connection's links
     qd_user_fd_t     *ufd;
     uint64_t         connection_id; // A unique identifier for the qd_connection_t. The underlying pn_connection already has one but it is long and clunky.
+    qd_policy_settings_t *policy_settings;
 
     qd_deferred_call_list_t  deferred_calls;
     sys_mutex_t             *deferred_call_lock;

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/tests/policy-2/test-router-with-policy.json.in
----------------------------------------------------------------------
diff --git a/tests/policy-2/test-router-with-policy.json.in b/tests/policy-2/test-router-with-policy.json.in
index f7b7326..f5d9098 100644
--- a/tests/policy-2/test-router-with-policy.json.in
+++ b/tests/policy-2/test-router-with-policy.json.in
@@ -25,9 +25,10 @@
 	"module": "DEFAULT"
     }],
     ["policy", {
-	"maximumConnections": 2
+	"maximumConnections": 20,
+	"enableAccessRules": "true"
     }],
-  ["policyRuleset", {
+    ["policyRuleset", {
       "applicationName": "photoserver",
       "maxConnections": 50,
       "maxConnPerUser": 5,
@@ -142,6 +143,27 @@
           "targets": "public"
         }
       }
+  }],
+  ["policyRuleset", {
+      "applicationName": "0.0.0.0:21000",
+      "maxConnections": 50,
+      "maxConnPerUser": 5,
+      "maxConnPerHost": 20,
+      "connectionAllowDefault": true,
+      "settings": {
+        "default" : {
+          "maxFrameSize":     222222,
+          "maxMessageSize":   222222,
+          "maxSessionWindow": 222222,
+          "maxSessions":           2,
+          "maxSenders":           22,
+          "maxReceivers":         22,
+          "allowDynamicSrc":      true,
+          "allowAnonymousSender": true,
+          "sources": "public, private, $management",
+          "targets": "public, private, $management"
+        }
+      }
     }
   ]
 ]

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/tests/router_policy_test.py
----------------------------------------------------------------------
diff --git a/tests/router_policy_test.py b/tests/router_policy_test.py
index 5c767c9..9da56eb 100644
--- a/tests/router_policy_test.py
+++ b/tests/router_policy_test.py
@@ -144,7 +144,7 @@ class PolicyFile(TestCase):
         return len(added) == 0 and len(removed) == 0 and len(modified) == 0
 
     def test_policy1_test_zeke_ok(self):
-        p1 = PolicyFile.policy.lookup_user('zeke', '192.168.100.5', 'photoserver', '192.168.100.5:33333')
+        p1 = PolicyFile.policy.lookup_user('zeke', '192.168.100.5', 'photoserver', '192.168.100.5:33333', 1)
         self.assertTrue(p1 == 'test')
         upolicy = {}
         self.assertTrue(
@@ -165,19 +165,19 @@ class PolicyFile(TestCase):
 
     def test_policy1_test_zeke_bad_IP(self):
         self.assertTrue(
-            PolicyFile.policy.lookup_user('zeke', '10.18.0.1',    'photoserver', "connid") == '')
+            PolicyFile.policy.lookup_user('zeke', '10.18.0.1',    'photoserver', "connid", 2) == '')
         self.assertTrue(
-            PolicyFile.policy.lookup_user('zeke', '72.135.2.9',   'photoserver', "connid") == '')
+            PolicyFile.policy.lookup_user('zeke', '72.135.2.9',   'photoserver', "connid", 3) == '')
         self.assertTrue(
-            PolicyFile.policy.lookup_user('zeke', '127.0.0.1',    'photoserver', "connid") == '')
+            PolicyFile.policy.lookup_user('zeke', '127.0.0.1',    'photoserver', "connid", 4) == '')
 
     def test_policy1_test_zeke_bad_app(self):
         self.assertTrue(
-            PolicyFile.policy.lookup_user('zeke', '192.168.100.5','galleria', "connid") == '')
+            PolicyFile.policy.lookup_user('zeke', '192.168.100.5','galleria', "connid", 5) == '')
 
     def test_policy1_test_users_same_permissions(self):
-        zname = PolicyFile.policy.lookup_user('zeke', '192.168.100.5', 'photoserver', '192.168.100.5:33333')
-        yname = PolicyFile.policy.lookup_user('ynot', '10.48.255.254', 'photoserver', '192.168.100.5:33334')
+        zname = PolicyFile.policy.lookup_user('zeke', '192.168.100.5', 'photoserver', '192.168.100.5:33333', 6)
+        yname = PolicyFile.policy.lookup_user('ynot', '10.48.255.254', 'photoserver', '192.168.100.5:33334', 7)
         self.assertTrue( zname == yname )
 
     def test_policy1_lookup_unknown_application(self):

http://git-wip-us.apache.org/repos/asf/qpid-dispatch/blob/b9c73771/tests/system_tests_policy.py
----------------------------------------------------------------------
diff --git a/tests/system_tests_policy.py b/tests/system_tests_policy.py
index 74f93c4..30f1158 100644
--- a/tests/system_tests_policy.py
+++ b/tests/system_tests_policy.py
@@ -42,7 +42,7 @@ class AbsoluteConnectionCountLimit(TestCase):
     def address(self):
         return self.router.addresses[0]
 
-    def test_aaa_verify_maximum_connections(self):
+    def test_verify_maximum_connections(self):
         addr = self.address()
 
         # two connections should be ok


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@qpid.apache.org
For additional commands, e-mail: commits-help@qpid.apache.org