You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by cc...@apache.org on 2008/07/31 22:45:44 UTC

svn commit: r681479 - in /incubator/qpid/trunk/qpid/cpp/src: Makefile.am qpid/acl/ qpid/acl/Acl.cpp qpid/acl/Acl.h qpid/acl/AclPlugin.cpp qpid/broker/Broker.cpp qpid/broker/Broker.h qpid/broker/SessionAdapter.cpp

Author: cctrieloff
Date: Thu Jul 31 13:45:44 2008
New Revision: 681479

URL: http://svn.apache.org/viewvc?rev=681479&view=rev
Log:

- Implementation of ACL plugin
- Apply ACL to Exchange, Queue, Binding, Subscribe
- Follow Java ACL types, few added

To complete the implementation of ACL the following items are remaining.
- ACL on message transfer
- ACL on MGNT commands
- Reading ACL from File, into auth map.



Added:
    incubator/qpid/trunk/qpid/cpp/src/qpid/acl/
    incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/acl/AclPlugin.cpp
Modified:
    incubator/qpid/trunk/qpid/cpp/src/Makefile.am
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionAdapter.cpp

Modified: incubator/qpid/trunk/qpid/cpp/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/Makefile.am?rev=681479&r1=681478&r2=681479&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/Makefile.am (original)
+++ incubator/qpid/trunk/qpid/cpp/src/Makefile.am Thu Jul 31 13:45:44 2008
@@ -103,6 +103,7 @@
 lib_LTLIBRARIES = libqpidcommon.la libqpidbroker.la libqpidclient.la 
 
 include cluster.mk
+include acl.mk
 
 # The logger library uses boost::date_time to format time.
 # We have to disable the unused parameters warning to get around
@@ -377,6 +378,7 @@
   qpid/agent/ManagementAgent.h \
   qpid/agent/ManagementAgentImpl.h \
   qpid/broker/Broker.h \
