You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by cl...@apache.org on 2015/07/14 15:53:46 UTC

[1/3] activemq-artemis git commit: Create versioned instead of generic frames from STOMP decoder

Repository: activemq-artemis
Updated Branches:
  refs/heads/master f17991e58 -> 4167c5f2c


Create versioned instead of generic frames from STOMP decoder


Project: http://git-wip-us.apache.org/repos/asf/activemq-artemis/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq-artemis/commit/3deb20f0
Tree: http://git-wip-us.apache.org/repos/asf/activemq-artemis/tree/3deb20f0
Diff: http://git-wip-us.apache.org/repos/asf/activemq-artemis/diff/3deb20f0

Branch: refs/heads/master
Commit: 3deb20f049bec05c4c559b3654c36989924e643d
Parents: f17991e
Author: Ville Skyttä <vi...@iki.fi>
Authored: Tue Jul 14 11:53:14 2015 +0300
Committer: Clebert Suconic <cl...@apache.org>
Committed: Tue Jul 14 09:53:35 2015 -0400

----------------------------------------------------------------------
 .../activemq/artemis/core/protocol/stomp/StompDecoder.java  | 9 +++++++--
 .../activemq/artemis/core/protocol/stomp/StompFrame.java    | 2 +-
 .../core/protocol/stomp/v10/StompFrameHandlerV10.java       | 2 +-
 .../core/protocol/stomp/v11/StompFrameHandlerV11.java       | 7 ++++++-
 .../core/protocol/stomp/v12/StompFrameHandlerV12.java       | 8 +++++---
 5 files changed, 20 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3deb20f0/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java
index 19e7dda..3bbe88f 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java
@@ -162,8 +162,11 @@ public class StompDecoder
    //max len of EOL (default is 1 for '\n')
    protected int eolLen = 1;
 
-   public StompDecoder()
+   private final VersionedStompFrameHandler handler;
+
+   public StompDecoder(VersionedStompFrameHandler handler)
    {
+      this.handler = handler;
    }
 
    public boolean hasBytes()
@@ -276,7 +279,9 @@ public class StompDecoder
 
          // reset
 
-         StompFrame ret = new StompFrame(command, headers, content);
+         StompFrame ret = handler.createStompFrame(command);
+         ret.headers = headers;
+         ret.setByteBody(content);
 
          init();
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3deb20f0/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompFrame.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompFrame.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompFrame.java
index 491c6a9..cb6ede5 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompFrame.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompFrame.java
@@ -33,7 +33,7 @@ public class StompFrame
 
    protected final String command;
 
-   protected final Map<String, String> headers;
+   protected Map<String, String> headers;
 
    private String body;
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3deb20f0/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
index 7cbf771..e492060 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
@@ -35,7 +35,7 @@ public class StompFrameHandlerV10 extends VersionedStompFrameHandler implements
    public StompFrameHandlerV10(StompConnection connection)
    {
       super(connection);
-      decoder = new StompDecoder();
+      decoder = new StompDecoder(this);
       decoder.init();
       connection.addStompEventListener(this);
    }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3deb20f0/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
index 656ed8d..e30742e 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
@@ -41,7 +41,7 @@ public class StompFrameHandlerV11 extends VersionedStompFrameHandler implements
    {
       super(connection);
       connection.addStompEventListener(this);
-      decoder = new StompDecoderV11();
+      decoder = new StompDecoderV11(this);
       decoder.init();
    }
 
@@ -428,6 +428,11 @@ public class StompFrameHandlerV11 extends VersionedStompFrameHandler implements
       protected boolean isEscaping = false;
       protected SimpleBytes holder = new SimpleBytes(1024);
 
