summaryrefslogtreecommitdiff
path: root/qpid/java/client/src
diff options
context:
space:
mode:
authorRobert Godfrey <rgodfrey@apache.org>2014-10-17 14:29:21 +0000
committerRobert Godfrey <rgodfrey@apache.org>2014-10-17 14:29:21 +0000
commit95fc93485ab66966713611a4e1429d917dabde64 (patch)
tree09ee31bc9462449dbcfc62379a393017c8f39843 /qpid/java/client/src
parent28dbfe8d101dd14a95b1d75e799107bdaa6e18d0 (diff)
downloadqpid-python-95fc93485ab66966713611a4e1429d917dabde64.tar.gz
QPID-6164 : Add synchronous publish capability to 0-8/9/9-1
git-svn-id: https://svn.apache.org/repos/asf/qpid/trunk@1632585 13f79535-47bb-0310-9956-ffa450edef68
Diffstat (limited to 'qpid/java/client/src')
-rw-r--r--qpid/java/client/src/main/java/org/apache/qpid/client/AMQConnectionDelegate_8_0.java66
-rw-r--r--qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java12
-rw-r--r--qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer_0_8.java85
-rw-r--r--qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverNoopSupport.java2
-rw-r--r--qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverRetrySupport.java8
-rw-r--r--qpid/java/client/src/main/java/org/apache/qpid/client/handler/ClientMethodDispatcherImpl.java24
-rw-r--r--qpid/java/client/src/main/java/org/apache/qpid/client/protocol/AMQProtocolHandler.java4
7 files changed, 176 insertions, 25 deletions
diff --git a/qpid/java/client/src/main/java/org/apache/qpid/client/AMQConnectionDelegate_8_0.java b/qpid/java/client/src/main/java/org/apache/qpid/client/AMQConnectionDelegate_8_0.java
index 176eb5d0c4..bcf0721aab 100644
--- a/qpid/java/client/src/main/java/org/apache/qpid/client/AMQConnectionDelegate_8_0.java
+++ b/qpid/java/client/src/main/java/org/apache/qpid/client/AMQConnectionDelegate_8_0.java
@@ -46,6 +46,8 @@ import org.apache.qpid.common.ServerPropertyNames;
import org.apache.qpid.configuration.ClientProperties;
import org.apache.qpid.framing.ChannelOpenBody;
import org.apache.qpid.framing.ChannelOpenOkBody;
+import org.apache.qpid.framing.ConfirmSelectBody;
+import org.apache.qpid.framing.ConfirmSelectOkBody;
import org.apache.qpid.framing.FieldTable;
import org.apache.qpid.framing.ProtocolVersion;
import org.apache.qpid.framing.TxSelectBody;
@@ -68,6 +70,8 @@ public class AMQConnectionDelegate_8_0 implements AMQConnectionDelegate
private final AMQConnection _conn;
private boolean _messageCompressionSupported;
private boolean _addrSyntaxSupported;
+ private boolean _confirmedPublishSupported;
+ private boolean _confirmedPublishNonTransactionalSupported;
public void closeConnection(long timeout) throws JMSException, AMQException
{
@@ -94,6 +98,11 @@ public class AMQConnectionDelegate_8_0 implements AMQConnectionDelegate
return ((cause instanceof ConnectException) || (cause instanceof UnresolvedAddressException));
}
+ public boolean isConfirmedPublishSupported()
+ {
+ return _confirmedPublishSupported;
+ }
+
public ProtocolVersion makeBrokerConnection(BrokerDetails brokerDetail) throws AMQException, IOException
{
if (_logger.isDebugEnabled())
@@ -146,6 +155,8 @@ public class AMQConnectionDelegate_8_0 implements AMQConnectionDelegate
_conn.setConnected(true);
_conn.logConnected(network.getLocalAddress(), network.getRemoteAddress());
_messageCompressionSupported = checkMessageCompressionSupported();
+ _confirmedPublishSupported = checkConfirmedPublishSupported();
+ _confirmedPublishNonTransactionalSupported = checkConfirmedPublishNonTransactionalSupported();
return null;
}
else
@@ -155,6 +166,32 @@ public class AMQConnectionDelegate_8_0 implements AMQConnectionDelegate
}
+ // RabbitMQ supports confirmed publishing, but only on non transactional sessions
+ private boolean checkConfirmedPublishNonTransactionalSupported()
+ {
+ FieldTable serverProperties = _conn.getProtocolHandler().getProtocolSession().getConnectionStartServerProperties();
+ if( serverProperties != null
+ && serverProperties.containsKey("capabilities")
+ && serverProperties.get("capabilities") instanceof FieldTable)
+ {
+ FieldTable capabilities = serverProperties.getFieldTable("capabilities");
+ if(capabilities.containsKey("publisher_confirms")
+ && capabilities.get("publisher_confirms") instanceof Boolean
+ && capabilities.getBoolean("publisher_confirms"))
+ {
+ return true;
+ }
+ else
+ {
+ return false;
+ }
+ }
+ else
+ {
+ return false;
+ }
+ }
+
public org.apache.qpid.jms.Session createSession(final boolean transacted, final int acknowledgeMode, final int prefetch)
throws JMSException
{
@@ -266,9 +303,21 @@ public class AMQConnectionDelegate_8_0 implements AMQConnectionDelegate
}
TxSelectBody body = _conn.getProtocolHandler().getMethodRegistry().createTxSelectBody();
- // TODO: Be aware of possible changes to parameter order as versions change.
+
_conn.getProtocolHandler().syncWrite(body.generateFrame(channelId), TxSelectOkBody.class);
}
+ boolean useConfirms = (_confirmedPublishSupported || (!transacted && _confirmedPublishNonTransactionalSupported))
+ && "all".equals(_conn.getSyncPublish());
+ if(useConfirms)
+ {
+ if (_logger.isDebugEnabled())
+ {
+ _logger.debug("Issuing ConfirmSelect for " + channelId);
+ }
+ ConfirmSelectBody body = new ConfirmSelectBody(false);
+
+ _conn.getProtocolHandler().syncWrite(body.generateFrame(channelId), ConfirmSelectOkBody.class);
+ }
}
public void failoverPrep()
@@ -340,7 +389,7 @@ public class AMQConnectionDelegate_8_0 implements AMQConnectionDelegate
}
catch (IllegalStateException e)
{
- if (!(e.getMessage().startsWith("Fail-over interupted no-op failover support")))
+ if (!(e.getMessage().startsWith("Fail-over interrupted no-op failover support")))
{
throw e;
}
@@ -424,6 +473,14 @@ public class AMQConnectionDelegate_8_0 implements AMQConnectionDelegate
}
+ private boolean checkConfirmedPublishSupported()
+ {
+ FieldTable serverProperties = _conn.getProtocolHandler().getProtocolSession().getConnectionStartServerProperties();
+ return serverProperties != null
+ && Boolean.parseBoolean(serverProperties.getString(ConnectionStartProperties.QPID_CONFIRMED_PUBLISH_SUPPORTED));
+
+ }
+
public boolean isMessageCompressionSupported()
{
return _messageCompressionSupported;
@@ -433,4 +490,9 @@ public class AMQConnectionDelegate_8_0 implements AMQConnectionDelegate
{
return _addrSyntaxSupported;
}
+
+ public boolean isConfirmedPublishNonTransactionalSupported()
+ {
+ return _confirmedPublishNonTransactionalSupported;
+ }
}
diff --git a/qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java b/qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java
index 1d47ce9a07..f72d915c25 100644
--- a/qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java
+++ b/qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer.java
@@ -122,7 +122,7 @@ public abstract class BasicMessageProducer extends Closeable implements org.apac
? System.getProperty("qpid.default_mandatory")
: "false"));
- private PublishMode publishMode = PublishMode.ASYNC_PUBLISH_ALL;
+ private PublishMode _publishMode = PublishMode.ASYNC_PUBLISH_ALL;
protected BasicMessageProducer(Logger logger,AMQConnection connection, AMQDestination destination, boolean transacted, int channelId,
AMQSession session, long producerId, Boolean immediate, Boolean mandatory) throws AMQException
@@ -165,16 +165,16 @@ public abstract class BasicMessageProducer extends Closeable implements org.apac
// Support for deprecated option sync_persistence
if (syncPub.equals("persistent") || _connection.getSyncPersistence())
{
- publishMode = PublishMode.SYNC_PUBLISH_PERSISTENT;
+ _publishMode = PublishMode.SYNC_PUBLISH_PERSISTENT;
}
else if (syncPub.equals("all"))
{
- publishMode = PublishMode.SYNC_PUBLISH_ALL;
+ _publishMode = PublishMode.SYNC_PUBLISH_ALL;
}
if (_logger.isDebugEnabled())
{
- _logger.debug("MessageProducer " + toString() + " using publish mode : " + publishMode);
+ _logger.debug("MessageProducer " + toString() + " using publish mode : " + _publishMode);
}
}
@@ -720,12 +720,12 @@ public abstract class BasicMessageProducer extends Closeable implements org.apac
protected PublishMode getPublishMode()
{
- return publishMode;
+ return _publishMode;
}
protected void setPublishMode(PublishMode publishMode)
{
- this.publishMode = publishMode;
+ _publishMode = publishMode;
}
Logger getLogger()
diff --git a/qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer_0_8.java b/qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer_0_8.java
index 69d02566bf..e4c879aca8 100644
--- a/qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer_0_8.java
+++ b/qpid/java/client/src/main/java/org/apache/qpid/client/BasicMessageProducer_0_8.java
@@ -32,14 +32,19 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.qpid.AMQException;
+import org.apache.qpid.client.failover.FailoverException;
import org.apache.qpid.client.message.AMQMessageDelegate_0_8;
import org.apache.qpid.client.message.AbstractJMSMessage;
import org.apache.qpid.client.message.QpidMessageProperties;
import org.apache.qpid.client.protocol.AMQProtocolHandler;
+import org.apache.qpid.client.protocol.BlockingMethodFrameListener;
import org.apache.qpid.configuration.ClientProperties;
import org.apache.qpid.framing.AMQFrame;
+import org.apache.qpid.framing.AMQMethodBody;
import org.apache.qpid.framing.AMQShortString;
+import org.apache.qpid.framing.BasicAckBody;
import org.apache.qpid.framing.BasicContentHeaderProperties;
+import org.apache.qpid.framing.BasicNackBody;
import org.apache.qpid.framing.BasicPublishBody;
import org.apache.qpid.framing.CompositeAMQDataBlock;
import org.apache.qpid.framing.ContentBody;
@@ -211,9 +216,6 @@ public class BasicMessageProducer_0_8 extends BasicMessageProducer
}
- // TODO: This is a hacky way of getting the AMQP class-id for the Basic class
- int classIfForBasic = getSession().getMethodRegistry().createBasicQosOkBody().getClazz();
-
AMQFrame contentHeaderFrame =
ContentHeaderBody.createAMQFrame(getChannelId(),
contentHeaderProperties, size);
@@ -232,7 +234,7 @@ public class BasicMessageProducer_0_8 extends BasicMessageProducer
frames[0] = publishFrame;
frames[1] = contentHeaderFrame;
- CompositeAMQDataBlock compositeFrame = new CompositeAMQDataBlock(frames);
+ final CompositeAMQDataBlock compositeFrame = new CompositeAMQDataBlock(frames);
try
{
@@ -246,7 +248,40 @@ public class BasicMessageProducer_0_8 extends BasicMessageProducer
throw jmse;
}
- getConnection().getProtocolHandler().writeFrame(compositeFrame);
+ AMQConnectionDelegate_8_0 connectionDelegate80 = (AMQConnectionDelegate_8_0) (getConnection().getDelegate());
+
+ boolean useConfirms = getPublishMode() == PublishMode.SYNC_PUBLISH_ALL
+ && (connectionDelegate80.isConfirmedPublishSupported()
+ || (!getSession().isTransacted() && connectionDelegate80.isConfirmedPublishNonTransactionalSupported()));
+
+ if(!useConfirms)
+ {
+ getConnection().getProtocolHandler().writeFrame(compositeFrame);
+ }
+ else
+ {
+ final PublishConfirmMessageListener frameListener = new PublishConfirmMessageListener(getChannelId());
+ try
+ {
+
+ getConnection().getProtocolHandler().writeCommandFrameAndWaitForReply(compositeFrame,
+ frameListener);
+
+ if(frameListener.isRejected())
+ {
+ throw new JMSException("The message was not accepted by the server (e.g. because the address was no longer valid)");
+ }
+ }
+ catch (AMQException e)
+ {
+ throw new JMSAMQException(e);
+ }
+ catch (FailoverException e)
+ {
+ throw new JMSAMQException("Fail-over interrupted send. Status of the send is uncertain.", e);
+
+ }
+ }
}
/**
@@ -290,7 +325,7 @@ public class BasicMessageProducer_0_8 extends BasicMessageProducer
private int calculateContentBodyFrameCount(ByteBuffer payload)
{
- // we substract one from the total frame maximum size to account for the end of frame marker in a body frame
+ // we subtract one from the total frame maximum size to account for the end of frame marker in a body frame
// (0xCE byte).
int frameCount;
if ((payload == null) || (payload.remaining() == 0))
@@ -313,4 +348,42 @@ public class BasicMessageProducer_0_8 extends BasicMessageProducer
{
return (AMQSession_0_8) super.getSession();
}
+
+ private static class PublishConfirmMessageListener extends BlockingMethodFrameListener
+ {
+ private boolean _rejected;
+
+ /**
+ * Creates a new method listener, that filters incoming method to just those that match the specified channel id.
+ *
+ * @param channelId The channel id to filter incoming methods with.
+ */
+ public PublishConfirmMessageListener(final int channelId)
+ {
+ super(channelId);
+ }
+
+ @Override
+ public boolean processMethod(final int channelId, final AMQMethodBody frame)
+ {
+ if (frame instanceof BasicAckBody)
+ {
+ return true;
+ }
+ else if (frame instanceof BasicNackBody)
+ {
+ _rejected = true;
+ return true;
+ }
+ else
+ {
+ return false;
+ }
+ }
+
+ public boolean isRejected()
+ {
+ return _rejected;
+ }
+ }
}
diff --git a/qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverNoopSupport.java b/qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverNoopSupport.java
index a69e808880..da17bedcfd 100644
--- a/qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverNoopSupport.java
+++ b/qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverNoopSupport.java
@@ -68,7 +68,7 @@ public class FailoverNoopSupport<T, E extends Exception> implements FailoverSupp
}
catch (FailoverException e)
{
- throw new IllegalStateException("Fail-over interupted no-op failover support. "
+ throw new IllegalStateException("Fail-over interrupted no-op failover support. "
+ "No-op support should only be used where the caller is certain fail-over cannot occur.", e);
}
}
diff --git a/qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverRetrySupport.java b/qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverRetrySupport.java
index ffe0baecd8..74bf9a54fd 100644
--- a/qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverRetrySupport.java
+++ b/qpid/java/client/src/main/java/org/apache/qpid/client/failover/FailoverRetrySupport.java
@@ -45,7 +45,7 @@ import org.apache.qpid.client.AMQConnection;
* <p>
* Wrapping a synchronous method in a FailoverRetrySupport will have the effect that the operation will not be
* started during fail-over, but be delayed until any current fail-over has completed. Should a fail-over process want
- * to start whilst waiting for the synchrnous reply, the FailoverRetrySupport will detect this and rety the operation
+ * to start whilst waiting for the synchronous reply, the FailoverRetrySupport will detect this and retry the operation
* until it succeeds. Synchronous methods are usually coordinated with a
* {@link org.apache.qpid.client.protocol.BlockingMethodFrameListener} which is notified when a fail-over process wants
* to start and throws a FailoverException in response to this.
@@ -53,12 +53,6 @@ import org.apache.qpid.client.AMQConnection;
* Wrapping an asynchronous method in a FailoverRetrySupport will have the effect that the operation will not be
* started during fail-over, but be delayed until any current fail-over has completed.
* <p>
- * TODO Another continuation. Could use an interface Continuation (as described in other todos)
- * Then have a wrapping continuation (this), which blocks on an arbitrary
- * Condition or Latch (specified in constructor call), that this blocks on before calling the wrapped Continuation.
- * Must work on Java 1.4, so check retrotranslator works on Lock/Condition or latch first. Argument and return type
- * to match wrapped condition as type parameters. Rename to AsyncConditionalContinuation or something like that.
- * <p>
* TODO InterruptedException not handled well.
*/
public class FailoverRetrySupport<T, E extends Exception> implements FailoverSupport<T, E>
diff --git a/qpid/java/client/src/main/java/org/apache/qpid/client/handler/ClientMethodDispatcherImpl.java b/qpid/java/client/src/main/java/org/apache/qpid/client/handler/ClientMethodDispatcherImpl.java
index e6eb2d814f..de2f2f52a9 100644
--- a/qpid/java/client/src/main/java/org/apache/qpid/client/handler/ClientMethodDispatcherImpl.java
+++ b/qpid/java/client/src/main/java/org/apache/qpid/client/handler/ClientMethodDispatcherImpl.java
@@ -53,6 +53,7 @@ public class ClientMethodDispatcherImpl implements MethodDispatcher
private static final Logger _logger = LoggerFactory.getLogger(ClientMethodDispatcherImpl.class);
+
private static interface DispatcherFactory
{
public ClientMethodDispatcherImpl createMethodDispatcher(AMQProtocolSession session);
@@ -147,6 +148,13 @@ public class ClientMethodDispatcherImpl implements MethodDispatcher
return false;
}
+ @Override
+ public boolean dispatchConfirmSelectOk(final ConfirmSelectOkBody confirmSelectOkBody, final int channelId)
+ throws AMQException
+ {
+ return false;
+ }
+
public boolean dispatchBasicCancelOk(BasicCancelOkBody body, int channelId) throws AMQException
{
_basicCancelOkMethodHandler.methodReceived(_session, body, channelId);
@@ -271,11 +279,19 @@ public class ClientMethodDispatcherImpl implements MethodDispatcher
throw new AMQMethodNotImplementedException(body);
}
+ @Override
public boolean dispatchBasicAck(BasicAckBody body, int channelId) throws AMQException
{
- throw new AMQMethodNotImplementedException(body);
+ return false;
}
+ @Override
+ public boolean dispatchBasicNack(final BasicNackBody basicNackBody, final int channelId)
+ {
+ return false;
+ }
+
+
public boolean dispatchBasicCancel(BasicCancelBody body, int channelId) throws AMQException
{
throw new AMQMethodNotImplementedException(body);
@@ -400,6 +416,12 @@ public class ClientMethodDispatcherImpl implements MethodDispatcher
return false;
}
+ @Override
+ public boolean dispatchConfirmSelect(final ConfirmSelectBody body, final int channelId) throws AMQException
+ {
+ throw new AMQMethodNotImplementedException(body);
+ }
+
public boolean dispatchExchangeBoundOk(ExchangeBoundOkBody body, int channelId) throws AMQException
{
_exchangeBoundOkMethodHandler.methodReceived(_session, body, channelId);
diff --git a/qpid/java/client/src/main/java/org/apache/qpid/client/protocol/AMQProtocolHandler.java b/qpid/java/client/src/main/java/org/apache/qpid/client/protocol/AMQProtocolHandler.java
index bb98c0abbd..4886eabb90 100644
--- a/qpid/java/client/src/main/java/org/apache/qpid/client/protocol/AMQProtocolHandler.java
+++ b/qpid/java/client/src/main/java/org/apache/qpid/client/protocol/AMQProtocolHandler.java
@@ -661,7 +661,7 @@ public class AMQProtocolHandler implements ProtocolEngine
* @param frame
* @param listener the blocking listener. Note the calling thread will block.
*/
- public AMQMethodEvent writeCommandFrameAndWaitForReply(AMQFrame frame, BlockingMethodFrameListener listener)
+ public AMQMethodEvent writeCommandFrameAndWaitForReply(AMQDataBlock frame, BlockingMethodFrameListener listener)
throws AMQException, FailoverException
{
return writeCommandFrameAndWaitForReply(frame, listener, DEFAULT_SYNC_TIMEOUT);
@@ -674,7 +674,7 @@ public class AMQProtocolHandler implements ProtocolEngine
* @param frame
* @param listener the blocking listener. Note the calling thread will block.
*/
- public AMQMethodEvent writeCommandFrameAndWaitForReply(AMQFrame frame, BlockingMethodFrameListener listener,
+ public AMQMethodEvent writeCommandFrameAndWaitForReply(AMQDataBlock frame, BlockingMethodFrameListener listener,
long timeout) throws AMQException, FailoverException
{
try