+  qpid/broker/AclModule.h \
   qpid/broker/SessionAdapter.h \
   qpid/broker/Exchange.h \
   qpid/broker/Queue.h \

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.cpp?rev=681479&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.cpp Thu Jul 31 13:45:44 2008
@@ -0,0 +1,105 @@
+/*
+ *
+ * Copyright (c) 2006 The Apache Software Foundation
+ *
+ * Licensed 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.
+ *
+ */
+
+#include "Acl.h"
+
+
+#include "qpid/broker/Broker.h"
+#include "qpid/cluster/Cluster.h"
+#include "qpid/Plugin.h"
+#include "qpid/Options.h"
+#include "qpid/shared_ptr.h"
+
+#include <boost/utility/in_place_factory.hpp>
+
+namespace qpid {
+namespace acl {
+
+using namespace std;
+
+   Acl::Acl (AclValues& av, broker::Broker& b): aclValues(av), broker(&b)
+   {
+       if (!readAclFile()) throw Exception("Could not read ACL file");
+	   QPID_LOG(info, "ACL Plugin loaded");
+
+   }
+
+   std::string Acl::printAction(acl::Action action)
+   {
+      switch (action)
+	  {
+	   case CONSUME: return "Consume";
+	   case PUBLISH: return "Publish";
+	   case CREATE: return "Create";
+	   case ACCESS: return "Access";
+	   case BIND: return "Bind";
+	   case UNBIND: return "Unbind";
+	   case DELETE: return "Delete";
+	   case PURGE: return "Purge";
+	   default: return "Unknown";
+	  }
+   }
+   
+   std::string Acl::printObjType(acl::ObjectType objType)
+   {
+      switch (objType)
+	  {
+      case QUEUE: return "Queue";
+	  case EXCHANGE: return "Exchnage";
+	  case ROUTINGKEY: return "RoutingKey";
+	  case SESSION: return "Session";
+	  default: return "Unknown";
+	  }
+   }
+
+   bool Acl::authorise(std::string id, acl::Action action, acl::ObjectType objType, std::string name, std::map<std::string, std::string>*
+   /*params*/)
+   {
+      if (aclValues.noEnforce) return true;
+   
+      // add real ACL check here... 
+      AclResult aclreslt = ALLOWLOG;  // hack to test, set based on real decision.
+	  
+	  switch (aclreslt)
+	  {
+	  case ALLOWLOG:
+          QPID_LOG(info, "ACL Allow log id:" << id <<" action:" << printAction(action) << " ObjectType:" << printObjType(objType) << " Name:" << name );  
+	  case ALLOW:
+	      return true;
+	  case DENYNOLOG:
+	      return false;
+	  case DENY:
+	  default:
+	      QPID_LOG(info, "ACL Deny id:" << id << " action:" << printAction(action) << " ObjectType:" << printObjType(objType) << " Name:" << name);  
+	      return false;
+	  }
+   
+      return false;  
+   }
+   
+   bool Acl::readAclFile()
+   {
+   
+      return true;
+   }
+
+   Acl::~Acl(){}
+
+
+    
+}} // namespace qpid::acl

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.h?rev=681479&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.h (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/acl/Acl.h Thu Jul 31 13:45:44 2008
@@ -0,0 +1,75 @@
+#ifndef QPID_ACL_ACL_H
+#define QPID_ACL_ACL_H
+
+
+/*
+ *
+ * Copyright (c) 2006 The Apache Software Foundation
+ *
+ * Licensed 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.
+ *
+ */
+
+
+
+#include "qpid/shared_ptr.h"
+#include "qpid/RefCounted.h"
+#include "qpid/broker/AclModule.h"
+#include <map>
+#include <string>
+
+
+namespace qpid {
+namespace broker {
+class Broker;
+}
+
+namespace acl {
+
+struct AclValues {
+    public:
+	bool noEnforce;
+    std::string aclFile;
+
+    AclValues() {noEnforce = false; aclFile = "policy.acl"; }
+};
+
+
+class Acl : public broker::AclModule, public RefCounted 
+{
+
+public:
+   Acl (AclValues& av, broker::Broker& b);
+
+   void initialize();
+   
+   virtual bool authorise(std::string id, acl::Action action, acl::ObjectType objType, std::string name, std::map<std::string, std::string>* params);
+   // create specilied authorise methods for cases that need faster matching as needed.
+
+   virtual ~Acl();
+private:
+   std::string printAction(acl::Action action);
+   std::string printObjType(acl::ObjectType objType);
+
+   acl::AclValues aclValues;
+   broker::Broker* broker;
+   
+   bool readAclFile();
+      
+};
+
+
+    
+}} // namespace qpid::acl
+
+#endif // QPID_ACL_ACL_H

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/acl/AclPlugin.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/acl/AclPlugin.cpp?rev=681479&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/acl/AclPlugin.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/acl/AclPlugin.cpp Thu Jul 31 13:45:44 2008
@@ -0,0 +1,89 @@
+/*
+ *
+ * Copyright (c) 2006 The Apache Software Foundation
+ *
+ * Licensed 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.
+ *
+ */
+
+#include "qpid/acl/Acl.h"
+#include "qpid/broker/Broker.h"
+#include "qpid/Plugin.h"
+#include "qpid/Options.h"
+#include "qpid/shared_ptr.h"
+#include "qpid/log/Statement.h"
+
+#include <boost/utility/in_place_factory.hpp>
+
+namespace qpid {
+namespace acl {
+
+using namespace std;
+
+/** Note separating options from values to work around boost version differences.
+ *  Old boost takes a reference to options objects, but new boost makes a copy.
+ *  New boost allows a shared_ptr but that's not compatible with old boost.
+ */
+struct AclOptions : public Options {
+    AclValues& values; 
+
+    AclOptions(AclValues& v) : Options("ACL Options"), values(v) {
+        addOptions()
+            ("no-enforce-acl", optValue(values.noEnforce), "Do not enforce ACL")
+            ("acl-file", optValue(values.aclFile, "FILE"), "The policy file to load from, loaded from data dir")
+            ;
+    }
+};
+
+struct AclPlugin : public Plugin {
+
+    AclValues values;
+    AclOptions options;
+    boost::intrusive_ptr<Acl> acl;
+    
+    AclPlugin() : options(values) {}
+
+    Options* getOptions() { return &options; }
+
+    void init(broker::Broker& b) {
+        if (values.noEnforce){
+		    QPID_LOG(info, "ACL Disabled, no ACL checking being done.");
+			return;  
+		}
+        if (acl) throw Exception("ACL plugin cannot be initialized twice in one process.");
+        acl = new Acl(values, b);
+		b.setAcl(acl.get());
+        b.addFinalizer(boost::bind(&AclPlugin::shutdown, this));
+    }
+
+    template <class T> bool init(Plugin::Target& target) {
+        T* t = dynamic_cast<T*>(&target);
+        if (t) init(*t);
+        return t;
+    }
+
+    void earlyInitialize(Plugin::Target&) {}
+
+    void initialize(Plugin::Target& target) {
+        init<broker::Broker>(target);
+    }
+
+    void shutdown() { acl = 0; }
+};
+
+static AclPlugin instance; // Static initialization.
+
+// For test purposes.
+boost::intrusive_ptr<Acl> getGlobalAcl() { return instance.acl; }
+    
+}} // namespace qpid::acl

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.cpp?rev=681479&r1=681478&r2=681479&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.cpp Thu Jul 31 13:45:44 2008
@@ -128,6 +128,7 @@
     config(conf),
     managementAgentSingleton(!config.enableMgmt),
     store(0),
+	acl(0),
     dataDir(conf.noDataDir ? std::string () : conf.dataDir),
     links(this),
     factory(*this),

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.h?rev=681479&r1=681478&r2=681479&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/Broker.h Thu Jul 31 13:45:44 2008
@@ -45,6 +45,7 @@
 #include "qpid/framing/ProtocolInitiation.h"
 #include "qpid/sys/Runnable.h"
 #include "qpid/RefCounted.h"
+#include "AclModule.h"
 
 #include <boost/intrusive_ptr.hpp>
 #include <vector>
@@ -88,7 +89,33 @@
         size_t replayHardLimit;
         uint queueLimit;
     };
