You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@activemq.apache.org by shailendra14k <gi...@git.apache.org> on 2018/01/02 11:09:30 UTC

[GitHub] activemq-artemis pull request #1744: [ARTEMIS-1577] Address-settings policie...

GitHub user shailendra14k opened a pull request:

    https://github.com/apache/activemq-artemis/pull/1744

    [ARTEMIS-1577] Address-settings policies aren't being applied when using an older version of JMS client

    JIRA :-  https://issues.apache.org/jira/browse/ARTEMIS-1577

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/shailendra14k/activemq-artemis ARTEMIS-1577

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/activemq-artemis/pull/1744.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1744
    
----
commit 19390150fee5ee716a67ffa707b65f3291a07ed8
Author: Shailendra Kumar Singh <sh...@...>
Date:   2018-01-02T11:07:13Z

    [ARTEMIS-1577] Address-settings policies aren't being applied when using an older version of JMS client

----


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    I don't agree with changing the CoreMessage for this... but there's something we can do at the Server's message.


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    @shailendra14k just going through the code around the producer credits you mentioned (my inquisitiveness got the better of me)
    
    It looks like this is possibly fixed maybe in Clebert's recent changes for client compatibility.
    
    See here in ServerSessionImpl the address is being handled by method removePrefix
    
    ```
       @Override
       public void requestProducerCredits(SimpleString address, final int credits) throws Exception {
          final SimpleString addr = removePrefix(address);
          PagingStore store = server.getPagingManager().getPageStore(addr);
    
          if (!store.checkMemory(new Runnable() {
             @Override
             public void run() {
                callback.sendProducerCreditsMessage(credits, address);
             }
          })) {
             callback.sendProducerCreditsFailMessage(credits, address);
          }
       }
    ```
    
    removePrefix only operates if prefix's are enabled for that session:
    ```
    @Override
       public SimpleString removePrefix(SimpleString address) {
          if (prefixEnabled && address != null) {
             return PrefixUtil.getAddress(address, prefixes);
          }
          return address;
       }
    ```
    
    And then in ActiveMQPacketHandler its setting the prefixes to replace when connection version is < addressing change.
    ```
    if (connection.getChannelVersion() < PacketImpl.ADDRESSING_CHANGE_VERSION) {
                routingTypeMap = new HashMap<>();
                routingTypeMap.put(PacketImpl.OLD_QUEUE_PREFIX, RoutingType.ANYCAST);
                routingTypeMap.put(PacketImpl.OLD_TOPIC_PREFIX, RoutingType.MULTICAST);
             }
    ```
    
    Does your issue occur if you take latest master?
    



---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by shailendra14k <gi...@git.apache.org>.
Github user shailendra14k commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    Thank you, Closing this PR


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by mtaylor <gi...@git.apache.org>.
Github user mtaylor commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    @shailendra14k This has been handled using two approaches internally which might have caused some confusion.  But, there is an acceptor setting anycastPrefix and multicastPrefix that does what you've put here.  
    
    Try appending the following to you acceptor "anycastPrefix=jms.queue.;multicastPrefix=jms.topic."


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by shailendra14k <gi...@git.apache.org>.
Github user shailendra14k commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    @mtaylor I tested with the latest master and anycastPrefix does not help. I got the same behaviour i.e using old version client ignores address-settings policies.
    
    `<acceptor name="artemis">tcp://0.0.0.0:61616?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;anycastPrefix=jms.queue.;multicastPrefix=jms.topic.</acceptor>`
    



---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by shailendra14k <gi...@git.apache.org>.
Github user shailendra14k commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    @michaelandrepearce  Yes, I tested on the latest master as fas as I remember, will retest it again. 
    >>removePrefix only operates if prefix's are enabled for that session:
    Does that mean I have to enable it for the session? or it will be done automatically?



---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    I will handle this... 


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    My view on this is that this is for back compatibility with 1.5.X clients and should be done at protocol handling levels (not within core guts). As Clebert has done for other back compatibility bits, the stripping (including those other area's you noted) should only occur if the channel client version < addressing version. (i think its 129).


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    This breaks anyone actually wanting to name their addresses with prefixes. 


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    @shailendra14k there also is now a compatibility test framework that Clebert added, so if you do find this is broken for a particular client version and not fixed by the recent changes in master, i think first bit will be to add the case its broken to that compatibility test bundle, that way any fix can be verified (and ensured it won't regress)
    
    You'll find that all here:
    /tests/compatibility-tests/src/test/java/org/apache/activemq/artemis/tests/compatibility
    
    see the read me's of the current tests on how to run / run within an IDE.


---

[GitHub] activemq-artemis pull request #1744: [ARTEMIS-1577] Address-settings policie...

Posted by shailendra14k <gi...@git.apache.org>.
Github user shailendra14k closed the pull request at:

    https://github.com/apache/activemq-artemis/pull/1744


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by shailendra14k <gi...@git.apache.org>.
Github user shailendra14k commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    @michaelandrepearce, Most the places what I observed is prefixes are removed like:-
    
    artemis-core-client:- QueueAbstractPacket class while getting the address prefix are removed.
    https://github.com/apache/activemq-artemis/blob/master/artemis-core-client/src/main/java/org/apache/activemq/artemis/core/protocol/core/impl/wireformat/QueueAbstractPacket.java#L49-L60
    
    When getting the ProducerCredits [1], the prefix is removed before getting the store from PagingManager, due to which the address setting policies are ignored as it will always get a wrong store.
    
     https://github.com/apache/activemq-artemis/blob/master/artemis-server/src/main/java/org/apache/activemq/artemis/core/server/impl/ServerSessionImpl.java#L1412-L1424
    
    I was not able to find a way other then updating the CoreMessage method getAddressSimpleString which is used at most places for getting the queue name.
    
    
     


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    If you could review the new one please ?


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by shailendra14k <gi...@git.apache.org>.
Github user shailendra14k commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    @michaelandrepearce  thank you for the update, I will try to include this at the protocol level.  


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by michaelandrepearce <gi...@git.apache.org>.
Github user michaelandrepearce commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    @shailendra14k if you step through the code and as i highlighted the ActiveMQPacketHandler is setting this when the session is made and the client version is pre-addressing changes.
    
    see code extract:
    ```
    if (connection.getChannelVersion() < PacketImpl.ADDRESSING_CHANGE_VERSION) {
                routingTypeMap = new HashMap<>();
                routingTypeMap.put(PacketImpl.OLD_QUEUE_PREFIX, RoutingType.ANYCAST);
                routingTypeMap.put(PacketImpl.OLD_TOPIC_PREFIX, RoutingType.MULTICAST);
             }
    ```


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    if you could please close this PR?
    
    #1787 superceedes this.. although this PR will be closed automagically when #1787 is merged.


---

[GitHub] activemq-artemis issue #1744: [ARTEMIS-1577] Address-settings policies aren'...

Posted by shailendra14k <gi...@git.apache.org>.
Github user shailendra14k commented on the issue:

    https://github.com/apache/activemq-artemis/pull/1744
  
    @clebertsuconic , I did some testing, with the changes [#1787](https://github.com/apache/activemq-artemis/pull/1787), Address-settings policies work as expected with the older client version.


---