+      public StompDecoderV11(StompFrameHandlerV11 handler)
+      {
+         super(handler);
+      }
+
       @Override
       public void init(StompDecoder decoder)
       {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/3deb20f0/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java
index ca9b6fe..b1632e7 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java
@@ -35,7 +35,7 @@ public class StompFrameHandlerV12 extends StompFrameHandlerV11 implements FrameE
    public StompFrameHandlerV12(StompConnection connection)
    {
       super(connection);
-      decoder = new StompDecoderV12();
+      decoder = new StompDecoderV12(this);
       decoder.init();
    }
 
@@ -99,9 +99,11 @@ public class StompFrameHandlerV12 extends StompFrameHandlerV11 implements FrameE
    {
       protected boolean nextEOLChar = false;
 
-      public StompDecoderV12()
+      public StompDecoderV12(StompFrameHandlerV12 handler)
       {
-         //1.2 allow '\r\n'
+         super(handler);
+
+         //1.2 allows '\r\n'
          eolLen = 2;
       }
 


[2/3] activemq-artemis git commit: Set frame handlers to ActiveMQStompExceptions

Posted by cl...@apache.org.
Set frame handlers to ActiveMQStompExceptions


Project: http://git-wip-us.apache.org/repos/asf/activemq-artemis/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq-artemis/commit/6ee22e7c
Tree: http://git-wip-us.apache.org/repos/asf/activemq-artemis/tree/6ee22e7c
Diff: http://git-wip-us.apache.org/repos/asf/activemq-artemis/diff/6ee22e7c

Branch: refs/heads/master
Commit: 6ee22e7c5e849c1c5b989a085fccf79da9907168
Parents: 3deb20f
Author: Ville Skyttä <vi...@iki.fi>
Authored: Tue Jul 14 12:55:41 2015 +0300
Committer: Clebert Suconic <cl...@apache.org>
Committed: Tue Jul 14 09:53:36 2015 -0400

----------------------------------------------------------------------
 .../protocol/stomp/ActiveMQStompException.java  |  3 +-
 .../core/protocol/stomp/StompConnection.java    | 34 ++++++++++----------
 .../core/protocol/stomp/StompDecoder.java       | 10 +++---
 .../protocol/stomp/StompProtocolManager.java    | 11 ++++---
 .../core/protocol/stomp/StompSession.java       |  4 +--
 .../stomp/VersionedStompFrameHandler.java       | 12 +++----
 .../stomp/v10/StompFrameHandlerV10.java         |  2 +-
 .../stomp/v11/StompFrameHandlerV11.java         | 10 +++---
 .../stomp/v12/StompFrameHandlerV12.java         |  5 ++-
 9 files changed, 46 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/6ee22e7c/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/ActiveMQStompException.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/ActiveMQStompException.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/ActiveMQStompException.java
index a42e9d4..ba75460 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/ActiveMQStompException.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/ActiveMQStompException.java
@@ -128,8 +128,9 @@ public class ActiveMQStompException extends Exception
       code = newCode;
    }
 
-   public void setHandler(VersionedStompFrameHandler frameHandler)
+   public ActiveMQStompException setHandler(VersionedStompFrameHandler frameHandler)
    {
       this.handler = frameHandler;
+      return this;
    }
 }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/6ee22e7c/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
index cf67df4..df03baf 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompConnection.java
@@ -249,7 +249,7 @@ public final class StompConnection implements RemotingConnection
 
       if (!manager.destinationExists(destination))
       {
-         throw BUNDLE.destinationNotExist(destination);
+         throw BUNDLE.destinationNotExist(destination).setHandler(frameHandler);
       }
    }
 
@@ -266,7 +266,7 @@ public final class StompConnection implements RemotingConnection
          }
          catch (Exception e)
          {
-            throw new ActiveMQStompException(e.getMessage(), e);
+            throw new ActiveMQStompException(e.getMessage(), e).setHandler(frameHandler);
          }
          autoCreated = true;
       }