-    
+ 
+  private:
+    boost::shared_ptr<sys::Poller> poller;
+    Options config;
+    management::ManagementAgent::Singleton managementAgentSingleton;
+    std::vector< boost::shared_ptr<sys::ProtocolFactory> > protocolFactories;
+    MessageStore* store;
+	AclModule* acl;
+    DataDir dataDir;
+
+    QueueRegistry queues;
+    ExchangeRegistry exchanges;
+    LinkRegistry links;
+    ConnectionFactory factory;
+    DtxManager dtxManager;
+    SessionManager sessionManager;
+    management::ManagementAgent* managementAgent;
+    management::Broker*          mgmtObject;
+    Vhost::shared_ptr            vhostObject;
+    System::shared_ptr           systemObject;
+
+    void declareStandardExchange(const std::string& name, const std::string& type);
+
+
+  public:
+
+  
     virtual ~Broker();
 
     Broker(const Options& configuration);
@@ -114,6 +141,8 @@
 
     void setStore (MessageStore*);
     MessageStore& getStore() { return *store; }
+    void setAcl (AclModule* _acl) {acl = _acl;}
+    AclModule* getAcl() { return acl; }
     QueueRegistry& getQueues() { return queues; }
     ExchangeRegistry& getExchanges() { return exchanges; }
     LinkRegistry& getLinks() { return links; }
@@ -147,26 +176,6 @@
     // For the present just return the first ProtocolFactory registered.
     boost::shared_ptr<sys::ProtocolFactory> getProtocolFactory() const;
 
