You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@activemq.apache.org by gaohoward <gi...@git.apache.org> on 2018/01/26 04:43:54 UTC

[GitHub] activemq-artemis pull request #1819: ARTEMIS-1639 HornetQClientProtocolManag...

GitHub user gaohoward opened a pull request:

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

    ARTEMIS-1639 HornetQClientProtocolManager sending unsupported packet

    HornetQClientProtocolManager is used to connect HornteQ servers.
    During reconnect, it sends a CheckFailoverMessage packet to the
    server as part of reconnection. This packet is not supported by
    HornetQ server (existing release), so it will break the backward
    compatibility.

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

    $ git pull https://github.com/gaohoward/activemq-artemis master_chk_failover

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

    https://github.com/apache/activemq-artemis/pull/1819.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 #1819
    
----
commit 02397268695cbd8e865c38961f3aaa876ee6bd66
Author: Howard Gao <ho...@...>
Date:   2018-01-26T04:40:24Z

    ARTEMIS-1639 HornetQClientProtocolManager sending unsupported packet
    
    HornetQClientProtocolManager is used to connect HornteQ servers.
    During reconnect, it sends a CheckFailoverMessage packet to the
    server as part of reconnection. This packet is not supported by
    HornetQ server (existing release), so it will break the backward
    compatibility.

----


---

[GitHub] activemq-artemis issue #1819: ARTEMIS-1639 HornetQClientProtocolManager send...

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

    https://github.com/apache/activemq-artemis/pull/1819
  
    not running at all...always fail...
    
    The compatibility testsuite is part of the fast-tests.. and gitpr check.. we cannot even add intermittent failures on this testsuite.. it has to be fast, reliable and reproduceable. 


---

[GitHub] activemq-artemis issue #1819: ARTEMIS-1639 HornetQClientProtocolManager send...

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

    https://github.com/apache/activemq-artemis/pull/1819
  
    Howard I made a few changes here. Please review.  It’s merged already. 


---