@@ -490,7 +490,7 @@ public final class StompConnection implements RemotingConnection
          else
          {
             //not a supported version!
-            ActiveMQStompException error = BUNDLE.versionNotSupported(acceptVersion);
+            ActiveMQStompException error = BUNDLE.versionNotSupported(acceptVersion).setHandler(frameHandler);
             error.addHeader(Stomp.Headers.Error.VERSION, manager.getSupportedVersionsAsErrorVersion());
             error.addHeader(Stomp.Headers.CONTENT_TYPE, "text/plain");
             error.setBody("Supported protocol versions are " + manager.getSupportedVersionsAsString());
@@ -514,7 +514,7 @@ public final class StompConnection implements RemotingConnection
    {
       if (host == null)
       {
-         ActiveMQStompException error = BUNDLE.nullHostHeader();
+         ActiveMQStompException error = BUNDLE.nullHostHeader().setHandler(frameHandler);
          error.setBody(BUNDLE.hostCannotBeNull());
          throw error;
       }
@@ -522,7 +522,7 @@ public final class StompConnection implements RemotingConnection
       String localHost = manager.getVirtualHostName();
       if (!host.equals(localHost))
       {
-         ActiveMQStompException error = BUNDLE.hostNotMatch();
+         ActiveMQStompException error = BUNDLE.hostNotMatch().setHandler(frameHandler);
          error.setBody(BUNDLE.hostNotMatchDetails(host));
          throw error;
       }
@@ -542,13 +542,13 @@ public final class StompConnection implements RemotingConnection
       {
          if (isDestroyed())
          {
-            throw BUNDLE.connectionDestroyed();
+            throw BUNDLE.connectionDestroyed().setHandler(frameHandler);
          }
          if (!initialized)
          {
             if (!(Stomp.Commands.CONNECT.equals(cmd) || Stomp.Commands.STOMP.equals(cmd)))
             {
-               throw BUNDLE.connectionNotEstablished();
+               throw BUNDLE.connectionNotEstablished().setHandler(frameHandler);
             }
             //decide version
             negotiateVersion(request);
@@ -609,7 +609,7 @@ public final class StompConnection implements RemotingConnection
       }
       catch (Exception e)
       {
-         throw BUNDLE.errorGetSession(e);
+         throw BUNDLE.errorGetSession(e).setHandler(frameHandler);
       }
 
       return session;
@@ -619,7 +619,7 @@ public final class StompConnection implements RemotingConnection
    {
       if (!this.valid)
       {
-         throw BUNDLE.invalidConnection();
+         throw BUNDLE.invalidConnection().setHandler(frameHandler);
       }
    }
 
@@ -649,7 +649,7 @@ public final class StompConnection implements RemotingConnection
       }
       catch (Exception e)
       {
-         throw BUNDLE.errorSendMessage(message, e);
+         throw BUNDLE.errorSendMessage(message, e).setHandler(frameHandler);
       }
    }
 
@@ -677,7 +677,7 @@ public final class StompConnection implements RemotingConnection
       }
       catch (Exception e)
       {
-         throw BUNDLE.errorBeginTx(txID, e);
+         throw BUNDLE.errorBeginTx(txID, e).setHandler(frameHandler);
       }
    }
 
@@ -689,7 +689,7 @@ public final class StompConnection implements RemotingConnection
       }
       catch (Exception e)
       {
-         throw BUNDLE.errorCommitTx(txID, e);
+         throw BUNDLE.errorCommitTx(txID, e).setHandler(frameHandler);
       }
    }
 
@@ -705,7 +705,7 @@ public final class StompConnection implements RemotingConnection
       }
       catch (Exception e)
       {
-         throw BUNDLE.errorAbortTx(txID, e);
+         throw BUNDLE.errorAbortTx(txID, e).setHandler(frameHandler);
       }
    }
 
@@ -740,7 +740,7 @@ public final class StompConnection implements RemotingConnection
       {
          if (destination == null)
          {
-            throw BUNDLE.noDestination();
+            throw BUNDLE.noDestination().setHandler(frameHandler);
          }
          subscriptionID = "subscription/" + destination;
       }
@@ -755,7 +755,7 @@ public final class StompConnection implements RemotingConnection
       }
       catch (Exception e)
       {
-         throw BUNDLE.errorCreatSubscription(subscriptionID, e);
+         throw BUNDLE.errorCreatSubscription(subscriptionID, e).setHandler(frameHandler);
       }
    }
 
@@ -771,7 +771,7 @@ public final class StompConnection implements RemotingConnection
       }
       catch (Exception e)
       {
-         throw BUNDLE.errorUnsubscrib(subscriptionID, e);
+         throw BUNDLE.errorUnsubscrib(subscriptionID, e).setHandler(frameHandler);
       }
    }
 
@@ -787,7 +787,7 @@ public final class StompConnection implements RemotingConnection
       }
       catch (Exception e)
       {
-         throw BUNDLE.errorAck(messageID, e);
+         throw BUNDLE.errorAck(messageID, e).setHandler(frameHandler);
       }
    }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/6ee22e7c/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java
index 3bbe88f..d0c13d1 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompDecoder.java
@@ -162,7 +162,7 @@ public class StompDecoder
    //max len of EOL (default is 1 for '\n')
    protected int eolLen = 1;
 
