You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by ac...@apache.org on 2008/11/18 20:55:59 UTC

svn commit: r718693 - in /incubator/qpid/trunk/qpid/cpp: ./ src/ src/qpid/cluster/ src/tests/

Author: aconway
Date: Tue Nov 18 11:55:59 2008
New Revision: 718693

URL: http://svn.apache.org/viewvc?rev=718693&view=rev
Log:
Optional cluster integration with cman quorum service.

Added:
    incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum.h   (with props)
    incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.cpp   (with props)
    incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.h   (with props)
    incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_null.h   (with props)
Modified:
    incubator/qpid/trunk/qpid/cpp/configure.ac
    incubator/qpid/trunk/qpid/cpp/src/cluster.mk
    incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp
    incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.h
    incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/OutputInterceptor.cpp
    incubator/qpid/trunk/qpid/cpp/src/tests/cluster.mk
    incubator/qpid/trunk/qpid/cpp/src/tests/ssl_test

Modified: incubator/qpid/trunk/qpid/cpp/configure.ac
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/configure.ac?rev=718693&r1=718692&r2=718693&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/configure.ac (original)
+++ incubator/qpid/trunk/qpid/cpp/configure.ac Tue Nov 18 11:55:59 2008
@@ -166,7 +166,7 @@
 # Link with whichever variant of libboost_thread is available.
 AC_SEARCH_LIBS([_ZN5boost6thread4joinEv],[boost_thread-mt boost_thread],,[AC_MSG_ERROR([No boost thread library found])])
 
-# Check for optional CPG requirement.
+# Check for optional cluster requirements.
 tmp_LIBS=$LIBS
 LDFLAGS="$LDFLAGS -L/usr/lib/openais -L/usr/lib64/openais"
 AC_ARG_WITH([cpg],  
@@ -180,16 +180,34 @@
     no) with_CPG=no ;; 
     *) AC_MSG_ERROR([Bad value ${withval} for --with-cpg option]) ;;
    esac],
-   [		   # not specified - default no
-    with_CPG=no
+   [		   # not specified - use if present
+    with_CPG=yes
+    AC_CHECK_LIB([cpg],[cpg_local_get],,[with_CPG=no])
+    AC_CHECK_HEADERS([openais/cpg.h],,[with_CPG=no])
    ]
 )
-LIBS=$tmp_LIBS
+AM_CONDITIONAL([HAVE_LIBCPG], [test x$with_CPG = xyes])
 
-AM_CONDITIONAL([CPG], [test x$with_CPG = xyes])
-if test x$with_CPG = xyes; then
-  CPPFLAGS+=" -DCPG"
-fi
+AC_ARG_WITH([libcman],  
+  [AS_HELP_STRING([--with-libcman], [Integration with libcman quorum service.])],
+  [case "${withval}" in
+    yes)			# yes - enable
+    with_LIBCMAN=yes
+    AC_CHECK_LIB([cman],[cman_is_quorate],,[AC_MSG_ERROR([libcman not found, install libcman package.])])
+    AC_CHECK_HEADERS([libcman.h],,[AC_MSG_ERROR([libcman.h not found, install cman-devel])])
+    ;;
+    no) with_LIBCMAN=no ;; 
+    *) AC_MSG_ERROR([Bad value ${withval} for --with-libcman option]) ;;
+   esac],
+   [		   # not specified - use if present and with_CPG=yes
+    with_LIBCMAN=$with_CPG
+    AC_CHECK_LIB([cman],[cman_is_quorate],,with_LIBCMAN=no)
+    AC_CHECK_HEADERS([libcman.h],,with_LIBCMAN=no)
+   ]
+)
+AM_CONDITIONAL([HAVE_LIBCMAN], [test x$with_LIBCMAN = xyes])
+
+LIBS=$tmp_LIBS
 
 # Setup --with-sasl/--without-sasl as arguments to configure
 AC_ARG_WITH([sasl],

Modified: incubator/qpid/trunk/qpid/cpp/src/cluster.mk
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/cluster.mk?rev=718693&r1=718692&r2=718693&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/cluster.mk (original)
+++ incubator/qpid/trunk/qpid/cpp/src/cluster.mk Tue Nov 18 11:55:59 2008
@@ -1,11 +1,20 @@
 #
 # Cluster library makefile fragment, to be included in Makefile.am
 # 
-if CPG
+# Optional CMAN support
+if HAVE_LIBCMAN
+CMAN_SOURCES= qpid/cluster/Quorum_cman.h qpid/cluster/Quorum_cman.cpp
+libcman = -lcman
+else
+CMAN_SOURCES= qpid/cluster/Quorum_null.h
+endif
+
+if HAVE_LIBCPG
 
 dmodule_LTLIBRARIES += cluster.la
 
 cluster_la_SOURCES = \
+  $(CMAN_SOURCES) \
   qpid/cluster/types.h \
   qpid/cluster/Cluster.cpp \
   qpid/cluster/Cluster.h \
@@ -32,9 +41,7 @@
   qpid/cluster/FailoverExchange.h \
   qpid/cluster/FailoverExchange.cpp
 
-cluster_la_LIBADD= -lcpg libqpidbroker.la libqpidclient.la
-
+cluster_la_LIBADD=  -lcpg $(libcman) libqpidbroker.la libqpidclient.la
 cluster_la_LDFLAGS = $(PLUGINLDFLAGS)
 
-endif
-
+endif				# HAVE_LIBCPG

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp?rev=718693&r1=718692&r2=718693&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.cpp Tue Nov 18 11:55:59 2008
@@ -86,6 +86,7 @@
 };
 
 Cluster::Cluster(const std::string& name_, const Url& url_, broker::Broker& b) :