-  private:
-    boost::shared_ptr<sys::Poller> poller;
-    Options config;
-    management::ManagementAgent::Singleton managementAgentSingleton;
-    std::vector< boost::shared_ptr<sys::ProtocolFactory> > protocolFactories;
-    MessageStore* store;
-    DataDir dataDir;
-
-    QueueRegistry queues;
-    ExchangeRegistry exchanges;
-    LinkRegistry links;
-    ConnectionFactory factory;
-    DtxManager dtxManager;
-    SessionManager sessionManager;
-    management::ManagementAgent* managementAgent;
-    management::Broker*          mgmtObject;
-    Vhost::shared_ptr            vhostObject;
-    System::shared_ptr           systemObject;
-
-    void declareStandardExchange(const std::string& name, const std::string& type);
 };
 
 }}

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionAdapter.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionAdapter.cpp?rev=681479&r1=681478&r2=681479&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionAdapter.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/broker/SessionAdapter.cpp Thu Jul 31 13:45:44 2008
@@ -53,6 +53,18 @@
                                                   const string& alternateExchange, 
                                                   bool passive, bool durable, bool /*autoDelete*/, const FieldTable& args){
 
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+        std::map<std::string, std::string> params;
+		params.insert(make_pair("TYPE", type));
+		params.insert(make_pair("ALT", alternateExchange));
+		params.insert(make_pair("PAS", std::string(passive ? "Y" : "N") ));
+		params.insert(make_pair("DURA", std::string(durable ? "Y" : "N")));
+	    if (!acl->authorise(getConnection().getUserId(),acl::CREATE,acl::EXCHANGE,exchange,&params) )
+	        throw NotAllowedException("ACL denied exhange declare request");
+    }
+	
     //TODO: implement autoDelete
     Exchange::shared_ptr alternate;
     if (!alternateExchange.empty()) {
@@ -100,6 +112,15 @@
 }
                 
 void SessionAdapter::ExchangeHandlerImpl::delete_(const string& name, bool /*ifUnused*/){
+
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+	    if (!acl->authorise(getConnection().getUserId(),acl::DELETE,acl::EXCHANGE,name,NULL) )
+	        throw NotAllowedException("ACL denied exhange delete request");
+    }
+
+
     //TODO: implement unused
     Exchange::shared_ptr exchange(getBroker().getExchanges().get(name));
     if (exchange->inUseAsAlternate()) throw NotAllowedException(QPID_MSG("Exchange in use as alternate-exchange."));
@@ -110,6 +131,14 @@
 
 ExchangeQueryResult SessionAdapter::ExchangeHandlerImpl::query(const string& name)
 {
+
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+	    if (!acl->authorise(getConnection().getUserId(),acl::ACCESS,acl::EXCHANGE,name,NULL) )
+	        throw NotAllowedException("ACL denied exhange query request");
+    }
+
     try {
         Exchange::shared_ptr exchange(getBroker().getExchanges().get(name));
         return ExchangeQueryResult(exchange->getType(), exchange->isDurable(), false, exchange->getArgs());
@@ -121,6 +150,15 @@
                                            const string& exchangeName, const string& routingKey, 
                                            const FieldTable& arguments){
 
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+        std::map<std::string, std::string> params;
+		params.insert(make_pair("RKEY", routingKey));
+	    if (!acl->authorise(getConnection().getUserId(),acl::BIND,acl::EXCHANGE,exchangeName,&params) )
+	        throw NotAllowedException("ACL denied exhange bind request");
+    }
+
     Queue::shared_ptr queue = getQueue(queueName);
     Exchange::shared_ptr exchange = getBroker().getExchanges().get(exchangeName);
     if(exchange){
@@ -142,6 +180,18 @@
                                         const string& exchangeName,
                                         const string& routingKey)
 {
+
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+        std::map<std::string, std::string> params;
+		params.insert(make_pair("QN", queueName));
+		params.insert(make_pair("RKEY", routingKey));
+	    if (!acl->authorise(getConnection().getUserId(),acl::UNBIND,acl::EXCHANGE,exchangeName,&params) )
+	        throw NotAllowedException("ACL denied exchange unbind request");
+    }
+
+
     Queue::shared_ptr queue = getQueue(queueName);
     if (!queue.get()) throw NotFoundException("Unbind failed. No such exchange: " + exchangeName);
 