-   private final VersionedStompFrameHandler handler;
+   protected final VersionedStompFrameHandler handler;
 
    public StompDecoder(VersionedStompFrameHandler handler)
    {
@@ -417,7 +417,7 @@ public class StompDecoder
          }
          else if (workingBuffer[offset] == CR)
          {
-            if (nextChar) throw BUNDLE.invalidTwoCRs();
+            if (nextChar) throw BUNDLE.invalidTwoCRs().setHandler(handler);
             nextChar = true;
          }
          else
@@ -429,7 +429,7 @@ public class StompDecoder
 
       if (nextChar)
       {
-         throw BUNDLE.badCRs();
+         throw BUNDLE.badCRs().setHandler(handler);
       }
 
       if (data < 4 + offset)
@@ -621,7 +621,7 @@ public class StompDecoder
       if (workingBuffer[pos - 1] != NEW_LINE)
       {
          //give a signal to try other versions
-         ActiveMQStompException error = BUNDLE.notValidNewLine(workingBuffer[pos - 1]);
+         ActiveMQStompException error = BUNDLE.notValidNewLine(workingBuffer[pos - 1]).setHandler(handler);
          error.setCode(ActiveMQStompException.INVALID_EOL_V10);
          error.setBody(BUNDLE.unexpectedNewLine(workingBuffer[pos - 1]));
          throw error;
@@ -632,7 +632,7 @@ public class StompDecoder
 
    public void throwInvalid() throws ActiveMQStompException
    {
-      ActiveMQStompException error = BUNDLE.invalidCommand(this.dumpByteArray(workingBuffer));
+      ActiveMQStompException error = BUNDLE.invalidCommand(this.dumpByteArray(workingBuffer)).setHandler(handler);
       error.setCode(ActiveMQStompException.INVALID_COMMAND);
       error.setBody(BUNDLE.invalidFrame(this.dumpByteArray(workingBuffer)));
       throw error;

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/6ee22e7c/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java
index fa9d0c0..a4e8ea5 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompProtocolManager.java
@@ -364,7 +364,8 @@ class StompProtocolManager implements ProtocolManager<StompFrameInterceptor>, No
             ActiveMQServerLogger.LOGGER.errorProcessingIOCallback(errorCode, errorMessage);
 
             ActiveMQStompException e = new ActiveMQStompException("Error sending reply",
-                                                                  ActiveMQExceptionType.createException(errorCode, errorMessage));
+                                                                  ActiveMQExceptionType.createException(errorCode, errorMessage))
+                  .setHandler(connection.getFrameHandler());
 
             StompFrame error = e.getFrame();
             send(connection, error);
@@ -426,7 +427,7 @@ class StompProtocolManager implements ProtocolManager<StompFrameInterceptor>, No
       StompSession session = getTransactedSession(connection, txID);
       if (session == null)
       {
-         throw new ActiveMQStompException("No transaction started: " + txID);
+         throw new ActiveMQStompException(connection, "No transaction started: " + txID);
       }
       transactedSessions.remove(txID);
       session.getSession().commit();
@@ -437,7 +438,7 @@ class StompProtocolManager implements ProtocolManager<StompFrameInterceptor>, No
       StompSession session = getTransactedSession(connection, txID);
       if (session == null)
       {
-         throw new ActiveMQStompException("No transaction started: " + txID);
+         throw new ActiveMQStompException(connection, "No transaction started: " + txID);
       }
       transactedSessions.remove(txID);
       session.getSession().rollback(false);
@@ -452,7 +453,7 @@ class StompProtocolManager implements ProtocolManager<StompFrameInterceptor>, No
       stompSession.setNoLocal(noLocal);
       if (stompSession.containsSubscription(subscriptionID))
       {
-         throw new ActiveMQStompException("There already is a subscription for: " + subscriptionID +
+         throw new ActiveMQStompException(connection, "There already is a subscription for: " + subscriptionID +
                                              ". Either use unique subscription IDs or do not create multiple subscriptions for the same destination");
       }
       long consumerID = server.getStorageManager().generateID();
@@ -473,7 +474,7 @@ class StompProtocolManager implements ProtocolManager<StompFrameInterceptor>, No
       boolean unsubscribed = stompSession.unsubscribe(subscriptionID, durableSubscriberName);
       if (!unsubscribed)
       {
-         throw new ActiveMQStompException("Cannot unsubscribe as no subscription exists for id: " + subscriptionID);
+         throw new ActiveMQStompException(connection, "Cannot unsubscribe as no subscription exists for id: " + subscriptionID);
       }
    }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/6ee22e7c/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompSession.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompSession.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompSession.java
index d12fa06..3541ef0 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompSession.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/StompSession.java
@@ -223,7 +223,7 @@ public class StompSession implements SessionCallback
 
       if (pair == null)
       {
-         throw BUNDLE.failToAckMissingID(id);
+         throw BUNDLE.failToAckMissingID(id).setHandler(connection.getFrameHandler());
       }
 
       long consumerID = pair.getA();
@@ -235,7 +235,7 @@ public class StompSession implements SessionCallback
       {
          if (!sub.getID().equals(subscriptionID))
          {
-            throw BUNDLE.subscriptionIDMismatch(subscriptionID, sub.getID());
+            throw BUNDLE.subscriptionIDMismatch(subscriptionID, sub.getID()).setHandler(connection.getFrameHandler());
          }
       }
 

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/6ee22e7c/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/VersionedStompFrameHandler.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/VersionedStompFrameHandler.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/VersionedStompFrameHandler.java
index 2bed68c..ccb4a1b 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/VersionedStompFrameHandler.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/VersionedStompFrameHandler.java
@@ -158,7 +158,7 @@ public abstract class VersionedStompFrameHandler
 
    public StompFrame onUnknown(String command)
    {
-      ActiveMQStompException error = BUNDLE.unknownCommand(command);
+      ActiveMQStompException error = BUNDLE.unknownCommand(command).setHandler(this);
       StompFrame response = error.getFrame();
       return response;
    }
@@ -178,7 +178,7 @@ public abstract class VersionedStompFrameHandler
       String txID = request.getHeader(Stomp.Headers.TRANSACTION);
       if (txID == null)
       {
-         ActiveMQStompException error = BUNDLE.needTxIDHeader();
+         ActiveMQStompException error = BUNDLE.needTxIDHeader().setHandler(this);
          response = error.getFrame();
          return response;
       }
@@ -230,7 +230,7 @@ public abstract class VersionedStompFrameHandler
       }
       catch (Exception e)
       {
-         ActiveMQStompException error = BUNDLE.errorHandleSend(e);
+         ActiveMQStompException error = BUNDLE.errorHandleSend(e).setHandler(this);
          response = error.getFrame();
       }
 
