summaryrefslogtreecommitdiff
path: root/qpid/java/broker
diff options
context:
space:
mode:
authorRobert Godfrey <rgodfrey@apache.org>2007-02-25 01:08:57 +0000
committerRobert Godfrey <rgodfrey@apache.org>2007-02-25 01:08:57 +0000
commit2e9743ac06fc05609155769bf04f4fa442d848c2 (patch)
tree76e110ec10ab5912d50175484e904d624710e735 /qpid/java/broker
parentf4d8d42443bbfb942a73c9da32cbdf8fe7ad35e7 (diff)
downloadqpid-python-2e9743ac06fc05609155769bf04f4fa442d848c2.tar.gz
QPID-391 : Broker Refactoring - initial tidy... add some mechanisms for multi version
git-svn-id: https://svn.apache.org/repos/asf/incubator/qpid/trunk@511389 13f79535-47bb-0310-9956-ffa450edef68
Diffstat (limited to 'qpid/java/broker')
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/AMQChannel.java9
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/Main.java24
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/ack/UnacknowledgedMessageMapImpl.java2
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/filter/PropertyExpression.java31
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/output/ProtocolOutputConverter.java57
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/output/ProtocolOutputConverterRegistry.java62
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/output/amqp0_8/ProtocolOutputConverterImpl.java288
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQMinaProtocolSession.java254
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQNoMethodHandlerException.java34
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQPFastProtocolHandler.java15
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSession.java3
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSessionMBean.java7
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/UnknnownMessageTypeException.java33
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQMessage.java40
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQQueueMBean.java13
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/queue/ConcurrentSelectorDeliveryManager.java3
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/queue/SubscriptionImpl.java16
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/store/ContentChunkAdapter.java57
-rw-r--r--qpid/java/broker/src/main/java/org/apache/qpid/server/store/MessagePublishInfoAdapter.java62
19 files changed, 700 insertions, 310 deletions
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/AMQChannel.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/AMQChannel.java
index 7ceb3a7eef..be2cee79ee 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/AMQChannel.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/AMQChannel.java
@@ -433,7 +433,10 @@ public class AMQChannel
}
- /** Called to resend all outstanding unacknowledged messages to this same channel. */
+ /** Called to resend all outstanding unacknowledged messages to this same channel.
+ * @param session the session
+ * @param requeue if true then requeue, else resend
+ * @throws org.apache.qpid.AMQException */
public void resend(final AMQProtocolSession session, final boolean requeue) throws AMQException
{
final List<UnacknowledgedMessage> msgToRequeue = new LinkedList<UnacknowledgedMessage>();
@@ -752,7 +755,9 @@ public class AMQChannel
for (RequiredDeliveryException bouncedMessage : _returnMessages)
{
AMQMessage message = bouncedMessage.getAMQMessage();
- message.writeReturn(session, _channelId, bouncedMessage.getReplyCode().getCode(), new AMQShortString(bouncedMessage.getMessage()));
+ session.getProtocolOutputConverter().writeReturn(message, _channelId,
+ bouncedMessage.getReplyCode().getCode(),
+ new AMQShortString(bouncedMessage.getMessage()));
}
_returnMessages.clear();
}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/Main.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/Main.java
index 42fe8c5274..a48bc5df7f 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/Main.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/Main.java
@@ -45,7 +45,7 @@ import org.apache.mina.common.SimpleByteBufferAllocator;
import org.apache.mina.transport.socket.nio.SocketAcceptorConfig;
import org.apache.mina.transport.socket.nio.SocketSessionConfig;
import org.apache.qpid.AMQException;
-import org.apache.qpid.framing.ProtocolVersionList;
+import org.apache.qpid.framing.ProtocolVersion;
import org.apache.qpid.pool.ReadWriteThreadModel;
import org.apache.qpid.server.configuration.VirtualHostConfiguration;
import org.apache.qpid.server.protocol.AMQPFastProtocolHandler;
@@ -59,7 +59,8 @@ import org.apache.qpid.url.URLSyntaxException;
* Main entry point for AMQPD.
*
*/
-public class Main implements ProtocolVersionList
+@SuppressWarnings({"AccessStaticViaInstance"})
+public class Main
{
private static final Logger _logger = Logger.getLogger(Main.class);
@@ -143,12 +144,21 @@ public class Main implements ProtocolVersionList
else if (commandLine.hasOption("v"))
{
String ver = "Qpid 0.9.0.0";
- String protocol = "AMQP version(s) [major.minor]: ";
- for (int i=0; i<pv.length; i++)
+ StringBuilder protocol = new StringBuilder("AMQP version(s) [major.minor]: ");
+
+ boolean first = true;
+ for (ProtocolVersion pv : ProtocolVersion.getSupportedProtocolVersions())
{
- if (i > 0)
- protocol += ", ";
- protocol += pv[i][PROTOCOL_MAJOR] + "." + pv[i][PROTOCOL_MINOR];
+ if(first)
+ {
+ first = false;
+ }
+ else
+ {
+ protocol.append(", ");
+ }
+ protocol.append(pv.getMajorVersion()).append('-').append(pv.getMinorVersion());
+
}
System.out.println(ver + " (" + protocol + ")");
}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/ack/UnacknowledgedMessageMapImpl.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/ack/UnacknowledgedMessageMapImpl.java
index fdf087fdea..99cc60011a 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/ack/UnacknowledgedMessageMapImpl.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/ack/UnacknowledgedMessageMapImpl.java
@@ -209,7 +209,7 @@ public class UnacknowledgedMessageMapImpl implements UnacknowledgedMessageMap
if(consumerTag != null)
{
- msg.writeDeliver(protocolSession, channelId, deliveryTag, consumerTag);
+ protocolSession.getProtocolOutputConverter().writeDeliver(msg, channelId, deliveryTag, consumerTag);
}
}
}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/filter/PropertyExpression.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/filter/PropertyExpression.java
index 348bfa5e68..bdabcbf5be 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/filter/PropertyExpression.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/filter/PropertyExpression.java
@@ -25,7 +25,8 @@ import java.util.HashMap;
import org.apache.log4j.Logger;
import org.apache.qpid.AMQException;
-import org.apache.qpid.framing.BasicContentHeaderProperties;
+import org.apache.qpid.framing.CommonContentHeaderProperties;
+import org.apache.qpid.framing.AMQShortString;
import org.apache.qpid.server.queue.AMQMessage;
/**
@@ -63,8 +64,9 @@ public class PropertyExpression implements Expression
{
try
{
- BasicContentHeaderProperties _properties = (BasicContentHeaderProperties) message.getContentHeaderBody().properties;
- return _properties.getReplyTo();
+ CommonContentHeaderProperties _properties = (CommonContentHeaderProperties) message.getContentHeaderBody().properties;
+ AMQShortString replyTo = _properties.getReplyTo();
+ return replyTo == null ? null : replyTo.toString();
}
catch (AMQException e)
{
@@ -83,8 +85,9 @@ public class PropertyExpression implements Expression
{
try
{
- BasicContentHeaderProperties _properties = (BasicContentHeaderProperties) message.getContentHeaderBody().properties;
- return _properties.getType();
+ CommonContentHeaderProperties _properties = (CommonContentHeaderProperties) message.getContentHeaderBody().properties;
+ AMQShortString type = _properties.getType();
+ return type == null ? null : type.toString();
}
catch (AMQException e)
{
@@ -126,7 +129,7 @@ public class PropertyExpression implements Expression
{
try
{
- BasicContentHeaderProperties _properties = (BasicContentHeaderProperties) message.getContentHeaderBody().properties;
+ CommonContentHeaderProperties _properties = (CommonContentHeaderProperties) message.getContentHeaderBody().properties;
return (int) _properties.getPriority();
}
catch (AMQException e)
@@ -147,8 +150,9 @@ public class PropertyExpression implements Expression
try
{
- BasicContentHeaderProperties _properties = (BasicContentHeaderProperties) message.getContentHeaderBody().properties;
- return _properties.getMessageId();
+ CommonContentHeaderProperties _properties = (CommonContentHeaderProperties) message.getContentHeaderBody().properties;
+ AMQShortString messageId = _properties.getMessageId();
+ return messageId == null ? null : messageId;
}
catch (AMQException e)
{
@@ -168,7 +172,7 @@ public class PropertyExpression implements Expression
try
{
- BasicContentHeaderProperties _properties = (BasicContentHeaderProperties) message.getContentHeaderBody().properties;
+ CommonContentHeaderProperties _properties = (CommonContentHeaderProperties) message.getContentHeaderBody().properties;
return _properties.getTimestamp();
}
catch (AMQException e)
@@ -189,8 +193,9 @@ public class PropertyExpression implements Expression
try
{
- BasicContentHeaderProperties _properties = (BasicContentHeaderProperties) message.getContentHeaderBody().properties;
- return _properties.getCorrelationId();
+ CommonContentHeaderProperties _properties = (CommonContentHeaderProperties) message.getContentHeaderBody().properties;
+ AMQShortString correlationId = _properties.getCorrelationId();
+ return correlationId == null ? null : correlationId.toString();
}
catch (AMQException e)
{
@@ -210,7 +215,7 @@ public class PropertyExpression implements Expression
try
{
- BasicContentHeaderProperties _properties = (BasicContentHeaderProperties) message.getContentHeaderBody().properties;
+ CommonContentHeaderProperties _properties = (CommonContentHeaderProperties) message.getContentHeaderBody().properties;
return _properties.getExpiration();
}
catch (AMQException e)
@@ -254,7 +259,7 @@ public class PropertyExpression implements Expression
else
{
- BasicContentHeaderProperties _properties = (BasicContentHeaderProperties) message.getContentHeaderBody().properties;
+ CommonContentHeaderProperties _properties = (CommonContentHeaderProperties) message.getContentHeaderBody().properties;
if(_logger.isDebugEnabled())
{
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/output/ProtocolOutputConverter.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/output/ProtocolOutputConverter.java
new file mode 100644
index 0000000000..e01c5aabbf
--- /dev/null
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/output/ProtocolOutputConverter.java
@@ -0,0 +1,57 @@
+/*
+ *
+ * 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.
+ *
+ */
+
+/*
+ * This file is auto-generated by Qpid Gentools v.0.1 - do not modify.
+ * Supported AMQP versions:
+ * 8-0
+ */
+package org.apache.qpid.server.output;
+
+import org.apache.qpid.server.queue.AMQMessage;
+import org.apache.qpid.server.protocol.AMQProtocolSession;
+import org.apache.qpid.framing.AMQShortString;
+import org.apache.qpid.framing.AMQDataBlock;
+import org.apache.qpid.AMQException;
+
+public interface ProtocolOutputConverter
+{
+ void confirmConsumerAutoClose(int channelId, AMQShortString consumerTag);
+
+ interface Factory
+ {
+ ProtocolOutputConverter newInstance(AMQProtocolSession session);
+ }
+
+ void writeDeliver(AMQMessage message, int channelId, long deliveryTag, AMQShortString consumerTag)
+ throws AMQException;
+
+ void writeGetOk(AMQMessage message, int channelId, long deliveryTag, int queueSize) throws AMQException;
+
+ byte getProtocolMinorVersion();
+
+ byte getProtocolMajorVersion();
+
+ void writeReturn(AMQMessage message, int channelId, int replyCode, AMQShortString replyText)
+ throws AMQException;
+
+ void writeFrame(AMQDataBlock block);
+}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/output/ProtocolOutputConverterRegistry.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/output/ProtocolOutputConverterRegistry.java
new file mode 100644
index 0000000000..8366c426dd
--- /dev/null
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/output/ProtocolOutputConverterRegistry.java
@@ -0,0 +1,62 @@
+/*
+ *
+ * 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.
+ *
+ */
+
+/*
+ * This file is auto-generated by Qpid Gentools v.0.1 - do not modify.
+ * Supported AMQP versions:
+ * 8-0
+ */
+package org.apache.qpid.server.output;
+
+import org.apache.qpid.server.output.ProtocolOutputConverter.Factory;
+import org.apache.qpid.server.output.amqp0_8.ProtocolOutputConverterImpl;
+import org.apache.qpid.server.protocol.AMQProtocolSession;
+
+import java.util.Map;
+import java.util.HashMap;
+
+public class ProtocolOutputConverterRegistry
+{
+
+ private static final Map<Byte, Map<Byte, Factory>> _registry =
+ new HashMap<Byte, Map<Byte, Factory>>();
+
+
+ static
+ {
+ register((byte) 8, (byte) 0, ProtocolOutputConverterImpl.getInstanceFactory());
+ }
+
+ private static void register(byte major, byte minor, Factory converter)
+ {
+ if(!_registry.containsKey(major))
+ {
+ _registry.put(major, new HashMap<Byte, Factory>());
+ }
+ _registry.get(major).put(minor, converter);
+ }
+
+
+ public static ProtocolOutputConverter getConverter(AMQProtocolSession session)
+ {
+ return _registry.get(session.getProtocolMajorVersion()).get(session.getProtocolMinorVersion()).newInstance(session);
+ }
+}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/output/amqp0_8/ProtocolOutputConverterImpl.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/output/amqp0_8/ProtocolOutputConverterImpl.java
new file mode 100644
index 0000000000..bd5bb632fe
--- /dev/null
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/output/amqp0_8/ProtocolOutputConverterImpl.java
@@ -0,0 +1,288 @@
+/*
+ *
+ * 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.
+ *
+ */
+
+/*
+ * This file is auto-generated by Qpid Gentools v.0.1 - do not modify.
+ * Supported AMQP versions:
+ * 8-0
+ */
+package org.apache.qpid.server.output.amqp0_8;
+
+import org.apache.qpid.server.protocol.AMQProtocolSession;
+import org.apache.qpid.server.queue.AMQMessage;
+import org.apache.qpid.server.queue.AMQMessageHandle;
+import org.apache.qpid.server.store.StoreContext;
+import org.apache.qpid.server.output.ProtocolOutputConverter;
+import org.apache.qpid.framing.*;
+import org.apache.qpid.framing.abstraction.ContentChunk;
+import org.apache.qpid.framing.abstraction.MessagePublishInfo;
+import org.apache.qpid.AMQException;
+
+import org.apache.mina.common.ByteBuffer;
+
+import java.util.Iterator;
+
+public class ProtocolOutputConverterImpl implements ProtocolOutputConverter
+{
+
+ public static Factory getInstanceFactory()
+ {
+ return new Factory()
+ {
+
+ public ProtocolOutputConverter newInstance(AMQProtocolSession session)
+ {
+ return new ProtocolOutputConverterImpl(session);
+ }
+ };
+ }
+
+ private final AMQProtocolSession _protocolSession;
+
+ private ProtocolOutputConverterImpl(AMQProtocolSession session)
+ {
+ _protocolSession = session;
+ }
+
+
+ public AMQProtocolSession getProtocolSession()
+ {
+ return _protocolSession;
+ }
+
+ public void writeDeliver(AMQMessage message, int channelId, long deliveryTag, AMQShortString consumerTag)
+ throws AMQException
+ {
+ ByteBuffer deliver = createEncodedDeliverFrame(message, channelId, deliveryTag, consumerTag);
+ AMQDataBlock contentHeader = ContentHeaderBody.createAMQFrame(channelId,
+ message.getContentHeaderBody());
+
+ final AMQMessageHandle messageHandle = message.getMessageHandle();
+ final StoreContext storeContext = message.getStoreContext();
+ final long messageId = message.getMessageId();
+
+ final int bodyCount = messageHandle.getBodyCount(storeContext,messageId);
+
+ if(bodyCount == 0)
+ {
+ SmallCompositeAMQDataBlock compositeBlock = new SmallCompositeAMQDataBlock(deliver,
+ contentHeader);
+
+ writeFrame(compositeBlock);
+ }
+ else
+ {
+
+
+ //
+ // Optimise the case where we have a single content body. In that case we create a composite block
+ // so that we can writeDeliver out the deliver, header and body with a single network writeDeliver.
+ //
+ ContentChunk cb = messageHandle.getContentChunk(storeContext,messageId, 0);
+
+ AMQDataBlock firstContentBody = new AMQFrame(channelId, getProtocolSession().getRegistry().getProtocolVersionMethodConverter().convertToBody(cb));
+ AMQDataBlock[] headerAndFirstContent = new AMQDataBlock[]{contentHeader, firstContentBody};
+ CompositeAMQDataBlock compositeBlock = new CompositeAMQDataBlock(deliver, headerAndFirstContent);
+ writeFrame(compositeBlock);
+
+ //
+ // Now start writing out the other content bodies
+ //
+ for(int i = 1; i < bodyCount; i++)
+ {
+ cb = messageHandle.getContentChunk(storeContext,messageId, i);
+ writeFrame(new AMQFrame(channelId, getProtocolSession().getRegistry().getProtocolVersionMethodConverter().convertToBody(cb)));
+ }
+
+
+ }
+
+
+ }
+
+
+ public void writeGetOk(AMQMessage message, int channelId, long deliveryTag, int queueSize) throws AMQException
+ {
+
+ final AMQMessageHandle messageHandle = message.getMessageHandle();
+ final StoreContext storeContext = message.getStoreContext();
+ final long messageId = message.getMessageId();
+
+ ByteBuffer deliver = createEncodedGetOkFrame(message, channelId, deliveryTag, queueSize);
+
+
+ AMQDataBlock contentHeader = ContentHeaderBody.createAMQFrame(channelId,
+ message.getContentHeaderBody());
+
+ final int bodyCount = messageHandle.getBodyCount(storeContext,messageId);
+ if(bodyCount == 0)
+ {
+ SmallCompositeAMQDataBlock compositeBlock = new SmallCompositeAMQDataBlock(deliver,
+ contentHeader);
+ writeFrame(compositeBlock);
+ }
+ else
+ {
+
+
+ //
+ // Optimise the case where we have a single content body. In that case we create a composite block
+ // so that we can writeDeliver out the deliver, header and body with a single network writeDeliver.
+ //
+ ContentChunk cb = messageHandle.getContentChunk(storeContext,messageId, 0);
+
+ AMQDataBlock firstContentBody = new AMQFrame(channelId, getProtocolSession().getRegistry().getProtocolVersionMethodConverter().convertToBody(cb));
+ AMQDataBlock[] headerAndFirstContent = new AMQDataBlock[]{contentHeader, firstContentBody};
+ CompositeAMQDataBlock compositeBlock = new CompositeAMQDataBlock(deliver, headerAndFirstContent);
+ writeFrame(compositeBlock);
+
+ //
+ // Now start writing out the other content bodies
+ //
+ for(int i = 1; i < bodyCount; i++)
+ {
+ cb = messageHandle.getContentChunk(storeContext, messageId, i);
+ writeFrame(new AMQFrame(channelId, getProtocolSession().getRegistry().getProtocolVersionMethodConverter().convertToBody(cb)));
+ }
+
+
+ }
+
+
+ }
+
+
+ private ByteBuffer createEncodedDeliverFrame(AMQMessage message, int channelId, long deliveryTag, AMQShortString consumerTag)
+ throws AMQException
+ {
+ final MessagePublishInfo pb = message.getMessagePublishInfo();
+ final AMQMessageHandle messageHandle = message.getMessageHandle();
+
+ AMQFrame deliverFrame = BasicDeliverBody.createAMQFrame(channelId, getProtocolMajorVersion(),
+ getProtocolMinorVersion(),
+ consumerTag,
+ deliveryTag, pb.getExchange(), messageHandle.isRedelivered(),
+ pb.getRoutingKey());
+
+ ByteBuffer buf = ByteBuffer.allocate((int) deliverFrame.getSize()); // XXX: Could cast be a problem?
+ deliverFrame.writePayload(buf);
+ buf.flip();
+ return buf;
+ }
+
+ private ByteBuffer createEncodedGetOkFrame(AMQMessage message, int channelId, long deliveryTag, int queueSize)
+ throws AMQException
+ {
+ final MessagePublishInfo pb = message.getMessagePublishInfo();
+ final AMQMessageHandle messageHandle = message.getMessageHandle();
+
+ AMQFrame getOkFrame = BasicGetOkBody.createAMQFrame(channelId,
+ getProtocolMajorVersion(),
+ getProtocolMinorVersion(),
+ deliveryTag, pb.getExchange(),
+ queueSize,
+ messageHandle.isRedelivered(),
+ pb.getRoutingKey());
+ ByteBuffer buf = ByteBuffer.allocate((int) getOkFrame.getSize()); // XXX: Could cast be a problem?
+ getOkFrame.writePayload(buf);
+ buf.flip();
+ return buf;
+ }
+
+ public byte getProtocolMinorVersion()
+ {
+ return getProtocolSession().getProtocolMinorVersion();
+ }
+
+ public byte getProtocolMajorVersion()
+ {
+ return getProtocolSession().getProtocolMajorVersion();
+ }
+
+ private ByteBuffer createEncodedReturnFrame(AMQMessage message, int channelId, int replyCode, AMQShortString replyText) throws AMQException
+ {
+ AMQFrame returnFrame = BasicReturnBody.createAMQFrame(channelId,
+ getProtocolMajorVersion(),
+ getProtocolMinorVersion(),
+ message.getMessagePublishInfo().getExchange(),
+ replyCode, replyText,
+ message.getMessagePublishInfo().getRoutingKey());
+ ByteBuffer buf = ByteBuffer.allocate((int) returnFrame.getSize()); // XXX: Could cast be a problem?
+ returnFrame.writePayload(buf);
+ buf.flip();
+ return buf;
+ }
+
+ public void writeReturn(AMQMessage message, int channelId, int replyCode, AMQShortString replyText)
+ throws AMQException
+ {
+ ByteBuffer returnFrame = createEncodedReturnFrame(message, channelId, replyCode, replyText);
+
+ AMQDataBlock contentHeader = ContentHeaderBody.createAMQFrame(channelId,
+ message.getContentHeaderBody());
+
+ Iterator<AMQDataBlock> bodyFrameIterator = message.getBodyFrameIterator(getProtocolSession(), channelId);
+ //
+ // Optimise the case where we have a single content body. In that case we create a composite block
+ // so that we can writeDeliver out the deliver, header and body with a single network writeDeliver.
+ //
+ if (bodyFrameIterator.hasNext())
+ {
+ AMQDataBlock firstContentBody = bodyFrameIterator.next();
+ AMQDataBlock[] headerAndFirstContent = new AMQDataBlock[]{contentHeader, firstContentBody};
+ CompositeAMQDataBlock compositeBlock = new CompositeAMQDataBlock(returnFrame, headerAndFirstContent);
+ writeFrame(compositeBlock);
+ }
+ else
+ {
+ CompositeAMQDataBlock compositeBlock = new CompositeAMQDataBlock(returnFrame,
+ new AMQDataBlock[]{contentHeader});
+
+ writeFrame(compositeBlock);
+ }
+
+ //
+ // Now start writing out the other content bodies
+ // TODO: MINA needs to be fixed so the the pending writes buffer is not unbounded
+ //
+ while (bodyFrameIterator.hasNext())
+ {
+ writeFrame(bodyFrameIterator.next());
+ }
+ }
+
+
+ public void writeFrame(AMQDataBlock block)
+ {
+ getProtocolSession().writeFrame(block);
+ }
+
+
+ public void confirmConsumerAutoClose(int channelId, AMQShortString consumerTag)
+ {
+
+ writeFrame(BasicCancelOkBody.createAMQFrame(channelId,
+ getProtocolMajorVersion(),
+ getProtocolMinorVersion(),
+ consumerTag // consumerTag
+ ));
+ }
+}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQMinaProtocolSession.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQMinaProtocolSession.java
index 2de32c2f0f..d71f6e3046 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQMinaProtocolSession.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQMinaProtocolSession.java
@@ -43,25 +43,14 @@ import org.apache.qpid.AMQException;
import org.apache.qpid.codec.AMQCodecFactory;
import org.apache.qpid.codec.AMQDecoder;
import org.apache.qpid.common.ClientProperties;
-import org.apache.qpid.framing.AMQDataBlock;
-import org.apache.qpid.framing.AMQFrame;
-import org.apache.qpid.framing.AMQMethodBody;
-import org.apache.qpid.framing.AMQShortString;
-import org.apache.qpid.framing.ConnectionStartBody;
-import org.apache.qpid.framing.ContentBody;
-import org.apache.qpid.framing.ContentHeaderBody;
-import org.apache.qpid.framing.FieldTable;
-import org.apache.qpid.framing.HeartbeatBody;
-import org.apache.qpid.framing.MainRegistry;
-import org.apache.qpid.framing.ProtocolInitiation;
-import org.apache.qpid.framing.ProtocolVersionList;
-import org.apache.qpid.framing.VersionSpecificRegistry;
-import org.apache.qpid.framing.ChannelCloseOkBody;
+import org.apache.qpid.framing.*;
import org.apache.qpid.pool.ReadWriteThreadModel;
import org.apache.qpid.protocol.AMQMethodEvent;
import org.apache.qpid.protocol.AMQMethodListener;
import org.apache.qpid.protocol.AMQConstant;
import org.apache.qpid.server.AMQChannel;
+import org.apache.qpid.server.output.ProtocolOutputConverter;
+import org.apache.qpid.server.output.ProtocolOutputConverterRegistry;
import org.apache.qpid.server.management.Managable;
import org.apache.qpid.server.management.ManagedObject;
import org.apache.qpid.server.registry.ApplicationRegistry;
@@ -71,7 +60,6 @@ import org.apache.qpid.server.virtualhost.VirtualHost;
import org.apache.qpid.server.virtualhost.VirtualHostRegistry;
public class AMQMinaProtocolSession implements AMQProtocolSession,
- ProtocolVersionList,
Managable
{
private static final Logger _logger = Logger.getLogger(AMQProtocolSession.class);
@@ -111,12 +99,13 @@ public class AMQMinaProtocolSession implements AMQProtocolSession,
private long _maxNoOfChannels = 1000;
/* AMQP Version for this session */
- private byte _major = pv[pv.length - 1][PROTOCOL_MAJOR];
- private byte _minor = pv[pv.length - 1][PROTOCOL_MINOR];
+ private ProtocolVersion _protocolVersion = ProtocolVersion.getLatestSupportedVersion();
+
private FieldTable _clientProperties;
private final List<Task> _taskList = new CopyOnWriteArrayList<Task>();
- private VersionSpecificRegistry _registry = MainRegistry.getVersionSpecificRegistry(pv[pv.length - 1][PROTOCOL_MAJOR], pv[pv.length - 1][PROTOCOL_MINOR]);
+ private VersionSpecificRegistry _registry = MainRegistry.getVersionSpecificRegistry(_protocolVersion);
private List<Integer> _closingChannelsList = new ArrayList<Integer>();
+ private ProtocolOutputConverter _protocolOutputConverter;
public ManagedObject getManagedObject()
@@ -195,86 +184,116 @@ public class AMQMinaProtocolSession implements AMQProtocolSession,
_lastReceived = message;
if (message instanceof ProtocolInitiation)
{
- ProtocolInitiation pi = (ProtocolInitiation) message;
- // this ensures the codec never checks for a PI message again
- ((AMQDecoder) _codecFactory.getDecoder()).setExpectProtocolInitiation(false);
- try
- {
- pi.checkVersion(this); // Fails if not correct
+ protocolInitiationReceived((ProtocolInitiation) message);
- // This sets the protocol version (and hence framing classes) for this session.
- setProtocolVersion(pi.protocolMajor, pi.protocolMinor);
+ }
+ else if (message instanceof AMQFrame)
+ {
+ AMQFrame frame = (AMQFrame) message;
+ frameReceived(frame);
- String mechanisms = ApplicationRegistry.getInstance().getAuthenticationManager().getMechanisms();
+ }
+ else
+ {
+ throw new UnknnownMessageTypeException(message);
+ }
+ }
- String locales = "en_US";
+ private void frameReceived(AMQFrame frame)
+ throws AMQException
+ {
+ int channelId = frame.getChannel();
+ AMQBody body = frame.getBodyFrame();
- // Interfacing with generated code - be aware of possible changes to parameter order as versions change.
- AMQFrame response = ConnectionStartBody.createAMQFrame((short) 0,
- _major, _minor, // AMQP version (major, minor)
- locales.getBytes(), // locales
- mechanisms.getBytes(), // mechanisms
- null, // serverProperties
- (short) _major, // versionMajor
- (short) _minor); // versionMinor
- _minaProtocolSession.write(response);
- }
- catch (AMQException e)
- {
- _logger.error("Received incorrect protocol initiation", e);
- /* Find last protocol version in protocol version list. Make sure last protocol version
- listed in the build file (build-module.xml) is the latest version which will be used
- here. */
- int i = pv.length - 1;
- _minaProtocolSession.write(new ProtocolInitiation(pv[i][PROTOCOL_MAJOR], pv[i][PROTOCOL_MINOR]));
- // TODO: Close connection (but how to wait until message is sent?)
- // ritchiem 2006-12-04 will this not do?
-// WriteFuture future = _minaProtocolSession.write(new ProtocolInitiation(pv[i][PROTOCOL_MAJOR], pv[i][PROTOCOL_MINOR]));
-// future.join();
-// close connection
+ if(_logger.isDebugEnabled())
+ {
+ _logger.debug("Frame Received: " + frame);
+ }
- }
+ if (body instanceof AMQMethodBody)
+ {
+ methodFrameReceived(channelId, (AMQMethodBody)body);
+ }
+ else if (body instanceof ContentHeaderBody)
+ {
+ contentHeaderReceived(channelId, (ContentHeaderBody)body);
+ }
+ else if (body instanceof ContentBody)
+ {
+ contentBodyReceived(channelId, (ContentBody)body);
+ }
+ else if (body instanceof HeartbeatBody)
+ {
+ // NO OP
}
else
{
- AMQFrame frame = (AMQFrame) message;
-
- if (frame.getBodyFrame() instanceof AMQMethodBody)
- {
- methodFrameReceived(frame);
- }
- else
- {
- contentFrameReceived(frame);
- }
+ _logger.warn("Unrecognised frame " + frame.getClass().getName());
}
}
- private void methodFrameReceived(AMQFrame frame)
+ private void protocolInitiationReceived(ProtocolInitiation pi)
{
- if (_logger.isDebugEnabled())
+ // this ensures the codec never checks for a PI message again
+ ((AMQDecoder) _codecFactory.getDecoder()).setExpectProtocolInitiation(false);
+ try
{
- _logger.debug("Method frame received: " + frame);
+ pi.checkVersion(); // Fails if not correct
+
+ // This sets the protocol version (and hence framing classes) for this session.
+ setProtocolVersion(pi._protocolMajor, pi._protocolMinor);
+
+ String mechanisms = ApplicationRegistry.getInstance().getAuthenticationManager().getMechanisms();
+
+ String locales = "en_US";
+
+ // Interfacing with generated code - be aware of possible changes to parameter order as versions change.
+ AMQFrame response = ConnectionStartBody.createAMQFrame((short) 0,
+ getProtocolMajorVersion(), getProtocolMinorVersion(), // AMQP version (major, minor)
+ locales.getBytes(), // locales
+ mechanisms.getBytes(), // mechanisms
+ null, // serverProperties
+ (short) getProtocolMajorVersion(), // versionMajor
+ (short) getProtocolMinorVersion()); // versionMinor
+ _minaProtocolSession.write(response);
+ }
+ catch (AMQException e)
+ {
+ _logger.error("Received incorrect protocol initiation", e);
+
+ _minaProtocolSession.write(new ProtocolInitiation(ProtocolVersion.getLatestSupportedVersion()));
+
+ // TODO: Close connection (but how to wait until message is sent?)
+ // ritchiem 2006-12-04 will this not do?
+// WriteFuture future = _minaProtocolSession.write(new ProtocolInitiation(pv[i][PROTOCOLgetProtocolMajorVersion()], pv[i][PROTOCOLgetProtocolMinorVersion()]));
+// future.join();
+// close connection
+
}
+ }
+
- final AMQMethodEvent<AMQMethodBody> evt = new AMQMethodEvent<AMQMethodBody>(frame.getChannel(),
- (AMQMethodBody) frame.getBodyFrame());
+ private void methodFrameReceived(int channelId, AMQMethodBody methodBody)
+ {
+
+ final AMQMethodEvent<AMQMethodBody> evt = new AMQMethodEvent<AMQMethodBody>(channelId,
+ methodBody);
//Check that this channel is not closing
- if (channelAwaitingClosure(frame.getChannel()))
+ if (channelAwaitingClosure(channelId))
{
if ((evt.getMethod() instanceof ChannelCloseOkBody))
{
if (_logger.isInfoEnabled())
{
- _logger.info("Channel[" + frame.getChannel() + "] awaiting closure - processing close-ok");
+ _logger.info("Channel[" + channelId + "] awaiting closure - processing close-ok");
}
}
else
{
if (_logger.isInfoEnabled())
{
- _logger.info("Channel[" + frame.getChannel() + "] awaiting closure ignoring");
+ _logger.info("Channel[" + channelId + "] awaiting closure ignoring");
}
return;
}
@@ -298,19 +317,19 @@ public class AMQMinaProtocolSession implements AMQProtocolSession,
}
if (!wasAnyoneInterested)
{
- throw new AMQException("AMQMethodEvent " + evt + " was not processed by any listener on Broker.");
+ throw new AMQNoMethodHandlerException(evt);
}
}
catch (AMQChannelException e)
{
- if (getChannel(frame.getChannel()) != null)
+ if (getChannel(channelId) != null)
{
if (_logger.isInfoEnabled())
{
_logger.info("Closing channel due to: " + e.getMessage());
}
- writeFrame(e.getCloseFrame(frame.getChannel()));
- closeChannel(frame.getChannel());
+ writeFrame(e.getCloseFrame(channelId));
+ closeChannel(channelId);
}
else
{
@@ -328,7 +347,7 @@ public class AMQMinaProtocolSession implements AMQProtocolSession,
AMQConstant.CHANNEL_ERROR.getName().toString());
_stateManager.changeState(AMQState.CONNECTION_CLOSING);
- writeFrame(ce.getCloseFrame(frame.getChannel()));
+ writeFrame(ce.getCloseFrame(channelId));
}
}
catch (AMQConnectionException e)
@@ -339,7 +358,7 @@ public class AMQMinaProtocolSession implements AMQProtocolSession,
}
closeSession();
_stateManager.changeState(AMQState.CONNECTION_CLOSING);
- writeFrame(e.getCloseFrame(frame.getChannel()));
+ writeFrame(e.getCloseFrame(channelId));
}
}
catch (Exception e)
@@ -353,61 +372,21 @@ public class AMQMinaProtocolSession implements AMQProtocolSession,
}
}
- private void contentFrameReceived(AMQFrame frame) throws AMQException
- {
- if (frame.getBodyFrame() instanceof ContentHeaderBody)
- {
- contentHeaderReceived(frame);
- }
- else if (frame.getBodyFrame() instanceof ContentBody)
- {
- contentBodyReceived(frame);
- }
- else if (frame.getBodyFrame() instanceof HeartbeatBody)
- {
- _logger.debug("Received heartbeat from client");
- }
- else
- {
- _logger.warn("Unrecognised frame " + frame.getClass().getName());
- }
- }
- private void contentHeaderReceived(AMQFrame frame) throws AMQException
+ private void contentHeaderReceived(int channelId, ContentHeaderBody body) throws AMQException
{
- if (_logger.isDebugEnabled())
- {
- _logger.debug("Content header frame received: " + frame);
- }
- AMQChannel channel = getChannel(frame.getChannel());
+ AMQChannel channel = getAndAssertChannel(channelId);
+
+ channel.publishContentHeader(body);
- if (channel == null)
- {
- throw new AMQException(AMQConstant.NOT_FOUND, "Channel not found with id:" + frame.getChannel());
- }
- else
- {
- channel.publishContentHeader((ContentHeaderBody) frame.getBodyFrame());
- }
}
- private void contentBodyReceived(AMQFrame frame) throws AMQException
+ private void contentBodyReceived(int channelId, ContentBody body) throws AMQException
{
- if (_logger.isDebugEnabled())
- {
- _logger.debug("Content body frame received: " + frame);
- }
- AMQChannel channel = getChannel(frame.getChannel());
+ AMQChannel channel = getAndAssertChannel(channelId);
- if (channel == null)
- {
- throw new AMQException(AMQConstant.NOT_FOUND, "Channel not found with id:" + frame.getChannel());
- }
- else
- {
- channel.publishContentBody((ContentBody) frame.getBodyFrame(), this);
- }
+ channel.publishContentBody(body, this);
}
/**
@@ -437,6 +416,16 @@ public class AMQMinaProtocolSession implements AMQProtocolSession,
return new ArrayList<AMQChannel>(_channelMap.values());
}
+ public AMQChannel getAndAssertChannel(int channelId) throws AMQException
+ {
+ AMQChannel channel = getChannel(channelId);
+ if (channel == null)
+ {
+ throw new AMQException(AMQConstant.NOT_FOUND, "Channel not found with id:" + channelId);
+ }
+ return channel;
+ }
+
public AMQChannel getChannel(int channelId) throws AMQException
{
if (channelAwaitingClosure(channelId))
@@ -685,24 +674,26 @@ public class AMQMinaProtocolSession implements AMQProtocolSession,
private void setProtocolVersion(byte major, byte minor)
{
- _major = major;
- _minor = minor;
- _registry = MainRegistry.getVersionSpecificRegistry(major, minor);
+ _protocolVersion = new ProtocolVersion(major,minor);
+
+ _registry = MainRegistry.getVersionSpecificRegistry(_protocolVersion);
+
+ _protocolOutputConverter = ProtocolOutputConverterRegistry.getConverter(this);
}
public byte getProtocolMajorVersion()
{
- return _major;
+ return _protocolVersion.getMajorVersion();
}
public byte getProtocolMinorVersion()
{
- return _minor;
+ return _protocolVersion.getMinorVersion();
}
public boolean isProtocolVersion(byte major, byte minor)
{
- return _major == major && _minor == minor;
+ return getProtocolMajorVersion() == major && getProtocolMinorVersion() == minor;
}
public VersionSpecificRegistry getRegistry()
@@ -739,5 +730,10 @@ public class AMQMinaProtocolSession implements AMQProtocolSession,
_taskList.remove(task);
}
+ public ProtocolOutputConverter getProtocolOutputConverter()
+ {
+ return _protocolOutputConverter;
+ }
+
}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQNoMethodHandlerException.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQNoMethodHandlerException.java
new file mode 100644
index 0000000000..16d74b6fc0
--- /dev/null
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQNoMethodHandlerException.java
@@ -0,0 +1,34 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.qpid.server.protocol;
+
+import org.apache.qpid.framing.AMQMethodBody;
+import org.apache.qpid.protocol.AMQMethodEvent;
+import org.apache.qpid.AMQException;
+
+public class AMQNoMethodHandlerException extends AMQException
+{
+
+ public AMQNoMethodHandlerException(AMQMethodEvent<AMQMethodBody> evt)
+ {
+ super("AMQMethodEvent " + evt + " was not processed by any listener on Broker.");
+ }
+}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQPFastProtocolHandler.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQPFastProtocolHandler.java
index 9d397505dc..756a8b5ebe 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQPFastProtocolHandler.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQPFastProtocolHandler.java
@@ -39,7 +39,7 @@ import org.apache.qpid.framing.AMQShortString;
import org.apache.qpid.framing.ConnectionCloseBody;
import org.apache.qpid.framing.HeartbeatBody;
import org.apache.qpid.framing.ProtocolInitiation;
-import org.apache.qpid.framing.ProtocolVersionList;
+import org.apache.qpid.framing.ProtocolVersion;
import org.apache.qpid.server.registry.ApplicationRegistry;
import org.apache.qpid.server.registry.IApplicationRegistry;
import org.apache.qpid.server.transport.ConnectorConfiguration;
@@ -53,7 +53,7 @@ import org.apache.qpid.ssl.SSLContextFactory;
* the state for the connection.
*
*/
-public class AMQPFastProtocolHandler extends IoHandlerAdapter implements ProtocolVersionList
+public class AMQPFastProtocolHandler extends IoHandlerAdapter
{
private static final Logger _logger = Logger.getLogger(AMQPFastProtocolHandler.class);
@@ -162,12 +162,11 @@ public class AMQPFastProtocolHandler extends IoHandlerAdapter implements Protoco
AMQProtocolSession session = AMQMinaProtocolSession.getAMQProtocolSession(protocolSession);
if (throwable instanceof AMQProtocolHeaderException)
{
- /* Find last protocol version in protocol version list. Make sure last protocol version
- listed in the build file (build-module.xml) is the latest version which will be returned
- here. */
- int i = pv.length - 1;
- protocolSession.write(new ProtocolInitiation(pv[i][PROTOCOL_MAJOR], pv[i][PROTOCOL_MINOR]));
+
+ protocolSession.write(new ProtocolInitiation(ProtocolVersion.getLatestSupportedVersion()));
+
protocolSession.close();
+
_logger.error("Error in protocol initiation " + session + ": " + throwable.getMessage(), throwable);
}
else if(throwable instanceof IOException)
@@ -176,8 +175,6 @@ public class AMQPFastProtocolHandler extends IoHandlerAdapter implements Protoco
}
else
{
- // AMQP version change: Hardwire the version to 0-8 (major=8, minor=0)
- // TODO: Connect this to the session version obtained from ProtocolInitiation for this session.
// Be aware of possible changes to parameter order as versions change.
protocolSession.write(ConnectionCloseBody.createAMQFrame(0,
session.getProtocolMajorVersion(),
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSession.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSession.java
index 503dc8b554..4cfee06850 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSession.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSession.java
@@ -28,6 +28,7 @@ import org.apache.qpid.framing.AMQShortString;
import org.apache.qpid.framing.FieldTable;
import org.apache.qpid.protocol.AMQVersionAwareProtocolSession;
import org.apache.qpid.server.AMQChannel;
+import org.apache.qpid.server.output.ProtocolOutputConverter;
import org.apache.qpid.server.virtualhost.VirtualHost;
@@ -162,4 +163,6 @@ public interface AMQProtocolSession extends AMQVersionAwareProtocolSession
void removeSessionCloseTask(Task task);
+ public ProtocolOutputConverter getProtocolOutputConverter();
+
}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSessionMBean.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSessionMBean.java
index ea89136a62..d2a20cdf57 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSessionMBean.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/AMQProtocolSessionMBean.java
@@ -24,6 +24,7 @@ import javax.management.JMException;
import javax.management.MBeanException;
import javax.management.MBeanNotificationInfo;
import javax.management.Notification;
+import javax.management.NotCompliantMBeanException;
import javax.management.monitor.MonitorNotification;
import javax.management.openmbean.CompositeData;
import javax.management.openmbean.CompositeDataSupport;
@@ -65,7 +66,7 @@ public class AMQProtocolSessionMBean extends AMQManagedObject implements Managed
new AMQShortString("Broker Management Console has closed the connection.");
@MBeanConstructor("Creates an MBean exposing an AMQ Broker Connection")
- public AMQProtocolSessionMBean(AMQMinaProtocolSession session) throws JMException
+ public AMQProtocolSessionMBean(AMQMinaProtocolSession session) throws NotCompliantMBeanException, OpenDataException
{
super(ManagedConnection.class, ManagedConnection.TYPE);
_session = session;
@@ -74,6 +75,8 @@ public class AMQProtocolSessionMBean extends AMQManagedObject implements Managed
_name = jmxEncode(new StringBuffer(remote), 0).toString();
init();
}
+
+
static
{
try
@@ -94,7 +97,7 @@ public class AMQProtocolSessionMBean extends AMQManagedObject implements Managed
{
_channelType = new CompositeType("Channel", "Channel Details", _channelAtttibuteNames,
- _channelAtttibuteNames, _channelAttributeTypes);
+ _channelAtttibuteNames, _channelAttributeTypes);
_channelsType = new TabularType("Channels", "Channels", _channelType, _indexNames);
}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/UnknnownMessageTypeException.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/UnknnownMessageTypeException.java
new file mode 100644
index 0000000000..45d09e8f3e
--- /dev/null
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/protocol/UnknnownMessageTypeException.java
@@ -0,0 +1,33 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.qpid.server.protocol;
+
+import org.apache.qpid.framing.AMQDataBlock;
+import org.apache.qpid.AMQException;
+
+public class UnknnownMessageTypeException extends AMQException
+{
+ public UnknnownMessageTypeException(AMQDataBlock message)
+ {
+ super("Unknown message type: " + message.getClass().getName() + ": " + message);
+
+ }
+}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQMessage.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQMessage.java
index aa7ea16afc..dedea68d18 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQMessage.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQMessage.java
@@ -20,23 +20,34 @@
*/
package org.apache.qpid.server.queue;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.log4j.Logger;
-import org.apache.mina.common.ByteBuffer;
import org.apache.qpid.AMQException;
-import org.apache.qpid.framing.*;
+import org.apache.qpid.framing.AMQBody;
+import org.apache.qpid.framing.AMQDataBlock;
+import org.apache.qpid.framing.AMQFrame;
+import org.apache.qpid.framing.ContentHeaderBody;
+import org.apache.qpid.framing.abstraction.ContentChunk;
+import org.apache.qpid.framing.abstraction.MessagePublishInfo;
+import org.apache.qpid.framing.abstraction.ProtocolVersionMethodConverter;
import org.apache.qpid.server.protocol.AMQProtocolSession;
import org.apache.qpid.server.store.MessageStore;
import org.apache.qpid.server.store.StoreContext;
-import org.apache.qpid.framing.abstraction.MessagePublishInfo;
-import org.apache.qpid.framing.abstraction.ProtocolVersionMethodConverter;
-import org.apache.qpid.framing.abstraction.ContentChunk;
import org.apache.qpid.server.txn.TransactionalContext;
/** Combines the information that make up a deliverable message into a more manageable form. */
+
+import org.apache.log4j.Logger;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Combines the information that make up a deliverable message into a more manageable form.
+ */
public class AMQMessage
{
private static final Logger _log = Logger.getLogger(AMQMessage.class);
@@ -136,7 +147,7 @@ public class AMQMessage
}
}
- private StoreContext getStoreContext()
+ public StoreContext getStoreContext()
{
return _txnContext.getStoreContext();
}
@@ -579,6 +590,7 @@ public class AMQMessage
}
}
+/*
public void writeDeliver(AMQProtocolSession protocolSession, int channelId, long deliveryTag, AMQShortString consumerTag)
throws AMQException
{
@@ -746,6 +758,12 @@ public class AMQMessage
protocolSession.writeFrame(bodyFrameIterator.next());
}
}
+*/
+
+ public AMQMessageHandle getMessageHandle()
+ {
+ return _messageHandle;
+ }
public long getSize()
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQQueueMBean.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQQueueMBean.java
index 4fd89f39da..c9329a244c 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQQueueMBean.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/AMQQueueMBean.java
@@ -41,9 +41,9 @@ import javax.management.openmbean.TabularType;
import org.apache.log4j.Logger;
import org.apache.mina.common.ByteBuffer;
import org.apache.qpid.AMQException;
-import org.apache.qpid.framing.BasicContentHeaderProperties;
-import org.apache.qpid.framing.ContentBody;
+import org.apache.qpid.framing.CommonContentHeaderProperties;
import org.apache.qpid.framing.ContentHeaderBody;
+import org.apache.qpid.framing.AMQShortString;
import org.apache.qpid.framing.abstraction.ContentChunk;
import org.apache.qpid.server.management.AMQManagedObject;
import org.apache.qpid.server.management.MBeanConstructor;
@@ -344,12 +344,13 @@ public class AMQQueueMBean extends AMQManagedObject implements ManagedQueue, Que
try
{
// Create header attributes list
- BasicContentHeaderProperties headerProperties = (BasicContentHeaderProperties) msg.getContentHeaderBody().properties;
+ CommonContentHeaderProperties headerProperties = (CommonContentHeaderProperties) msg.getContentHeaderBody().properties;
String mimeType = null, encoding = null;
if (headerProperties != null)
{
- mimeType = headerProperties.getContentType();
- encoding = headerProperties.getEncoding() == null ? "" : headerProperties.getEncoding();
+ AMQShortString mimeTypeShortSting = headerProperties.getContentType();
+ mimeType = mimeTypeShortSting == null ? null : mimeTypeShortSting.toString();
+ encoding = headerProperties.getEncoding() == null ? "" : headerProperties.getEncoding().toString();
}
Object[] itemValues = {msgId, mimeType, encoding, msgContent.toArray(new Byte[0])};
return new CompositeDataSupport(_msgContentType, _msgContentAttributes, itemValues);
@@ -382,7 +383,7 @@ public class AMQQueueMBean extends AMQManagedObject implements ManagedQueue, Que
AMQMessage msg = list.get(i - 1);
ContentHeaderBody headerBody = msg.getContentHeaderBody();
// Create header attributes list
- BasicContentHeaderProperties headerProperties = (BasicContentHeaderProperties) headerBody.properties;
+ CommonContentHeaderProperties headerProperties = (CommonContentHeaderProperties) headerBody.properties;
String[] headerAttributes = headerProperties.toString().split(",");
Object[] itemValues = {msg.getMessageId(), headerAttributes, headerBody.bodySize, msg.isRedelivered()};
CompositeData messageData = new CompositeDataSupport(_messageDataType, _msgAttributeNames, itemValues);
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/ConcurrentSelectorDeliveryManager.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/ConcurrentSelectorDeliveryManager.java
index 208a59516c..e70926736d 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/ConcurrentSelectorDeliveryManager.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/ConcurrentSelectorDeliveryManager.java
@@ -270,7 +270,8 @@ public class ConcurrentSelectorDeliveryManager implements DeliveryManager
channel.addUnacknowledgedMessage(msg, deliveryTag, null, _queue);
}
- msg.writeGetOk(protocolSession, channel.getChannelId(), deliveryTag, _queue.getMessageCount());
+ protocolSession.getProtocolOutputConverter().writeGetOk(msg, channel.getChannelId(),
+ deliveryTag, _queue.getMessageCount());
_totalMessageSize.addAndGet(-msg.getSize());
}
}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/SubscriptionImpl.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/SubscriptionImpl.java
index ede7731a06..0a2e73880c 100644
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/SubscriptionImpl.java
+++ b/qpid/java/broker/src/main/java/org/apache/qpid/server/queue/SubscriptionImpl.java
@@ -29,9 +29,9 @@ import org.apache.qpid.protocol.AMQConstant;
import org.apache.qpid.common.AMQPFilterTypes;
import org.apache.qpid.common.ClientProperties;
import org.apache.qpid.framing.AMQShortString;
-import org.apache.qpid.framing.BasicCancelOkBody;
import org.apache.qpid.framing.FieldTable;
import org.apache.qpid.server.AMQChannel;
+import org.apache.qpid.server.output.ProtocolOutputConverter;
import org.apache.qpid.server.filter.FilterManager;
import org.apache.qpid.server.filter.FilterManagerFactory;
import org.apache.qpid.server.protocol.AMQProtocolSession;
@@ -258,7 +258,7 @@ public class SubscriptionImpl implements Subscription
{
channel.addUnacknowledgedBrowsedMessage(msg, deliveryTag, consumerTag, queue);
}
- msg.writeDeliver(protocolSession, channel.getChannelId(), deliveryTag, consumerTag);
+ protocolSession.getProtocolOutputConverter().writeDeliver(msg, channel.getChannelId(), deliveryTag, consumerTag);
}
}
@@ -294,7 +294,7 @@ public class SubscriptionImpl implements Subscription
msg.decrementReference(storeContext);
}
- msg.writeDeliver(protocolSession, channel.getChannelId(), deliveryTag, consumerTag);
+ protocolSession.getProtocolOutputConverter().writeDeliver(msg, channel.getChannelId(), deliveryTag, consumerTag);
}
}
@@ -466,13 +466,9 @@ public class SubscriptionImpl implements Subscription
if (_autoClose && !_sentClose)
{
_logger.info("Closing autoclose subscription:" + this);
- // AMQP version change: Hardwire the version to 0-8 (major=8, minor=0)
- // TODO: Connect this to the session version obtained from ProtocolInitiation for this session.
- // Be aware of possible changes to parameter order as versions change.
- protocolSession.writeFrame(BasicCancelOkBody.createAMQFrame(channel.getChannelId(),
- (byte) 8, (byte) 0, // AMQP version (major, minor)
- consumerTag // consumerTag
- ));
+ ProtocolOutputConverter converter = protocolSession.getProtocolOutputConverter();
+ converter.confirmConsumerAutoClose(channel.getChannelId(), consumerTag);
+
_sentClose = true;
}
}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/store/ContentChunkAdapter.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/store/ContentChunkAdapter.java
deleted file mode 100644
index 90aa7bb998..0000000000
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/store/ContentChunkAdapter.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- *
- */
-package org.apache.qpid.server.store;
-
-import org.apache.qpid.framing.ContentBody;
-import org.apache.qpid.framing.abstraction.ContentChunk;
-
-import org.apache.mina.common.ByteBuffer;
-
-public class ContentChunkAdapter
-{
- public static ContentBody toConentBody(ContentChunk contentBodyChunk)
- {
- return new ContentBody(contentBodyChunk.getData());
- }
-
- public static ContentChunk toConentChunk(final ContentBody contentBodyChunk)
- {
- return new ContentChunk() {
-
- public int getSize()
- {
- return contentBodyChunk.getSize();
- }
-
- public ByteBuffer getData()
- {
- return contentBodyChunk.payload;
- }
-
- public void reduceToFit()
- {
- contentBodyChunk.reduceBufferToFit();
- }
- };
-
- }
-
-}
diff --git a/qpid/java/broker/src/main/java/org/apache/qpid/server/store/MessagePublishInfoAdapter.java b/qpid/java/broker/src/main/java/org/apache/qpid/server/store/MessagePublishInfoAdapter.java
deleted file mode 100644
index 6ee2fa784d..0000000000
--- a/qpid/java/broker/src/main/java/org/apache/qpid/server/store/MessagePublishInfoAdapter.java
+++ /dev/null
@@ -1,62 +0,0 @@
-package org.apache.qpid.server.store;
-
-import org.apache.qpid.framing.AMQShortString;
-import org.apache.qpid.framing.BasicPublishBody;
-import org.apache.qpid.framing.abstraction.MessagePublishInfo;
-
-public class MessagePublishInfoAdapter
-{
- private final byte _majorVersion;
- private final byte _minorVersion;
- private final int _classId;
- private final int _methodId;
-
-
- public MessagePublishInfoAdapter(byte majorVersion, byte minorVersion)
- {
- _majorVersion = majorVersion;
- _minorVersion = minorVersion;
- _classId = BasicPublishBody.getClazz(majorVersion,minorVersion);
- _methodId = BasicPublishBody.getMethod(majorVersion,minorVersion);
- }
-
- public BasicPublishBody toMethodBody(MessagePublishInfo pubInfo)
- {
- return new BasicPublishBody(_majorVersion,
- _minorVersion,
- _classId,
- _methodId,
- pubInfo.getExchange(),
- pubInfo.isImmediate(),
- pubInfo.isMandatory(),
- pubInfo.getRoutingKey(),
- 0) ; // ticket
- }
-
- public MessagePublishInfo toMessagePublishInfo(final BasicPublishBody body)
- {
- return new MessagePublishInfo()
- {
-
- public AMQShortString getExchange()
- {
- return body.getExchange();
- }
-
- public boolean isImmediate()
- {
- return body.getImmediate();
- }
-
- public boolean isMandatory()
- {
- return body.getMandatory();
- }
-
- public AMQShortString getRoutingKey()
- {
- return body.getRoutingKey();
- }
- };
- }
-}