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 2020/05/04 18:29:48 UTC

[GitHub] [pulsar] zyllt opened a new issue #6872: Producer cannot connect after Unloading bundle

zyllt opened a new issue #6872:
URL: https://github.com/apache/pulsar/issues/6872


   **Describe the bug**
   I have 2 pulsar clusters in Beijing and Guangzhou, each with 2 physics machine deployed broker nodes.
   1、There are 5 producers in this Topic. After multiple shed loads it was found that some of the producers were unable to connect to the pulsar, and the producers that were unable to connect at the same time when the broker of the ownership was different. For example, broker numbers A and B, producers 1,2 and 3 can't connect when the townership of that topic is A; 3, 4, and 5 can't connect when the ownership is B.
   checking the broker log revealed that there was an exception that kept outputting `Producer with id persistent://xxx/xxx/xxx is already present on the connection`.
   I looked at the broker logs and found that when `Unloading bundle` occurs, a producer node will output a log of a `[ForkJoinPool.commonPool-worker-24] INFO  org.apache.pulsar.broker.service.ServerCnx - [/10.1.116.123:15070][xxxx] Creating producer. producerId=0` immediately after `Disconnecting producer:`, and this `Creating producer` has no subsequent results (either created successfully or failed). Then when the Unloading bundle occurs again and the downtownership of the topic returns to the broker, the producer node keeps having an exception that is `Producer with id [xxx] already present on the connection`.
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L900
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java#L516
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L892
   
   2、geo-replication of an exception `Producer with name 'pulsar.repl.bj-xxx-product' is already connected to topic`
   
   **To Reproduce**
   Unfortunately, I cannot reproduce the problem in a local environement.But now in the production environment has emerged.
   
   **Expected behavior**
   The producer can connect properly.
   
   **Screenshots**
   ![image](https://user-images.githubusercontent.com/8822016/80999100-b60fdf00-8e76-11ea-8042-94a1bb67828c.png)
   
   geo-replication
   ![image](https://user-images.githubusercontent.com/8822016/80998885-4a2d7680-8e76-11ea-9fad-93d409f75976.png)
   **Desktop (please complete the following information):**
   pulsar 2.5.0
   
   **Additional context**
   Attached is the log of one of the unloading bundles.
   [UnloadingBundle.log](https://github.com/apache/pulsar/files/4576535/UnloadingBundle.log)
   


----------------------------------------------------------------
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] zyllt removed a comment on issue #6872: Producer cannot connect after Unloading bundle

Posted by GitBox <gi...@apache.org>.
zyllt removed a comment on issue #6872:
URL: https://github.com/apache/pulsar/issues/6872#issuecomment-624138264


   @codelipenghui at  
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L639
   Only `thenApply` method was called, if `topicFuture` is exception here it should stay stuck,then 
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L902
   There's no way to get it done here.
   I was hoping to get confirmation from the log, but all I could find this log at 
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L1208
   but at 
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L665
   the log is has no output.


----------------------------------------------------------------
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] zyllt commented on issue #6872: Producer cannot connect after Unloading bundle

Posted by GitBox <gi...@apache.org>.
zyllt commented on issue #6872:
URL: https://github.com/apache/pulsar/issues/6872#issuecomment-624138264


   @codelipenghui at  
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L639
   Only `thenApply` method was called, if `topicFuture` is exception here it should stay stuck,then 
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L902
   There's no way to get it done here.
   I was hoping to get confirmation from the log, but all I could find this log at 
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L1208
   but at 
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L665
   the log is has no output.


----------------------------------------------------------------
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] codelipenghui commented on issue #6872: Producer cannot connect after Unloading bundle

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on issue #6872:
URL: https://github.com/apache/pulsar/issues/6872#issuecomment-624007410


   @zyllt Looks the problem is related to https://github.com/apache/pulsar/issues/6416 and 
    @addisonj pushed a PR https://github.com/apache/pulsar/pull/6489 for fixing this problem and released it in version 2.5.1. You can try out 2.5.1 and If you have any news about this issue, we can discuss it here.


----------------------------------------------------------------
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] addisonj commented on issue #6872: Producer cannot connect after Unloading bundle

Posted by GitBox <gi...@apache.org>.
addisonj commented on issue #6872:
URL: https://github.com/apache/pulsar/issues/6872#issuecomment-624204982


   Now that I see the real problem here I think i can get a patch in real quick
   


----------------------------------------------------------------
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] zyllt commented on issue #6872: Producer cannot connect after Unloading bundle

Posted by GitBox <gi...@apache.org>.
zyllt commented on issue #6872:
URL: https://github.com/apache/pulsar/issues/6872#issuecomment-624426457


   when `PersistentTopic#close()`,Can `brokerService.removeTopicFromCache(topic)` be called before `disconnect`?


----------------------------------------------------------------
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] addisonj commented on issue #6872: Producer cannot connect after Unloading bundle

Posted by GitBox <gi...@apache.org>.
addisonj commented on issue #6872:
URL: https://github.com/apache/pulsar/issues/6872#issuecomment-624139824


   @codelipenghui @zyllt I should update those issues, but will put some context here as well.
   
   I am still experiencing this same issue with 2.5.1.  I am currently knee deep in trying to track it down. I still see producerFutures that are "stuck" and not being cleaned up. From what I can tell, this only occurs if unloads happen in somewhat quick succession, where the producerFuture is not completed before another unload happens OR we are seeing issues with ownership, with multiple brokers have ownership of a bundle (still trying to dig through logs to determine which).
   
   Regardless, I think this is exacerbated by certain patterns that cause creation of producers to be delayed or where the number of producers/subscriptions places heavy load on metadata. In my case, I am writing a pulsar source which writes to hundred of different topics but at a somewhat low rate.  