[GitHub] activemq-artemis pull request #1819: ARTEMIS-1639 HornetQClientProtocolManag...

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1819#discussion_r165095255
  
    --- Diff: tests/compatibility-tests/src/main/resources/hqclientProtocolTest/failoverTest.groovy ---
    @@ -0,0 +1,85 @@
    +package meshTest
    +
    +import org.apache.activemq.artemis.tests.compatibility.GroovyRun
    +
    +import javax.jms.*
    +import java.util.concurrent.CountDownLatch
    +import java.util.concurrent.TimeUnit
    +
    +/*
    + * 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.
    + */
    +
    +// starts an artemis server
    +String serverType = arg[0];
    +String clientType = arg[1];
    +String operation = arg[2];
    +
    +
    +String queueName = "queue";
    +
    +
    +String textBody = "a rapadura e doce mas nao e mole nao";
    +
    +println("clientType " + clientType);
    +println("serverType " + serverType);
    +
    +if (clientType.startsWith("ARTEMIS")) {
    +    // Can't depend directly on artemis, otherwise it wouldn't compile in hornetq
    +    GroovyRun.evaluate("clients/artemisHQClientHA.groovy", "serverArg", serverType);
    +} else {
    +    throw new RuntimeException("The test is not meant for client type: " + clientType);
    +}
    +
    +
    +Connection connection = cf.createConnection();
    +Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
    +Queue queue = session.createQueue(queueName);
    +
    +if (operation.equals("failoverTestSend")) {
    +
    +    CountDownLatch latch = new CountDownLatch(10);
    +
    +    CompletionListener completionListener = new CompletionListener() {
    +        @Override
    +        void onCompletion(Message message) {
    +            latch.countDown();
    +        }
    +
    +        @Override
    +        void onException(Message message, Exception exception) {
    +
    +        }
    +    }
    +
    +    MessageProducer producer = session.createProducer(queue);
    +    producer.setDeliveryMode(DeliveryMode.PERSISTENT);
    +    for (int i = 0; i < 10; i++) {
    +        producer.send(session.createTextMessage(textBody + i), completionListener);
    +    }
    +
    +    GroovyRun.assertTrue(latch.await(10, TimeUnit.SECONDS));
    +
    +    System.out.println("Message sent");
    +
    +    return connection;
    --- End diff --
    
    if you are just returning the connection.. and using it on the test.. why are you using groovy here at all?
    
    these groovy tests were meant to cross isolation of classLoaders. if you were mixing versions the return wouldn't work at all as the producer and consumer would be on different versions.


---

[GitHub] activemq-artemis issue #1819: ARTEMIS-1639 HornetQClientProtocolManager send...

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

    https://github.com/apache/activemq-artemis/pull/1819
  
    @clebertsuconic I've reviewed your changes already. It's good, thanks!



---

[GitHub] activemq-artemis pull request #1819: ARTEMIS-1639 HornetQClientProtocolManag...

Posted by gaohoward <gi...@git.apache.org>.
Github user gaohoward commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1819#discussion_r164320845
  
    --- Diff: tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/remoting/compat/HQClientProtocolManagerTest.java ---
    @@ -0,0 +1,104 @@
    +/**
    + * 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.activemq.artemis.tests.integration.remoting.compat;
    +
    +import org.apache.activemq.artemis.api.core.ActiveMQException;
    +import org.apache.activemq.artemis.api.core.Interceptor;
    +import org.apache.activemq.artemis.api.core.SimpleString;
    +import org.apache.activemq.artemis.api.core.client.ClientSession;
    +import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
    +import org.apache.activemq.artemis.api.core.client.ServerLocator;
    +import org.apache.activemq.artemis.core.client.impl.ClientSessionInternal;
    +import org.apache.activemq.artemis.core.protocol.core.Packet;
    +import org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl;
    +import org.apache.activemq.artemis.core.protocol.hornetq.client.HornetQClientProtocolManagerFactory;
    +import org.apache.activemq.artemis.core.server.ActiveMQServer;
    +import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
    +import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
    +import org.junit.Test;
    +
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class HQClientProtocolManagerTest extends ActiveMQTestBase {
    --- End diff --
    
    ok will do.


---

[GitHub] activemq-artemis pull request #1819: ARTEMIS-1639 HornetQClientProtocolManag...

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1819#discussion_r165095346
  
    --- Diff: tests/compatibility-tests/src/main/resources/hqclientProtocolTest/failoverTest.groovy ---
    @@ -0,0 +1,85 @@
    +package meshTest
    +
    +import org.apache.activemq.artemis.tests.compatibility.GroovyRun
    +
    +import javax.jms.*
    +import java.util.concurrent.CountDownLatch
    +import java.util.concurrent.TimeUnit
    +
    +/*
    + * 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.
    + */
    +
    +// starts an artemis server
    +String serverType = arg[0];
    +String clientType = arg[1];
    +String operation = arg[2];
    +
    +
    +String queueName = "queue";
    +
    +
    +String textBody = "a rapadura e doce mas nao e mole nao";
    +
    +println("clientType " + clientType);
    +println("serverType " + serverType);
    +
    +if (clientType.startsWith("ARTEMIS")) {
    +    // Can't depend directly on artemis, otherwise it wouldn't compile in hornetq
    +    GroovyRun.evaluate("clients/artemisHQClientHA.groovy", "serverArg", serverType);
    +} else {
    +    throw new RuntimeException("The test is not meant for client type: " + clientType);
    +}
    +
    +
    +Connection connection = cf.createConnection();
    +Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
    +Queue queue = session.createQueue(queueName);
    +
    +if (operation.equals("failoverTestSend")) {
    +
    +    CountDownLatch latch = new CountDownLatch(10);
    +
    +    CompletionListener completionListener = new CompletionListener() {
    +        @Override
    +        void onCompletion(Message message) {
    +            latch.countDown();
    --- End diff --
    
    why are you using completion listener?


---

[GitHub] activemq-artemis pull request #1819: ARTEMIS-1639 HornetQClientProtocolManag...

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

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


---

[GitHub] activemq-artemis issue #1819: ARTEMIS-1639 HornetQClientProtocolManager send...

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

    https://github.com/apache/activemq-artemis/pull/1819
  
    There's no way how this would work.. you are using shared storage.. and each server on its own folder.



---

[GitHub] activemq-artemis pull request #1819: ARTEMIS-1639 HornetQClientProtocolManag...

Posted by clebertsuconic <gi...@git.apache.org>.
Github user clebertsuconic commented on a diff in the pull request:

    https://github.com/apache/activemq-artemis/pull/1819#discussion_r164164806
  
    --- Diff: tests/integration-tests/src/test/java/org/apache/activemq/artemis/tests/integration/remoting/compat/HQClientProtocolManagerTest.java ---
    @@ -0,0 +1,104 @@
    +/**
    + * 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.activemq.artemis.tests.integration.remoting.compat;
    +
    +import org.apache.activemq.artemis.api.core.ActiveMQException;
    +import org.apache.activemq.artemis.api.core.Interceptor;
    +import org.apache.activemq.artemis.api.core.SimpleString;
    +import org.apache.activemq.artemis.api.core.client.ClientSession;
    +import org.apache.activemq.artemis.api.core.client.ClientSessionFactory;
    +import org.apache.activemq.artemis.api.core.client.ServerLocator;
    +import org.apache.activemq.artemis.core.client.impl.ClientSessionInternal;
    +import org.apache.activemq.artemis.core.protocol.core.Packet;
    +import org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl;
    +import org.apache.activemq.artemis.core.protocol.hornetq.client.HornetQClientProtocolManagerFactory;
    +import org.apache.activemq.artemis.core.server.ActiveMQServer;
    +import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
    +import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
    +import org.junit.Test;
    +
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +public class HQClientProtocolManagerTest extends ActiveMQTestBase {
    --- End diff --
    
    Can you write a test using the new compatibility test framework instead?
    
    it would be nice to do a simple failover of hornetq client, artemis 1.x and artemis 2.x to see if there's any further issues.


---

[GitHub] activemq-artemis issue #1819: ARTEMIS-1639 HornetQClientProtocolManager send...

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

    https://github.com/apache/activemq-artemis/pull/1819
  
    I'm fixing it.


---

[GitHub] activemq-artemis issue #1819: ARTEMIS-1639 HornetQClientProtocolManager send...

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

    https://github.com/apache/activemq-artemis/pull/1819
  
    @clebertsuconic Hi Clebert, I added a new test to test the Artemis client failover on HornetQ server.
    The set up is a live and a backup HornetQ server. Stop the live and the client fails over. 
    Can you take a look?
    Thanks



---

[GitHub] activemq-artemis issue #1819: ARTEMIS-1639 HornetQClientProtocolManager send...

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

    https://github.com/apache/activemq-artemis/pull/1819
  
    @clebertsuconic forgot to mention that I fixed an issue where the connector configure passed from HornetQ server contains the hornetq's netty factory class name, which is not available in Artemis, I added a 'translation' to convert it to Artemis netty factory class.


---

[GitHub] activemq-artemis issue #1819: ARTEMIS-1639 HornetQClientProtocolManager send...

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

    https://github.com/apache/activemq-artemis/pull/1819
  
    The test you added is broken...


---