You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by "jiangpengcheng (via GitHub)" <gi...@apache.org> on 2024/01/17 11:11:27 UTC

[I] [Bug] Functions failed to recover after a crash if the input topic is partitioned and function is using `EFFECTIVELY_ONCE` [pulsar]

jiangpengcheng opened a new issue, #21911:
URL: https://github.com/apache/pulsar/issues/21911

   ### Search before asking
   
   - [X] I searched in the [issues](https://github.com/apache/pulsar/issues) and found nothing similar.
   
   
   ### Version
   
   3.1.1
   
   ### Minimal reproduce step
   
   1. Create a partitioned topic with partition=5
       ```
       bin/pulsar-admin topics create-partitioned-topic persistent://public/default/test-topic -p 5
       ```
   
   2. Create a new Function with replicas=5, and semantic to `EFFECTIVELY_ONCE`(using k8s runtime or function-mesh)
   
       ```
       bin/pulsar-admin functions create --jar /pulsar/examples/api-examples.jar --name test-func --className org.apache.pulsar.functions.api.examples.ExclamationFunction --inputs persistent://public/default/test-topic -o persistent://public/default/test-output --processing-guarantees EFFECTIVELY_ONCE --parallelism 5
       ```
   
   3. After all function pods are running, sending some message to the input topic
   
       ```
       bin/pulsar-client produce -m "hello" -n 100 persistent://public/default/test-topic 
       ```
   
   4. Delete 1 function pod, such as pod-0 to simulate crash
   
       ```
       kubectl delete pod test-func-function-0
       ```
   
   5. After the pod-0 restarted, sending messges to the input topic again
   
       ```
       bin/pulsar-client produce -m "hello" -n 100 persistent://public/default/test-topic 
       ``` 
   
   6. Check the status of function pods
   
       ```
       kubectl get pod
       ```
   
   ### What did you expect to see?
   
   All functions pods are running well
   
   ### What did you see instead?
   
   Some of function pods are keeping restarting
   
   ### Anything else?
   
   The reason is that with `EFFECTIVELY_ONCE` semantic, function pods will subscribe to the input topic using `FAILOVER` mode, at the beginning, the subscription relation is like below:
   
   pod0 -> partition0
   pod1 -> partition1
   pod2 -> partition2
   pod3 -> partition3
   pod4 -> partition4
   
   and when pod-0 is deleted, partition-0 will be assigned to another pod, such as pod-4, then pod-4 will create a producer with partition id like `persistent://public/default/test-topic-0-0` as the name
   
   then when pod-0 restarted, the pulsar broker will do a "shuffle" for the subscription, the relation may look like:
   
   pod0 -> partition0
   pod1 -> partition3
   pod2 -> partition1
   pod3 -> partition4
   pod4 -> partition2
   
   when new messages come to partition0, the pod0 cannot create a producer with name `persistent://public/default/test-topic-0-0` since it's already created in pod4, and so are other pods:
   
   - pod1 failed to create producer with name `persistent://public/default/test-topic-3-0` since it's already created in pod3
   - pod2 failed to create producer with name `persistent://public/default/test-topic-1-0` since it's already created in pod1
   - pod3 failed to create producer with name `persistent://public/default/test-topic-4-0` since it's already created in pod4
   - pod4 failed to create producer with name `persistent://public/default/test-topic-2-0` since it's already created in pod2
   
   as an end, no pods are able to start
   
   ### Are you willing to submit a PR?
   
   - [X] I'm willing to submit a 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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org.apache.org

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