@@ -248,7 +248,7 @@ public abstract class VersionedStompFrameHandler
       String txID = frame.getHeader(Stomp.Headers.TRANSACTION);
       if (txID == null)
       {
-         ActiveMQStompException error = BUNDLE.beginTxNoID();
+         ActiveMQStompException error = BUNDLE.beginTxNoID().setHandler(this);
          response = error.getFrame();
       }
       else
@@ -272,7 +272,7 @@ public abstract class VersionedStompFrameHandler
 
       if (txID == null)
       {
-         ActiveMQStompException error = BUNDLE.abortTxNoID();
+         ActiveMQStompException error = BUNDLE.abortTxNoID().setHandler(this);
          response = error.getFrame();
          return response;
       }
@@ -391,7 +391,7 @@ public abstract class VersionedStompFrameHandler
    /**
     * this method is called when a newer version of handler is created. It should
     * take over the state of the decoder of the existingHandler so that
-    * the decoding can be continued. For V10 handler it's never get called.
+    * the decoding can be continued. For V10 handler it's never called.
     *
     * @param existingHandler
     */

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/6ee22e7c/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
index e492060..3f3ceda 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v10/StompFrameHandlerV10.java
@@ -102,7 +102,7 @@ public class StompFrameHandlerV10 extends VersionedStompFrameHandler implements
       {
          if (destination == null)
          {
-            ActiveMQStompException error = BUNDLE.needIDorDestination();
+            ActiveMQStompException error = BUNDLE.needIDorDestination().setHandler(this);
             response = error.getFrame();
             return response;
          }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/6ee22e7c/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
index e30742e..d0f5ddf 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v11/StompFrameHandlerV11.java
@@ -124,7 +124,7 @@ public class StompFrameHandlerV11 extends VersionedStompFrameHandler implements
       String[] params = heartBeatHeader.split(",");
       if (params.length != 2)
       {
-         throw new ActiveMQStompException("Incorrect heartbeat header " + heartBeatHeader);
+         throw new ActiveMQStompException(connection, "Incorrect heartbeat header " + heartBeatHeader);
       }
 
       //client ping
@@ -171,7 +171,7 @@ public class StompFrameHandlerV11 extends VersionedStompFrameHandler implements
       }
       else
       {
-         response = BUNDLE.needSubscriptionID().getFrame();
+         response = BUNDLE.needSubscriptionID().setHandler(this).getFrame();
          return response;
       }
 