+    isQuorate(isQuorateImpl),
     broker(b),
     poller(b.getPoller()),
     cpg(*this),
@@ -591,6 +592,17 @@
     return broker; // Immutable,  no need to lock.
 }
 
+/** Default implementation for isQuorateImpl when there is no quorum service. */
+bool Cluster::isQuorateImpl() { return true; }
+
+void Cluster::checkQuorum() {
+    if (!isQuorate()) {
+        QPID_LOG(critical, *this << " disconnected from cluster quorum, shutting down");
+        leave();
+        throw Exception(QPID_MSG(*this << " disconnected from cluster quorum."));
+    }
+}
+
 void Cluster::setClusterId(const Uuid& uuid) {
     clusterId = uuid;
     if (mgmtObject)

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.h?rev=718693&r1=718692&r2=718693&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.h (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Cluster.h Tue Nov 18 11:55:59 2008
@@ -92,6 +92,9 @@
     MemberId getId() const;
     broker::Broker& getBroker() const;
 
+    boost::function<bool ()> isQuorate;
+    void checkQuorum();
+    
   private:
     typedef sys::LockPtr<Cluster,sys::Monitor> LockPtr;
     typedef sys::LockPtr<const Cluster,sys::Monitor> ConstLockPtr;
@@ -173,7 +176,8 @@
     void dumpOutDone(Lock&);
 
     void setClusterId(const framing::Uuid&);
-    
+    static bool isQuorateImpl();
+
     mutable sys::Monitor lock;
 
     broker::Broker& broker;

Modified: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/OutputInterceptor.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/OutputInterceptor.cpp?rev=718693&r1=718692&r2=718693&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/OutputInterceptor.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/OutputInterceptor.cpp Tue Nov 18 11:55:59 2008
@@ -37,6 +37,7 @@
 {}
 
 void OutputInterceptor::send(framing::AMQFrame& f) {
+    parent.getCluster().checkQuorum();
     Locker l(lock); 
     next->send(f);
     if (!parent.isCatchUp())

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum.h?rev=718693&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum.h (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum.h Tue Nov 18 11:55:59 2008
@@ -0,0 +1,32 @@
+#ifndef QPID_CLUSTER_QUORUM_H
+#define QPID_CLUSTER_QUORUM_H
+
+/*
+ *
+ * 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.
+ *
+ */
+#include "config.h"
+
+#if HAVE_LIBCMAN
+#include "Quorum_cman.h"
+#else
+#include "Quorum_null.h"
+#endif
+
+#endif  /*!QPID_CLUSTER_QUORUM_H*/

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum.h
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum.h
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.cpp?rev=718693&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.cpp Tue Nov 18 11:55:59 2008
@@ -0,0 +1,53 @@
+/*
+ *
+ * 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.
+ *
+ */
+#include "Quorum.h"
+#include "qpid/log/Statement.h"
+#include "qpid/Options.h"
+#include "qpid/sys/Time.h"
+
+namespace qpid {
+namespace cluster {
+
+Quorum::Quorum() : enable(false), cman(0) {}
+
+Quorum::~Quorum() { if (cman) cman_finish(cman); }
+
+void Quorum::addOption(Options& opts) {
+    opts.addOptions()("cluster-cman", optValue(enable), "Enable integration with CMAN Cluster Manager");
+}
+
+void Quorum::init() {
+    if (enable) {
+        cman = cman_init(0);
+        if (cman == 0) throw ErrnoException("Can't connect to cman service");
+        // FIXME aconway 2008-11-13: configure max wait.
+        for (int retry = 0;  !cman_is_quorate(cman) && retry < 30; retry++) {
+            QPID_LOG(notice, "Waiting for cluster quorum: " << sys::strError(errno));
+            sys::sleep(1);
+        }
+        if (!cman_is_quorate(cman))
+            throw ErrnoException("Timed out waiting for cluster quorum");
+    }
+}
+
+bool Quorum::isQuorate() { return cman_is_quorate(cman); }
+
+}} // namespace qpid::cluster

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.cpp
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.h?rev=718693&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.h (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.h Tue Nov 18 11:55:59 2008
@@ -0,0 +1,53 @@
+#ifndef QPID_CLUSTER_QUORUM_CMAN_H
+#define QPID_CLUSTER_QUORUM_CMAN_H
+
+/*
+ *
+ * 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.
+ *
+ */
+
+extern "C" {
+#include <libcman.h>
+}
+
+namespace qpid {
+
+class Options;
+
+namespace cluster {
+
+class Quorum {
+  public:
+    Quorum();
+    ~Quorum();
+    void addOption(Options& opts);
+    void init();
+    bool isQuorate();
+    
+  private:
+    bool enable;
+    cman_handle_t cman;
+};
+
+
+}} // namespace qpid::cluster
+
+ // namespace qpid::cluster
+
+#endif  /*!QPID_CLUSTER_QUORUM_CMAN_H*/

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.h
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_cman.h
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_null.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_null.h?rev=718693&view=auto
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_null.h (added)
+++ incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_null.h Tue Nov 18 11:55:59 2008
@@ -0,0 +1,39 @@
+#ifndef QPID_CLUSTER_QUORUM_NULL_H
+#define QPID_CLUSTER_QUORUM_NULL_H
+
+/*
+ *
+ * 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.
+ *
+ */
+namespace qpid {
+namespace cluster {
+
+/** Null implementation of quorum. */
+
+class Quorum {
+  public:
+    void init();
+    bool isQuorate() { return true; }
+    void addOption(Options& opts) {}
+};
+
+#endif
+
+
+#endif  /*!QPID_CLUSTER_QUORUM_NULL_H*/

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_null.h
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/qpid/trunk/qpid/cpp/src/qpid/cluster/Quorum_null.h
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/cluster.mk
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/cluster.mk?rev=718693&r1=718692&r2=718693&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/cluster.mk (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/cluster.mk Tue Nov 18 11:55:59 2008
@@ -1,10 +1,10 @@
-if CPG
+
+if HAVE_LIBCPG
+
 #
 # Cluster tests makefile fragment, to be included in Makefile.am
 # 
 
-lib_cluster = $(abs_builddir)/../cluster.la
-
 # NOTE: Programs using the openais library must be run with gid=ais
 # You should do "newgrp ais" before running the tests to run these.
 # 
@@ -16,8 +16,8 @@
 
 check_PROGRAMS+=cluster_test
 cluster_test_SOURCES=unit_test.cpp cluster_test.cpp
-cluster_test_LDADD=$(lib_client) $(lib_cluster) -lboost_unit_test_framework
+cluster_test_LDADD=$(lib_client) ../cluster.la -lboost_unit_test_framework
 
-unit_test_LDADD+=$(lib_cluster)
+unit_test_LDADD+=../cluster.la
 
 endif

Modified: incubator/qpid/trunk/qpid/cpp/src/tests/ssl_test
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/src/tests/ssl_test?rev=718693&r1=718692&r2=718693&view=diff
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/src/tests/ssl_test (original)
+++ incubator/qpid/trunk/qpid/cpp/src/tests/ssl_test Tue Nov 18 11:55:59 2008
@@ -19,8 +19,7 @@
 }
 
 start_broker() {
-    ../qpidd --daemon --transport ssl --port 0 --ssl-port 0 --no-data-dir --no-module-dir --auth no --config $CONFIG\
-        --load-module ../.libs/ssl.so --ssl-cert-db $CERT_DIR --ssl-cert-password-file $CERT_PW_FILE > qpidd.port
+    ../qpidd --daemon --transport ssl --port 0 --ssl-port 0 --no-data-dir --no-module-dir --auth no --config $CONFIG --load-module ../.libs/ssl.so --ssl-cert-db $CERT_DIR --ssl-cert-password-file $CERT_PW_FILE > qpidd.port
     PORT=`cat qpidd.port`
 }