You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/01/22 14:37:45 UTC

[GitHub] [pulsar] lhotari opened a new pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

lhotari opened a new pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286


   ### Motivation
   
   MessageIdTest class contains 2 of the flaky test cases in the Pulsar code base:
   * `org.apache.pulsar.client.impl.MessageIdTest.testChecksumVersionComptability` and 
   * `org.apache.pulsar.client.impl.MessageIdTest.testChecksumReconnection`
   
   These test cases don't have much to do with `MessageId`, but are tests for validating message checksum handling in cases where there are pre 1.15 version brokers and post 1.15 version brokers in a mixed broker environment. The tests might not be very relevant any more. However it was taken as a learning experiment to fix these tests and refactor them so that the flakiness of the test code would be eliminated. Similar patterns might be needed in other tests to eliminate flakiness.
   
   ### Modifications
   
   The changes aren't only to fix MessageIdTest. Most changes could help reduce flakiness of other tests as well.
   
   Improve shutdown of the broker and related services to reduce test flakiness
   - await for termination of executors
   - close the listen channel synchronously
   - use shutdown instead of shutdownNow in AbstractMetadataStore.close
     so that in-flight tasks get processed
   
   Handle special case where the executor rejects the task and the callback was never called
   
   Improve logging in MockedPulsarServiceBaseTest related to stopping and starting
   
   Refactor PulsarClient initialization and lifecycle management in tests
   
   Add getter and setter to access remoteEndpointProtocolVersion field
   - it makes it easier to override for tests
   
   Add hooks for overriding the producer implementation in PulsarClientImpl
   - useful for tests. Instead of relying on Mockito, there's a pure Java
     way to inject behavior to producer implementations for testing purposes
   
   Introduce PulsarTestClient that contains ways to prevent race conditions and test flakiness
   - provides features for simulating failure conditions, for example
     the case of the broker connection disconnecting
   
   Add solution for using Enums classes as source for TestNG DataProvider
   
   Fix flaky MessageIdTest and move checksum related tests to new class


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] merlimat commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
merlimat commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r562759077



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -207,7 +207,7 @@ public void accept(Notification n) {
 
     @Override
     public void close() throws Exception {
-        executor.shutdownNow();
+        executor.shutdown();

Review comment:
       Why do we need this change? The `shutdownNow()` will ensure that the thread is interrupted if it was waiting on anything. I think that, as a general rule, it should be preferable to wait for the thread to exit on its own.

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -542,8 +543,17 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) {
         }
 
         scheduledExecutor.shutdownNow();
-        orderedExecutor.shutdownNow();
+        if (!scheduledExecutor.awaitTermination(30, TimeUnit.SECONDS)) {
+           log.error("Cannot stop scheduledExecutor gracefully.");
+        }
+        orderedExecutor.shutdown();

Review comment:
       Why not: 
   ```suggestion
           orderedExecutor.shutdownNow();
   ```

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -512,6 +512,7 @@ public void shutdown() throws InterruptedException, ManagedLedgerException {
             ledger.asyncClose(new AsyncCallbacks.CloseCallback() {
                 @Override
                 public void closeComplete(Object ctx) {
+                    log.debug("[{}] Closed managed ledger.", ledger.getName());

Review comment:
       Same as above

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -408,7 +408,7 @@ public void initializeFailed(ManagedLedgerException e) {
                         pendingLedger.ledger.asyncClose(new CloseCallback() {
                             @Override
                             public void closeComplete(Object ctx) {
-                                // no-op
+                                log.debug("[{}] Closed pending initializer managed ledger.", name);

Review comment:
       `if (log.isDebugEnabled()) { ...} `




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#issuecomment-767070975


   > It seems there's a genuine test failure: 
   
   @merlimat Thanks for the heads up. I'll address it tomorrow.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563602467



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import io.netty.channel.EventLoopGroup;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.io.IOException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
+import org.apache.pulsar.common.util.netty.EventLoopUtil;
+import org.awaitility.Awaitility;
+
+/**
+ * A Pulsar Client that is used for testing scenarios where the different
+ * asynchronous operations of the client-broker interaction must be orchestrated by the test
+ * so that race conditions caused by the test code can be eliminated.
+ *
+ * features:
+ * - can override remote endpoint protocol version in a thread safe manner
+ * - can reject new connections from the client to the broker
+ * - can drop all OpSend messages after they have been added to pendingMessages and processed
+ *   by the client. This simulates a situation where sending messages go to a "black hole".
+ * - can synchronize operations with the help of the pending message callback which gets
+ *   called after the message to send out has been added to the pending messages in the client.
+ *
+ */
+public class PulsarTestClient extends PulsarClientImpl {

Review comment:
       +1, I hope this gets addressed when I revisit the PR and remove unrelated changes. Currently the PulsarTestClient is needed to fix the flaky tests, so perhaps it could be kept in this PR after the unrelated changes have been removed?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563598155



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -669,11 +669,23 @@ public void close() throws IOException {
         });
 
         if (listenChannel != null) {
-            listenChannel.close();
+            try {
+                if (!listenChannel.close().await(30, TimeUnit.SECONDS)) {

Review comment:
       Thanks for the review @sijie .
   I'll move this change to a separate PR as you have suggested for one of the other changes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] merlimat merged pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
merlimat merged pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563630840



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -669,11 +669,23 @@ public void close() throws IOException {
         });
 
         if (listenChannel != null) {
-            listenChannel.close();
+            try {
+                if (!listenChannel.close().await(30, TimeUnit.SECONDS)) {

Review comment:
       I have removed this change from this PR. The new PR for closing the channels is #9308 . PTAL @sijie




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] merlimat commented on pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
merlimat commented on pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#issuecomment-767069507


   @lhotari It seems there's a genuine test failure: 
   
   ```
   Error:  Failures: 
   Error:  org.apache.pulsar.client.impl.PartitionedProducerImplTest.testCustomMessageRouterInstance(org.apache.pulsar.client.impl.PartitionedProducerImplTest)
   [INFO]   Run 1: PASS
   Error:    Run 2: PartitionedProducerImplTest.testCustomMessageRouterInstance:99->getMessageRouter:105 ? NullPointer
   [INFO] 
   Error:  org.apache.pulsar.client.impl.PartitionedProducerImplTest.testRoundRobinPartitionMessageRouterImplInstance(org.apache.pulsar.client.impl.PartitionedProducerImplTest)
   [INFO]   Run 1: PASS
   Error:    Run 2: PartitionedProducerImplTest.testRoundRobinPartitionMessageRouterImplInstance:89->getMessageRouter:105 ? NullPointer
   [INFO] 
   Error:  org.apache.pulsar.client.impl.PartitionedProducerImplTest.testSinglePartitionMessageRouterImplInstance(org.apache.pulsar.client.impl.PartitionedProducerImplTest)
   [INFO]   Run 1: PASS
   Error:    Run 2: PartitionedProducerImplTest.testSinglePartitionMessageRouterImplInstance:80->getMessageRouter:105 ? NullPointer
   [INFO] 
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563630100



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -207,7 +207,7 @@ public void accept(Notification n) {
 
     @Override
     public void close() throws Exception {
-        executor.shutdownNow();
+        executor.shutdown();

Review comment:
       I have now removed the executor shutdown/shutdownNow related changes from this PR. PTAL @merlimat 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563595426



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -207,7 +207,7 @@ public void accept(Notification n) {
 
     @Override
     public void close() throws Exception {
-        executor.shutdownNow();
+        executor.shutdown();

Review comment:
       Thanks for the review @merlimat .
   
   > Why do we need this change?
   
   Valid point. It's not actually needed. 
   
   When I was debugging the flakiness, the key issue of the flakiness of MessageIdTest was related to the restarting of the broker not happening "cleanly". I started making an experiment where I'd revisit the shutdown of the broker. However these changes weren't relevant for making the test stable. I'll remove them completely out of this PR.
   
   The relevant problem in shutdown was in BrokerService where `listenChannel.close()` doesn't use `.await` to synchronously wait for the completion of shutting down the listeners. I'll submit that fix as a separate PR. @sijie suggested in his review comments to split some changes as separate PRs.

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -669,11 +669,23 @@ public void close() throws IOException {
         });
 
         if (listenChannel != null) {
-            listenChannel.close();
+            try {
+                if (!listenChannel.close().await(30, TimeUnit.SECONDS)) {

Review comment:
       Thanks for the review @sijie .
   I'll move this change to a separate PR as you have suggested for one of the other changes.

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java
##########
@@ -0,0 +1,249 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.service.BrokerTestBase;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.MessageRoutingMode;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.TypedMessageBuilder;
+import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg;
+import org.apache.pulsar.common.api.proto.MessageMetadata;
+import org.apache.pulsar.common.api.proto.ProtocolVersion;
+import org.apache.pulsar.common.protocol.ByteBufPair;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.protocol.Commands.ChecksumType;
+import org.apache.pulsar.tests.EnumValuesDataProvider;
+import org.awaitility.Awaitility;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+public class MessageChecksumTest extends BrokerTestBase {

Review comment:
       Thanks for the suggestion of splitting the changes to separate pull requests. I'll remove some changes from this PR to fix the flaky MessageIdTest. I was thinking that this change to split MessageChecksumTest from MessageIdTest is the relevant for fixing the flaky MessageIdTest, so I was thinking to to keep it in this PR, but move the other unrelated parts to new PRs. I hope this will be an acceptable way to resolve this.

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import io.netty.channel.EventLoopGroup;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.io.IOException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
+import org.apache.pulsar.common.util.netty.EventLoopUtil;
+import org.awaitility.Awaitility;
+
+/**
+ * A Pulsar Client that is used for testing scenarios where the different
+ * asynchronous operations of the client-broker interaction must be orchestrated by the test
+ * so that race conditions caused by the test code can be eliminated.
+ *
+ * features:
+ * - can override remote endpoint protocol version in a thread safe manner
+ * - can reject new connections from the client to the broker
+ * - can drop all OpSend messages after they have been added to pendingMessages and processed
+ *   by the client. This simulates a situation where sending messages go to a "black hole".
+ * - can synchronize operations with the help of the pending message callback which gets
+ *   called after the message to send out has been added to the pending messages in the client.
+ *
+ */
+public class PulsarTestClient extends PulsarClientImpl {

Review comment:
       +1, I hope this gets addressed when I revisit the PR and remove unrelated changes. Currently the PulsarTestClient is needed to fix the flaky tests, so perhaps it could be kept in this PR after the unrelated changes have been removed?

##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -207,7 +207,7 @@ public void accept(Notification n) {
 
     @Override
     public void close() throws Exception {
-        executor.shutdownNow();
+        executor.shutdown();

Review comment:
       I have now removed the executor shutdown/shutdownNow related changes from this PR. PTAL @merlimat 

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -512,6 +512,7 @@ public void shutdown() throws InterruptedException, ManagedLedgerException {
             ledger.asyncClose(new AsyncCallbacks.CloseCallback() {
                 @Override
                 public void closeComplete(Object ctx) {
+                    log.debug("[{}] Closed managed ledger.", ledger.getName());

Review comment:
       I have removed this change from this PR.

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -542,8 +543,17 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) {
         }
 
         scheduledExecutor.shutdownNow();
-        orderedExecutor.shutdownNow();
+        if (!scheduledExecutor.awaitTermination(30, TimeUnit.SECONDS)) {
+           log.error("Cannot stop scheduledExecutor gracefully.");
+        }
+        orderedExecutor.shutdown();

Review comment:
       I have removed this change from this PR.

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -669,11 +669,23 @@ public void close() throws IOException {
         });
 
         if (listenChannel != null) {
-            listenChannel.close();
+            try {
+                if (!listenChannel.close().await(30, TimeUnit.SECONDS)) {

Review comment:
       I have removed this change from this PR. The new PR for closing the channels is #9308 . PTAL @sijie

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java
##########
@@ -0,0 +1,249 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.service.BrokerTestBase;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.MessageRoutingMode;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.TypedMessageBuilder;
+import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg;
+import org.apache.pulsar.common.api.proto.MessageMetadata;
+import org.apache.pulsar.common.api.proto.ProtocolVersion;
+import org.apache.pulsar.common.protocol.ByteBufPair;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.protocol.Commands.ChecksumType;
+import org.apache.pulsar.tests.EnumValuesDataProvider;
+import org.awaitility.Awaitility;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+public class MessageChecksumTest extends BrokerTestBase {

Review comment:
       Thanks for the suggestion of splitting the changes to separate pull requests. I'll remove some changes from this PR to fix the flaky MessageIdTest. I was thinking that this change to split MessageChecksumTest from MessageIdTest is the relevant for fixing the flaky MessageIdTest, so I was thinking to to keep it in this PR, but move the other unrelated parts to new PRs. I hope this will be an acceptable way to resolve this. PTAL @sijie

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import io.netty.channel.EventLoopGroup;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.io.IOException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
+import org.apache.pulsar.common.util.netty.EventLoopUtil;
+import org.awaitility.Awaitility;
+
+/**
+ * A Pulsar Client that is used for testing scenarios where the different
+ * asynchronous operations of the client-broker interaction must be orchestrated by the test
+ * so that race conditions caused by the test code can be eliminated.
+ *
+ * features:
+ * - can override remote endpoint protocol version in a thread safe manner
+ * - can reject new connections from the client to the broker
+ * - can drop all OpSend messages after they have been added to pendingMessages and processed
+ *   by the client. This simulates a situation where sending messages go to a "black hole".
+ * - can synchronize operations with the help of the pending message callback which gets
+ *   called after the message to send out has been added to the pending messages in the client.
+ *
+ */
+public class PulsarTestClient extends PulsarClientImpl {

Review comment:
       +1, I hope this gets addressed when I revisit the PR and remove unrelated changes. Currently the PulsarTestClient is needed to fix the flaky tests, so perhaps it could be kept in this PR after the unrelated changes have been removed? PTAL @sijie 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563599940



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java
##########
@@ -0,0 +1,249 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.service.BrokerTestBase;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.MessageRoutingMode;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.TypedMessageBuilder;
+import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg;
+import org.apache.pulsar.common.api.proto.MessageMetadata;
+import org.apache.pulsar.common.api.proto.ProtocolVersion;
+import org.apache.pulsar.common.protocol.ByteBufPair;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.protocol.Commands.ChecksumType;
+import org.apache.pulsar.tests.EnumValuesDataProvider;
+import org.awaitility.Awaitility;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+public class MessageChecksumTest extends BrokerTestBase {

Review comment:
       Thanks for the suggestion of splitting the changes to separate pull requests. I'll remove some changes from this PR to fix the flaky MessageIdTest. I was thinking that this change to split MessageChecksumTest from MessageIdTest is the relevant for fixing the flaky MessageIdTest, so I was thinking to to keep it in this PR, but move the other unrelated parts to new PRs. I hope this will be an acceptable way to resolve this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] merlimat commented on pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
merlimat commented on pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#issuecomment-767069507


   @lhotari It seems there's a genuine test failure: 
   
   ```
   Error:  Failures: 
   Error:  org.apache.pulsar.client.impl.PartitionedProducerImplTest.testCustomMessageRouterInstance(org.apache.pulsar.client.impl.PartitionedProducerImplTest)
   [INFO]   Run 1: PASS
   Error:    Run 2: PartitionedProducerImplTest.testCustomMessageRouterInstance:99->getMessageRouter:105 ? NullPointer
   [INFO] 
   Error:  org.apache.pulsar.client.impl.PartitionedProducerImplTest.testRoundRobinPartitionMessageRouterImplInstance(org.apache.pulsar.client.impl.PartitionedProducerImplTest)
   [INFO]   Run 1: PASS
   Error:    Run 2: PartitionedProducerImplTest.testRoundRobinPartitionMessageRouterImplInstance:89->getMessageRouter:105 ? NullPointer
   [INFO] 
   Error:  org.apache.pulsar.client.impl.PartitionedProducerImplTest.testSinglePartitionMessageRouterImplInstance(org.apache.pulsar.client.impl.PartitionedProducerImplTest)
   [INFO]   Run 1: PASS
   Error:    Run 2: PartitionedProducerImplTest.testSinglePartitionMessageRouterImplInstance:80->getMessageRouter:105 ? NullPointer
   [INFO] 
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563595426



##########
File path: pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java
##########
@@ -207,7 +207,7 @@ public void accept(Notification n) {
 
     @Override
     public void close() throws Exception {
-        executor.shutdownNow();
+        executor.shutdown();

Review comment:
       Thanks for the review @merlimat .
   
   > Why do we need this change?
   
   Valid point. It's not actually needed. 
   
   When I was debugging the flakiness, the key issue of the flakiness of MessageIdTest was related to the restarting of the broker not happening "cleanly". I started making an experiment where I'd revisit the shutdown of the broker. However these changes weren't relevant for making the test stable. I'll remove them completely out of this PR.
   
   The relevant problem in shutdown was in BrokerService where `listenChannel.close()` doesn't use `.await` to synchronously wait for the completion of shutting down the listeners. I'll submit that fix as a separate PR. @sijie suggested in his review comments to split some changes as separate PRs.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
eolivelli commented on pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#issuecomment-765506888


   @codelipenghui  @zymap @sijie @rdhabalia this patch is going to fix the most Flaky test in the suite.
   
   please take a look
   
   thank you @lhotari  for contributing this work


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r562738608



##########
File path: pulsar-io/hbase/pom.xml
##########
@@ -70,12 +70,6 @@
             <version>${hbase.version}</version>
         </dependency>
 
-        <dependency>

Review comment:
       yes I cherry-picked it since the fix for the broken master branch wasn't merged yet when I pushed this PR. I'll rebase (the commit would get remove) this PR.

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -408,7 +408,7 @@ public void initializeFailed(ManagedLedgerException e) {
                         pendingLedger.ledger.asyncClose(new CloseCallback() {
                             @Override
                             public void closeComplete(Object ctx) {
-                                // no-op
+                                log.info("[{}] Closed pending initializer managed ledger.", name);

Review comment:
       good catch, I'll fix it.

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -512,6 +512,7 @@ public void shutdown() throws InterruptedException, ManagedLedgerException {
             ledger.asyncClose(new AsyncCallbacks.CloseCallback() {
                 @Override
                 public void closeComplete(Object ctx) {
+                    log.info("[{}] Closed managed ledger.", ledger.getName());

Review comment:
       good catch, I'll fix it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] sijie commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
sijie commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563014277



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java
##########
@@ -0,0 +1,249 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.service.BrokerTestBase;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.MessageRoutingMode;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.TypedMessageBuilder;
+import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg;
+import org.apache.pulsar.common.api.proto.MessageMetadata;
+import org.apache.pulsar.common.api.proto.ProtocolVersion;
+import org.apache.pulsar.common.protocol.ByteBufPair;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.protocol.Commands.ChecksumType;
+import org.apache.pulsar.tests.EnumValuesDataProvider;
+import org.awaitility.Awaitility;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+public class MessageChecksumTest extends BrokerTestBase {

Review comment:
       This should be a separate pull request. Don't couple multiple changes together here.

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import io.netty.channel.EventLoopGroup;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.io.IOException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
+import org.apache.pulsar.common.util.netty.EventLoopUtil;
+import org.awaitility.Awaitility;
+
+/**
+ * A Pulsar Client that is used for testing scenarios where the different
+ * asynchronous operations of the client-broker interaction must be orchestrated by the test
+ * so that race conditions caused by the test code can be eliminated.
+ *
+ * features:
+ * - can override remote endpoint protocol version in a thread safe manner
+ * - can reject new connections from the client to the broker
+ * - can drop all OpSend messages after they have been added to pendingMessages and processed
+ *   by the client. This simulates a situation where sending messages go to a "black hole".
+ * - can synchronize operations with the help of the pending message callback which gets
+ *   called after the message to send out has been added to the pending messages in the client.
+ *
+ */
+public class PulsarTestClient extends PulsarClientImpl {

Review comment:
       same comment above

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -669,11 +669,23 @@ public void close() throws IOException {
         });
 
         if (listenChannel != null) {
-            listenChannel.close();
+            try {
+                if (!listenChannel.close().await(30, TimeUnit.SECONDS)) {

Review comment:
       Please make timeout configurable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#issuecomment-767649620


   /pulsarbot run-failure-checks


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563630279



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -512,6 +512,7 @@ public void shutdown() throws InterruptedException, ManagedLedgerException {
             ledger.asyncClose(new AsyncCallbacks.CloseCallback() {
                 @Override
                 public void closeComplete(Object ctx) {
+                    log.debug("[{}] Closed managed ledger.", ledger.getName());

Review comment:
       I have removed this change from this PR.

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -542,8 +543,17 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) {
         }
 
         scheduledExecutor.shutdownNow();
-        orderedExecutor.shutdownNow();
+        if (!scheduledExecutor.awaitTermination(30, TimeUnit.SECONDS)) {
+           log.error("Cannot stop scheduledExecutor gracefully.");
+        }
+        orderedExecutor.shutdown();

Review comment:
       I have removed this change from this PR.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563602467



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java
##########
@@ -0,0 +1,218 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import io.netty.channel.EventLoopGroup;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.io.IOException;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
+import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
+import org.apache.pulsar.common.util.netty.EventLoopUtil;
+import org.awaitility.Awaitility;
+
+/**
+ * A Pulsar Client that is used for testing scenarios where the different
+ * asynchronous operations of the client-broker interaction must be orchestrated by the test
+ * so that race conditions caused by the test code can be eliminated.
+ *
+ * features:
+ * - can override remote endpoint protocol version in a thread safe manner
+ * - can reject new connections from the client to the broker
+ * - can drop all OpSend messages after they have been added to pendingMessages and processed
+ *   by the client. This simulates a situation where sending messages go to a "black hole".
+ * - can synchronize operations with the help of the pending message callback which gets
+ *   called after the message to send out has been added to the pending messages in the client.
+ *
+ */
+public class PulsarTestClient extends PulsarClientImpl {

Review comment:
       +1, I hope this gets addressed when I revisit the PR and remove unrelated changes. Currently the PulsarTestClient is needed to fix the flaky tests, so perhaps it could be kept in this PR after the unrelated changes have been removed? PTAL @sijie 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r562680344



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -408,7 +408,7 @@ public void initializeFailed(ManagedLedgerException e) {
                         pendingLedger.ledger.asyncClose(new CloseCallback() {
                             @Override
                             public void closeComplete(Object ctx) {
-                                // no-op
+                                log.info("[{}] Closed pending initializer managed ledger.", name);

Review comment:
       probably we want to leave this at `debug` level

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java
##########
@@ -512,6 +512,7 @@ public void shutdown() throws InterruptedException, ManagedLedgerException {
             ledger.asyncClose(new AsyncCallbacks.CloseCallback() {
                 @Override
                 public void closeComplete(Object ctx) {
+                    log.info("[{}] Closed managed ledger.", ledger.getName());

Review comment:
       probably we want to leave this at `debug` level

##########
File path: pulsar-io/hbase/pom.xml
##########
@@ -70,12 +70,6 @@
             <version>${hbase.version}</version>
         </dependency>
 
-        <dependency>

Review comment:
       this is patch of another patch




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on a change in pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#discussion_r563599940



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java
##########
@@ -0,0 +1,249 @@
+/**
+ * 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.pulsar.client.impl;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.pulsar.broker.service.BrokerTestBase;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.MessageRoutingMode;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.TypedMessageBuilder;
+import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg;
+import org.apache.pulsar.common.api.proto.MessageMetadata;
+import org.apache.pulsar.common.api.proto.ProtocolVersion;
+import org.apache.pulsar.common.protocol.ByteBufPair;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.protocol.Commands.ChecksumType;
+import org.apache.pulsar.tests.EnumValuesDataProvider;
+import org.awaitility.Awaitility;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+public class MessageChecksumTest extends BrokerTestBase {

Review comment:
       Thanks for the suggestion of splitting the changes to separate pull requests. I'll remove some changes from this PR to fix the flaky MessageIdTest. I was thinking that this change to split MessageChecksumTest from MessageIdTest is the relevant for fixing the flaky MessageIdTest, so I was thinking to to keep it in this PR, but move the other unrelated parts to new PRs. I hope this will be an acceptable way to resolve this. PTAL @sijie




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] lhotari commented on pull request #9286: Fix flaky MessageIdTest and introduce some testing improvements

Posted by GitBox <gi...@apache.org>.
lhotari commented on pull request #9286:
URL: https://github.com/apache/pulsar/pull/9286#issuecomment-767070975


   > It seems there's a genuine test failure: 
   
   @merlimat Thanks for the heads up. I'll address it tomorrow.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org