@@ -202,7 +202,7 @@ public class StompFrameHandlerV11 extends VersionedStompFrameHandler implements
 
       if (subscriptionID == null)
       {
-         response = BUNDLE.needSubscriptionID().getFrame();
+         response = BUNDLE.needSubscriptionID().setHandler(this).getFrame();
          return response;
       }
 
@@ -475,7 +475,7 @@ public class StompFrameHandlerV11 extends VersionedStompFrameHandler implements
             }
             else if (workingBuffer[offset] == CR)
             {
-               if (nextChar) throw BUNDLE.invalidTwoCRs();
+               if (nextChar) throw BUNDLE.invalidTwoCRs().setHandler(handler);
                nextChar = true;
             }
             else
@@ -488,7 +488,7 @@ public class StompFrameHandlerV11 extends VersionedStompFrameHandler implements
 
          if (nextChar)
          {
-            throw BUNDLE.badCRs();
+            throw BUNDLE.badCRs().setHandler(handler);
          }
 
          //if some EOLs have been processed, drop those bytes before parsing command

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/6ee22e7c/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java
----------------------------------------------------------------------
diff --git a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java
index b1632e7..7146f1f 100644
--- a/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java
+++ b/artemis-protocols/artemis-stomp-protocol/src/main/java/org/apache/activemq/artemis/core/protocol/stomp/v12/StompFrameHandlerV12.java
@@ -78,8 +78,7 @@ public class StompFrameHandlerV12 extends StompFrameHandlerV11 implements FrameE
 
       if (messageID == null)
       {
-         ActiveMQStompException error = BUNDLE.noIDInAck();
-         error.setHandler(connection.getFrameHandler());
+         ActiveMQStompException error = BUNDLE.noIDInAck().setHandler(connection.getFrameHandler());
          return error.getFrame();
       }
 
@@ -214,7 +213,7 @@ public class StompFrameHandlerV12 extends StompFrameHandlerV11 implements FrameE
                {
                   if (nextEOLChar)
                   {
-                     throw BUNDLE.invalidTwoCRs();
+                     throw BUNDLE.invalidTwoCRs().setHandler(handler);
                   }
                   nextEOLChar = true;
                   break;


[3/3] activemq-artemis git commit: This closes #78 Stomp changes

Posted by cl...@apache.org.
This closes #78 Stomp changes


Project: http://git-wip-us.apache.org/repos/asf/activemq-artemis/repo
Commit: http://git-wip-us.apache.org/repos/asf/activemq-artemis/commit/4167c5f2
Tree: http://git-wip-us.apache.org/repos/asf/activemq-artemis/tree/4167c5f2
Diff: http://git-wip-us.apache.org/repos/asf/activemq-artemis/diff/4167c5f2

Branch: refs/heads/master
Commit: 4167c5f2cc45a79caecd71fd990c3129533e0833
Parents: f17991e 6ee22e7
Author: Clebert Suconic <cl...@apache.org>
Authored: Tue Jul 14 09:53:36 2015 -0400
Committer: Clebert Suconic <cl...@apache.org>
Committed: Tue Jul 14 09:53:36 2015 -0400

----------------------------------------------------------------------
 .../protocol/stomp/ActiveMQStompException.java  |  3 +-
 .../core/protocol/stomp/StompConnection.java    | 34 ++++++++++----------
 .../core/protocol/stomp/StompDecoder.java       | 17 ++++++----
 .../artemis/core/protocol/stomp/StompFrame.java |  2 +-
 .../protocol/stomp/StompProtocolManager.java    | 11 ++++---
 .../core/protocol/stomp/StompSession.java       |  4 +--
 .../stomp/VersionedStompFrameHandler.java       | 12 +++----
 .../stomp/v10/StompFrameHandlerV10.java         |  4 +--
 .../stomp/v11/StompFrameHandlerV11.java         | 17 ++++++----
 .../stomp/v12/StompFrameHandlerV12.java         | 13 ++++----
 10 files changed, 65 insertions(+), 52 deletions(-)
----------------------------------------------------------------------