@@ -160,6 +210,16 @@
                                                                   const std::string& key,
                                                                   const framing::FieldTable& args)
 {
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+        std::map<std::string, std::string> params;
+		params.insert(make_pair("QUEUE", queueName));
+		params.insert(make_pair("RKEY", queueName));
+	    if (!acl->authorise(getConnection().getUserId(),acl::CREATE,acl::EXCHANGE,exchangeName,&params) )
+	        throw NotAllowedException("ACL denied exhange bound request");
+    }
+	
     Exchange::shared_ptr exchange;
     try {
         exchange = getBroker().getExchanges().get(exchangeName);
@@ -220,6 +280,13 @@
 
 QueueQueryResult SessionAdapter::QueueHandlerImpl::query(const string& name)
 {
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+	    if (!acl->authorise(getConnection().getUserId(),acl::ACCESS,acl::QUEUE,name,NULL) )
+	        throw NotAllowedException("ACL denied queue query request");
+    }
+	
     Queue::shared_ptr queue = session.getBroker().getQueues().find(name);
     if (queue) {
 
@@ -242,6 +309,19 @@
                                               bool passive, bool durable, bool exclusive, 
                                               bool autoDelete, const qpid::framing::FieldTable& arguments){
  
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+        std::map<std::string, std::string> params;
+		params.insert(make_pair("ALT", alternateExchange));
+		params.insert(make_pair("PAS", std::string(passive ? "Y" : "N") ));
+		params.insert(make_pair("DURA", std::string(durable ? "Y" : "N")));
+		params.insert(make_pair("EXCLUS", std::string(exclusive ? "Y" : "N")));
+		params.insert(make_pair("AUTOD", std::string(autoDelete ? "Y" : "N")));
+	    if (!acl->authorise(getConnection().getUserId(),acl::CREATE,acl::QUEUE,name,&params) )
+	        throw NotAllowedException("ACL denied queue create request");
+    }
+
     Exchange::shared_ptr alternate;
     if (!alternateExchange.empty()) {
         alternate = getBroker().getExchanges().get(alternateExchange);
@@ -289,10 +369,24 @@
         
         
 void SessionAdapter::QueueHandlerImpl::purge(const string& queue){
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+ 	    if (!acl->authorise(getConnection().getUserId(),acl::DELETE,acl::QUEUE,queue,NULL) )
+	        throw NotAllowedException("ACL denied queue purge request");
+    }
     getQueue(queue)->purge();
 } 
         
 void SessionAdapter::QueueHandlerImpl::delete_(const string& queue, bool ifUnused, bool ifEmpty){
+
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+ 	    if (!acl->authorise(getConnection().getUserId(),acl::DELETE,acl::QUEUE,queue,NULL) )
+	        throw NotAllowedException("ACL denied queue delete request");
+    }
+
     ChannelException error(0, "");
     Queue::shared_ptr q = getQueue(queue);
     if(ifEmpty && q->getMessageCount() > 0){
@@ -347,6 +441,15 @@
                               uint64_t /*resumeTtl*/,
                               const FieldTable& arguments)
 {
+
+	AclModule* acl = getBroker().getAcl();
+	if (acl)
+	{
+	    // add flags as needed
+ 	    if (!acl->authorise(getConnection().getUserId(),acl::CONSUME,acl::QUEUE,queueName,NULL) )
+	        throw NotAllowedException("ACL denied Session subscribe request");
+    }
+
     Queue::shared_ptr queue = getQueue(queueName);
     if(!destination.empty() && state.exists(destination))
         throw NotAllowedException(QPID_MSG("Consumer tags must be unique"));