You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@qpid.apache.org by gs...@apache.org on 2006/10/06 18:17:08 UTC

svn commit: r453657 - in /incubator/qpid/trunk/qpid/cpp: broker/inc/ broker/src/ broker/test/ client/inc/ client/src/ common/framing/generated/stylesheets/

Author: gsim
Date: Fri Oct  6 09:17:06 2006
New Revision: 453657

URL: http://svn.apache.org/viewvc?view=rev&rev=453657
Log:
Decoupled routing from the channel and message classes.


Added:
    incubator/qpid/trunk/qpid/cpp/broker/inc/Router.h   (with props)
    incubator/qpid/trunk/qpid/cpp/broker/src/Router.cpp   (with props)
    incubator/qpid/trunk/qpid/cpp/broker/test/ChannelTest.cpp   (with props)
    incubator/qpid/trunk/qpid/cpp/broker/test/RouterTest.cpp   (with props)
Modified:
    incubator/qpid/trunk/qpid/cpp/broker/inc/Channel.h
    incubator/qpid/trunk/qpid/cpp/broker/inc/Message.h
    incubator/qpid/trunk/qpid/cpp/broker/src/Channel.cpp
    incubator/qpid/trunk/qpid/cpp/broker/src/Message.cpp
    incubator/qpid/trunk/qpid/cpp/broker/src/SessionHandlerImpl.cpp
    incubator/qpid/trunk/qpid/cpp/client/inc/IncomingMessage.h
    incubator/qpid/trunk/qpid/cpp/client/src/IncomingMessage.cpp
    incubator/qpid/trunk/qpid/cpp/common/framing/generated/stylesheets/utils.xsl

Modified: incubator/qpid/trunk/qpid/cpp/broker/inc/Channel.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/broker/inc/Channel.h?view=diff&rev=453657&r1=453656&r2=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/broker/inc/Channel.h (original)
+++ incubator/qpid/trunk/qpid/cpp/broker/inc/Channel.h Fri Oct  6 09:17:06 2006
@@ -33,6 +33,10 @@
 
 namespace qpid {
     namespace broker {
+        /**
+         * Maintains state for an AMQP channel. Handles incoming and
+         * outgoing messages for that channel.
+         */
         class Channel{
         private:
             class ConsumerImpl : public virtual Consumer{
@@ -98,7 +102,15 @@
             qpid::concurrent::MonitorImpl deliveryLock;
 
             void deliver(Message::shared_ptr& msg, string& tag, Queue::shared_ptr& queue, bool ackExpected);            
-            void publish(ExchangeRegistry* exchanges);
+            void checkMessage(const std::string& text);
+
+            template<class Operation> void processMessage(Operation route){
+                if(message->isComplete()){
+                    route(message);
+                    message.reset();
+                }
+            }
+
         
         public:
             Channel(qpid::framing::OutputHandler* out, int id, u_int32_t framesize);
@@ -107,9 +119,6 @@
             inline Queue::shared_ptr getDefaultQueue(){ return defaultQueue; }
             inline u_int32_t setPrefetchSize(u_int32_t size){ prefetchSize = size; }
             inline u_int16_t setPrefetchCount(u_int16_t count){ prefetchCount = count; }
-            void handlePublish(Message* msg);
-            void handleHeader(qpid::framing::AMQHeaderBody::shared_ptr header, ExchangeRegistry* exchanges);
-            void handleContent(qpid::framing::AMQContentBody::shared_ptr content, ExchangeRegistry* exchanges);
             bool exists(string& consumerTag);
             void consume(string& tag, Queue::shared_ptr queue, bool acks, bool exclusive, ConnectionToken* const connection = 0);
             void cancel(string& tag);
@@ -119,6 +128,37 @@
             void rollback();
             void ack(u_int64_t deliveryTag, bool multiple);
             void recover(bool requeue);
+
+            /**
+             * Handles the initial publish request though a
+             * channel. The header and (if applicable) content will be
+             * accumulated through calls to handleHeader() and
+             * handleContent()
+             */
+            void handlePublish(Message* msg);
+
+            /**
+             * A template method that handles a received header and if
+             * there is no content routes it using the functor passed
+             * in.
+             */
+            template<class Operation> void handleHeader(qpid::framing::AMQHeaderBody::shared_ptr header, Operation route){
+                checkMessage("Invalid message sequence: got header before publish.");
+                message->setHeader(header);
+                processMessage(route);
+            }
+
+            /**
+             * A template method that handles a received content and
+             * if this completes the message, routes it using the
+             * functor passed in.
+             */
+            template<class Operation> void handleContent(qpid::framing::AMQContentBody::shared_ptr content, Operation route){
+                checkMessage("Invalid message sequence: got content before publish.");
+                message->addContent(content);
+                processMessage(route);
+            }
+
         };
     }
 }

Modified: incubator/qpid/trunk/qpid/cpp/broker/inc/Message.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/broker/inc/Message.h?view=diff&rev=453657&r1=453656&r2=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/broker/inc/Message.h (original)
+++ incubator/qpid/trunk/qpid/cpp/broker/inc/Message.h Fri Oct  6 09:17:06 2006
@@ -29,14 +29,19 @@
 namespace qpid {
     namespace broker {
         class ExchangeRegistry;
-
+ 
+        /**
+         * Represents an AMQP message, i.e. a header body, a list of
+         * content bodies and some details about the publication
+         * request.
+         */
         class Message{
             typedef std::vector<qpid::framing::AMQContentBody::shared_ptr> content_list;
             typedef content_list::iterator content_iterator;
 
             const ConnectionToken* const publisher;
-            string exchange;
-            string routingKey;
+            const string exchange;
+            const string routingKey;
             const bool mandatory;
             const bool immediate;
             bool redelivered;
@@ -44,8 +49,6 @@
             content_list content;
 
             u_int64_t contentSize();
-            qpid::framing::BasicHeaderProperties* getHeaderProperties();
-    
 
         public:
             typedef std::tr1::shared_ptr<Message> shared_ptr;
@@ -64,10 +67,10 @@
                          u_int32_t framesize);
             void redeliver();
 
-            friend bool route(Message::shared_ptr& msg, ExchangeRegistry* registry);
-
+            qpid::framing::BasicHeaderProperties* getHeaderProperties();
+            const string& getRoutingKey() const { return routingKey; }
+            const string& getExchange() const { return exchange; }
         };
-        bool route(Message::shared_ptr& msg, ExchangeRegistry* registry);
     }
 }
 