----------------------------------------------------------------
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] addisonj commented on issue #6872: Producer cannot connect after Unloading bundle

Posted by GitBox <gi...@apache.org>.
addisonj commented on issue #6872:
URL: https://github.com/apache/pulsar/issues/6872#issuecomment-624204832


   Okay, I have logs that I think show what is exactly going on here, this is a sample from a single broker for a single topic that illustrates the problem:
   
   ```
   [persistent://canvas-cdc/prod-iad/cluster96-canvas.cluster44_shard_14377.assessment_question_banks-partition-0] Unloading topic
   Disconnecting producer: Producer{topic=PersistentTopic{topic=persistent://canvas-cdc/prod-iad/cluster96-canvas.cluster44_shard_14377.assessment_question_banks-partition-0}, client=/10.9.26.114:53504, producerName=pulsar-prod-iad-418-15948, producerId=69}
   [/10.9.26.114:53504][persistent://canvas-cdc/prod-iad/cluster96-canvas.cluster44_shard_14377.assessment_question_banks-partition-0] Creating producer. producerId=69
   [persistent://canvas-cdc/prod-iad/cluster96-canvas.cluster44_shard_14377.assessment_question_banks-partition-0] Topic closed
   [/10.9.26.114:53504] persistent://canvas-cdc/prod-iad/cluster96-canvas.cluster44_shard_14377.assessment_question_banks-partition-0 configured with schema false
   Namespace bundle for topic (persistent://canvas-cdc/prod-iad/cluster96-canvas.cluster44_shard_14377.assessment_question_banks-partition-0) not served by this instance. Please redo the lookup. Request is denied: namespace=canvas-cdc/prod-iad
   ```
   The most interesting bit here is that a producer is re-created after the topic starts unloading and is able to proceed down to where it tries to load a schema (which indicates it was able to load the topic)
   
   On the client, we see the following:
   
   ```
   [persistent://canvas-cdc/prod-iad/cluster96-canvas.cluster44_shard_14377.assessment_question_banks-partition-0] [pulsar-prod-iad-418-15948] Creating producer on cnx [id: 0xfe4078a0, L:/10.9.26.114:53504 - R:pulsar-prod-broker-4.pulsar-prod-broker.pulsar.svc.cluster.local/10.9.62.180:6651]
   [persistent://canvas-cdc/prod-iad/cluster96-canvas.cluster44_shard_14377.assessment_question_banks-partition-0] [pulsar-prod-iad-418-15948] Failed to create producer: 164795 lookup request timedout after ms 30000
   ```
   
   After the lookup timeout, the client retries, which now properly gets routed to the new broker, and succeeds.
   
   However, some hours later, after the topic is getting reloaded back onto the same broker, the very first message we see, even before a ledger is opened is the following:
   
   ```
   [/10.9.26.114:53504][69] Producer with id persistent://canvas-cdc/prod-iad/cluster96-canvas.cluster44_shard_14377.assessment_question_banks-partition-0 is already present on the connection
   ```
   
   From these logs and the code we can assuming the following:
   1. the topic starts unloading
   2. during unload, the producer quickly tries to reconnect. It does so and is NOT rejected, and is able to get through all this chunk of code:
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L902-L945
   3. It calls into `topics.addProducer` which rechecks the namespace ownership. This throws a RuntimeException (at https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L1211) , however in https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L971 we ONLY catch a BrokerServiceException, since the exception is un-handled, it also doesn't hit the final chunk to remove the producer
   4. the client times out as it never gets a response and the producer just stays in the cache


----------------------------------------------------------------
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] addisonj edited a comment on issue #6872: Producer cannot connect after Unloading bundle

Posted by GitBox <gi...@apache.org>.
addisonj edited a comment on issue #6872:
URL: https://github.com/apache/pulsar/issues/6872#issuecomment-624139824


   @codelipenghui @zyllt I should update those issues, but will put some context here as well.
   
   I am still experiencing this same issue with 2.5.1.  I am currently knee deep in trying to track it down. I still see producerFutures that are "stuck" and not being cleaned up. ~~From what I can tell, this only occurs if unloads happen in somewhat quick succession, where the producerFuture is not completed before another unload happens OR we are seeing issues with ownership, with multiple brokers have ownership of a bundle (still trying to dig through logs to determine which).~~ EDIT: I had misinterpreted some logs, this is wrong
   
   Regardless, I think this is exacerbated by certain patterns that cause creation of producers to be delayed or where the number of producers/subscriptions places heavy load on metadata. In my case, I am writing a pulsar source which writes to hundred of different topics but at a somewhat low rate.  


----------------------------------------------------------------
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] zyllt commented on issue #6872: Producer cannot connect after Unloading bundle

Posted by GitBox <gi...@apache.org>.
zyllt commented on issue #6872:
URL: https://github.com/apache/pulsar/issues/6872#issuecomment-624090531


   @codelipenghui  thx,i also think the reason is dangling `ProducerFuture`.I rechecked the log by #6416 and found that the thread outputting `Creating producer` at
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L900
   is not the same as the thread for `Namespace bundle for topic` at 
   https://github.com/apache/pulsar/blob/f2afad353795f70c00910b6dcce5df8c62b94a67/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java#L1210
   which makes me confused.
   ![image](https://user-images.githubusercontent.com/8822016/81077540-ac8e8180-8f1f-11ea-992f-77f818f97147.png)
   


----------------------------------------------------------------
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