Added: incubator/qpid/trunk/qpid/cpp/broker/inc/Router.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/broker/inc/Router.h?view=auto&rev=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/broker/inc/Router.h (added)
+++ incubator/qpid/trunk/qpid/cpp/broker/inc/Router.h Fri Oct  6 09:17:06 2006
@@ -0,0 +1,39 @@
+/*
+ *
+ * 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.
+ *
+ */
+#ifndef _Router_
+#define _Router_
+
+#include "ExchangeRegistry.h"
+#include "Message.h"
+
+/**
+ * A routing functor
+ */
+namespace qpid {
+    namespace broker {
+        class Router{
+            ExchangeRegistry& registry;
+        public:
+            Router(ExchangeRegistry& registry);
+            void operator()(Message::shared_ptr& msg);
+        };
+    }
+}
+
+
+#endif

Propchange: incubator/qpid/trunk/qpid/cpp/broker/inc/Router.h
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/qpid/trunk/qpid/cpp/broker/src/Channel.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/broker/src/Channel.cpp?view=diff&rev=453657&r1=453656&r2=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/broker/src/Channel.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/broker/src/Channel.cpp Fri Oct  6 09:17:06 2006
@@ -126,38 +126,17 @@
     if(queue) queue->cancel(this);
 }
 
-void Channel::handlePublish(Message* msg){
-    if(message.get()){
-        THROW_QPID_ERROR(PROTOCOL_ERROR + 504, "Invalid message sequence: got publish before previous content was completed.");
-    }
-    message = Message::shared_ptr(msg);
-}
-
-void Channel::handleHeader(AMQHeaderBody::shared_ptr header, ExchangeRegistry* exchanges){
+void Channel::checkMessage(const std::string& text){
     if(!message.get()){
-        THROW_QPID_ERROR(PROTOCOL_ERROR + 504, "Invalid message sequence: got header before publish.");
-    }
-    message->setHeader(header);
-    if(message->isComplete()){
-        publish(exchanges);
+        THROW_QPID_ERROR(PROTOCOL_ERROR + 504, text);
     }
 }
 
-void Channel::handleContent(AMQContentBody::shared_ptr content, ExchangeRegistry* exchanges){
-    if(!message.get()){
-        THROW_QPID_ERROR(PROTOCOL_ERROR + 504, "Invalid message sequence: got content before publish.");
-    }
-    message->addContent(content);
-    if(message->isComplete()){
-        publish(exchanges);
-    }
-}
-
-void Channel::publish(ExchangeRegistry* exchanges){
-    if(!route(message, exchanges)){
-        std::cout << "WARNING: Could not route message." << std::endl;
+void Channel::handlePublish(Message* msg){
+    if(message.get()){
+        THROW_QPID_ERROR(PROTOCOL_ERROR + 504, "Invalid message sequence: got publish before previous content was completed.");
     }
-    message.reset();
+    message = Message::shared_ptr(msg);
 }
 
 void Channel::ack(u_int64_t deliveryTag, bool multiple){

Modified: incubator/qpid/trunk/qpid/cpp/broker/src/Message.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/broker/src/Message.cpp?view=diff&rev=453657&r1=453656&r2=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/broker/src/Message.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/broker/src/Message.cpp Fri Oct  6 09:17:06 2006
@@ -90,13 +90,3 @@
     return publisher;
 }
 
-bool qpid::broker::route(Message::shared_ptr& msg, ExchangeRegistry* registry){
-    Exchange* exchange = registry->get(msg->exchange);
-    if(exchange){
-        exchange->route(msg, msg->routingKey, &(msg->getHeaderProperties()->getHeaders()));
-        return true;
-    }else{
-        return false;
-    }
-}
-

Added: incubator/qpid/trunk/qpid/cpp/broker/src/Router.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/broker/src/Router.cpp?view=auto&rev=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/broker/src/Router.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/broker/src/Router.cpp Fri Oct  6 09:17:06 2006
@@ -0,0 +1,32 @@
+/*
+ *
+ * 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 "Router.h"
+
+using namespace qpid::broker;
+
+Router::Router(ExchangeRegistry& _registry) : registry(_registry){}
+
+void Router::operator()(Message::shared_ptr& msg){
+    Exchange* exchange = registry.get(msg->getExchange());
+    if(exchange){
+        exchange->route(msg, msg->getRoutingKey(), &(msg->getHeaderProperties()->getHeaders()));
+    }else{
+        std::cout << "WARNING: Could not route message, unknown exchange: " << msg->getExchange() << std::endl;
+    }
+
+}

Propchange: incubator/qpid/trunk/qpid/cpp/broker/src/Router.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/qpid/trunk/qpid/cpp/broker/src/SessionHandlerImpl.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/broker/src/SessionHandlerImpl.cpp?view=diff&rev=453657&r1=453656&r2=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/broker/src/SessionHandlerImpl.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/broker/src/SessionHandlerImpl.cpp Fri Oct  6 09:17:06 2006
@@ -18,8 +18,9 @@
 #include <iostream>
 #include "SessionHandlerImpl.h"
 #include "FanOutExchange.h"
-#include "TopicExchange.h"
 #include "HeadersExchange.h"
+#include "Router.h"
+#include "TopicExchange.h"
 #include "assert.h"
 
 using namespace std::tr1;
@@ -153,11 +154,11 @@
 }
 
 void SessionHandlerImpl::handleHeader(u_int16_t channel, AMQHeaderBody::shared_ptr body){
-    getChannel(channel)->handleHeader(body, exchanges);
+    getChannel(channel)->handleHeader(body, Router(*exchanges));
 }
 
 void SessionHandlerImpl::handleContent(u_int16_t channel, AMQContentBody::shared_ptr body){
-    getChannel(channel)->handleContent(body, exchanges);
+    getChannel(channel)->handleContent(body, Router(*exchanges));
 }
 
 void SessionHandlerImpl::handleHeartbeat(AMQHeartbeatBody::shared_ptr body){

Added: incubator/qpid/trunk/qpid/cpp/broker/test/ChannelTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/broker/test/ChannelTest.cpp?view=auto&rev=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/broker/test/ChannelTest.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/broker/test/ChannelTest.cpp Fri Oct  6 09:17:06 2006
@@ -0,0 +1,81 @@
+/*
+ *
+ * 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 "Channel.h"
+#include "Message.h"
+#include <cppunit/TestCase.h>
+#include <cppunit/TextTestRunner.h>
+#include <cppunit/extensions/HelperMacros.h>
+#include <cppunit/plugin/TestPlugIn.h>
+#include <iostream>
+#include <memory>
+
+using namespace qpid::broker;
+using namespace qpid::framing;
+using namespace qpid::concurrent;
+
+struct MessageHolder{
+    Message::shared_ptr last;
+};
+
+class DummyRouter{
+    MessageHolder& holder;
+
+public:
+    DummyRouter(MessageHolder& _holder) : holder(_holder){
+    }
+
+    void operator()(Message::shared_ptr& msg){
+        holder.last = msg;
+    }
+};
+
+
+class ChannelTest : public CppUnit::TestCase  
+{
+    CPPUNIT_TEST_SUITE(ChannelTest);
+    CPPUNIT_TEST(testIncoming);
+    CPPUNIT_TEST_SUITE_END();
+
+  public:
+
+    void testIncoming(){
+        Channel channel(0, 0, 10000);
+        string routingKey("my_routing_key");
+        channel.handlePublish(new Message(0, "test", routingKey, false, false));
+        AMQHeaderBody::shared_ptr header(new AMQHeaderBody(BASIC));
+        header->setContentSize(14);
+        string data1("abcdefg");
+        string data2("hijklmn");
+        AMQContentBody::shared_ptr part1(new AMQContentBody(data1));
+        AMQContentBody::shared_ptr part2(new AMQContentBody(data2));        
+
+        MessageHolder holder;
+        channel.handleHeader(header, DummyRouter(holder));
+        CPPUNIT_ASSERT(!holder.last);
+        channel.handleContent(part1, DummyRouter(holder));
+        CPPUNIT_ASSERT(!holder.last);
+        channel.handleContent(part2, DummyRouter(holder));
+        CPPUNIT_ASSERT(holder.last);
+        CPPUNIT_ASSERT_EQUAL(routingKey, holder.last->getRoutingKey());
+    }
+};
+
+// Make this test suite a plugin.
+CPPUNIT_PLUGIN_IMPLEMENT();
+CPPUNIT_TEST_SUITE_REGISTRATION(ChannelTest);
+

Propchange: incubator/qpid/trunk/qpid/cpp/broker/test/ChannelTest.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/qpid/trunk/qpid/cpp/broker/test/RouterTest.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/broker/test/RouterTest.cpp?view=auto&rev=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/broker/test/RouterTest.cpp (added)
+++ incubator/qpid/trunk/qpid/cpp/broker/test/RouterTest.cpp Fri Oct  6 09:17:06 2006
@@ -0,0 +1,91 @@
+/*
+ *
+ * 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 "Channel.h"
+#include "Exchange.h"
+#include "ExchangeRegistry.h"
+#include "Message.h"
+#include "Router.h"
+#include <cppunit/TestCase.h>
+#include <cppunit/TextTestRunner.h>
+#include <cppunit/extensions/HelperMacros.h>
+#include <cppunit/plugin/TestPlugIn.h>
+#include <iostream>
+#include <memory>
+
+using namespace qpid::broker;
+using namespace qpid::framing;
+using namespace qpid::concurrent;
+
+struct TestExchange : public Exchange{
+    Message::shared_ptr msg;
+    string routingKey; 
+    FieldTable* args;
+
+    TestExchange() : Exchange("test"), args(0) {}
+    
+    void bind(Queue::shared_ptr queue, const string& routingKey, FieldTable* args){
+    }
+
+    void unbind(Queue::shared_ptr queue, const string& routingKey, FieldTable* args){
+    }
+
+    void route(Message::shared_ptr& msg, const string& routingKey, FieldTable* args){
+        this->msg = msg;
+        this->routingKey = routingKey;
+        this->args = args;
+    }
+};
+
+class RouterTest : public CppUnit::TestCase  
+{
+    CPPUNIT_TEST_SUITE(RouterTest);
+    CPPUNIT_TEST(test);
+    CPPUNIT_TEST_SUITE_END();
+
+  public:
+
+    void test() 
+    {
+        ExchangeRegistry registry;
+        TestExchange* exchange = new TestExchange();
+        registry.declare(exchange);
+
+        string routingKey("my_routing_key");
+        string name("name");
+        string value("value");
+        Message::shared_ptr msg(new Message(0, "test", routingKey, false, false));
+        AMQHeaderBody::shared_ptr header(new AMQHeaderBody(BASIC));
+
+        dynamic_cast<BasicHeaderProperties*>(header->getProperties())->getHeaders().setString(name, value);
+        msg->setHeader(header);
+
+        Router router(registry);
+        router(msg);
+
+        CPPUNIT_ASSERT(exchange->msg);
+        CPPUNIT_ASSERT_EQUAL(msg, exchange->msg);
+        CPPUNIT_ASSERT_EQUAL(routingKey, exchange->msg->getRoutingKey());
+        CPPUNIT_ASSERT_EQUAL(routingKey, exchange->routingKey);
+        CPPUNIT_ASSERT_EQUAL(value, exchange->args->getString(name));
+    }
+};
+
+// Make this test suite a plugin.
+CPPUNIT_PLUGIN_IMPLEMENT();
+CPPUNIT_TEST_SUITE_REGISTRATION(RouterTest);
+

Propchange: incubator/qpid/trunk/qpid/cpp/broker/test/RouterTest.cpp
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/qpid/trunk/qpid/cpp/client/inc/IncomingMessage.h
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/client/inc/IncomingMessage.h?view=diff&rev=453657&r1=453656&r2=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/client/inc/IncomingMessage.h (original)
+++ incubator/qpid/trunk/qpid/cpp/client/inc/IncomingMessage.h Fri Oct  6 09:17:06 2006
@@ -47,7 +47,7 @@
 	bool isReturn();
 	bool isDelivery();
 	bool isResponse();
-	string& getConsumerTag();//only relevant if isDelivery()
+	const string& getConsumerTag();//only relevant if isDelivery()
 	qpid::framing::AMQHeaderBody::shared_ptr& getHeader();
         u_int64_t getDeliveryTag();
 	void getData(string& data);

Modified: incubator/qpid/trunk/qpid/cpp/client/src/IncomingMessage.cpp
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/client/src/IncomingMessage.cpp?view=diff&rev=453657&r1=453656&r2=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/client/src/IncomingMessage.cpp (original)
+++ incubator/qpid/trunk/qpid/cpp/client/src/IncomingMessage.cpp Fri Oct  6 09:17:06 2006
@@ -53,7 +53,7 @@
     return response;
 }
 
-string& IncomingMessage::getConsumerTag(){
+const string& IncomingMessage::getConsumerTag(){
     if(!isDelivery()) THROW_QPID_ERROR(CLIENT_ERROR, "Consumer tag only valid for delivery");
     return delivered->getConsumerTag();
 }

Modified: incubator/qpid/trunk/qpid/cpp/common/framing/generated/stylesheets/utils.xsl
URL: http://svn.apache.org/viewvc/incubator/qpid/trunk/qpid/cpp/common/framing/generated/stylesheets/utils.xsl?view=diff&rev=453657&r1=453656&r2=453657
==============================================================================
--- incubator/qpid/trunk/qpid/cpp/common/framing/generated/stylesheets/utils.xsl (original)
+++ incubator/qpid/trunk/qpid/cpp/common/framing/generated/stylesheets/utils.xsl Fri Oct  6 09:17:06 2006
@@ -23,8 +23,8 @@
     <xsl:choose>
 	 <xsl:when test="$t='octet'">u_int8_t</xsl:when> 		 		 
 	 <xsl:when test="$t='short'">u_int16_t</xsl:when> 		 		 
-	 <xsl:when test="$t='shortstr'">string&amp;</xsl:when> 		 		 
-	 <xsl:when test="$t='longstr'">string&amp;</xsl:when> 		 		 
+	 <xsl:when test="$t='shortstr'">const string&amp;</xsl:when> 		 		 
+	 <xsl:when test="$t='longstr'">const string&amp;</xsl:when> 		 		 
 	 <xsl:when test="$t='bit'">bool</xsl:when> 		 		 
 	 <xsl:when test="$t='long'">u_int32_t</xsl:when> 		 		 
 	 <xsl:when test="$t='longlong'">u_int64_t</xsl:when>