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/09/26 04:07:12 UTC

[GitHub] [pulsar] lordcheng10 opened a new pull request #12195: When calling producer.close or client.close to close the client, the …

lordcheng10 opened a new pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195


   ### Motivation
   In the following example, the data will be lost:
   ` try {
               PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://127.0.0.1:6650").build();
   
               Producer<byte[]> producer = client.newProducer().topic("test1").create();
   
               producer.sendAsync("hello".getBytes());
               client.close();
           } catch (PulsarClientException e) {
               e.printStackTrace();
           }
   
           Thread.sleep(1000);`
   
   The reason is because the producer did not send the data in the client buffer when it was closed;
   
   


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


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

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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


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

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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   Can you review it? @eolivelli 


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

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



[GitHub] [pulsar] lordcheng10 commented on a change in pull request #12195: When the producer or client is normally closed, data will be lost

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



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java
##########
@@ -843,72 +843,78 @@ protected WriteInEventLoopCallback newObject(Handle<WriteInEventLoopCallback> ha
 
     @Override
     public CompletableFuture<Void> closeAsync() {
-        final State currentState = getAndUpdateState(state -> {
-            if (state == State.Closed) {
-                return state;
-            }
-            return State.Closing;
-        });
-
-        if (currentState == State.Closed || currentState == State.Closing) {
-            return CompletableFuture.completedFuture(null);
-        }
-
-        Timeout timeout = sendTimeout;
-        if (timeout != null) {
-            timeout.cancel();
-            sendTimeout = null;
-        }
+        CompletableFuture<Void> flushAndCloseFuture = new CompletableFuture<>();
+        flushAsync().thenRun(() -> {

Review comment:
       You're right. I'll try to modify 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.

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

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



[GitHub] [pulsar] BewareMyPower commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   I agree with @eolivelli . IMO It should be implemented as a new API like `flushAndClose`, or some extra configs for client or producer. 


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

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



[GitHub] [pulsar] michaeljmarshall commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   https://github.com/apache/pulsar/blob/04604724dd89a4c8ff40b922e80c7045fedf112d/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Producer.java#L178-L186
   
   Given the Javadoc for `closeAsync`, I would expect the implementation to include a flush of all pending messages. Perhaps there is ambiguity for what constitutes a "pending write request"? I think a "pending write request" is any write request the producer has already accepted. This also aligns with the fact that the producer will not accept any more write requests.
   
   > First, we cannot assume pending messages are sent quickly. If your buffer memory is large enough, it might take long time to close. Assuming you have 100000 pending messages and in the timeout, only 20000 messages are persisted. What will you do now?
   
   If we add a timeout to the `closeAsync` logic, I think it is reasonable to fail all pending work that is not completed before the deadline and clean up all resources. It is up to the implementation to decide how long to wait for messages to deliver, and we already let users configure this via the `sendTimeoutMs` in the producer's config. This config is already used to fail messages that haven't delivered after some period of time.


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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






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

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



[GitHub] [pulsar] BewareMyPower edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
BewareMyPower edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927451267


   > The flushing on close was already the pre-existing behavior
   
   @merlimat I'm afraid not, in current `ProducerImpl#closeAsync`, before sending `CommandClose`, producer only cancels all timers, then call `clearPendingMessagesWhenClose` to complete all pending messages' callbacks with `AlreadyClosedException`.
   
   > Of course, we can also provide a close method with a timeout to satisfy some users who do not want the close method to take too much time.
   
   It's not so easy like you might think. We need to handle more corner cases if you added the flush semantics to `close`.
   
   First, we cannot assume pending messages are sent quickly. If your buffer memory is large enough, it might take long time to close. Assuming you have 100000 pending messages and in the timeout, only 20000 messages are persisted. What will you do now?
   1. Continue sending the left 80000 messages. It means your application continues after `close`, but the background thread is still running. To ensure messages are not lost, you need to keep the connection open. Then it's interesting that a client is closed while the resources are still active, including the connection, pending messages, etc.
   2. Discard the left 80000 messages. Then data is lost. In addition, what's different from Kafka, Pulsar producer will send a `CommandCloseProducer` to notify the broker to close the producer. What's more, it make your guarantee that messages are not lost after calling synchronous `close` not work.
   
   At any case, when you choose `sendAsync`, you should always make use of the returned future to confirm the result of all messages. In Kafka, it's the send callback.
   
   Assuming now `close` will wait until all messages are persisted. Then you might write following pseudo code.
   
   ```java
   for (int i = 0; i < N; i++) {
       producer.sendAsync("message-i");
   }
   producer.close();
   ```
   
   What if `close()` failed? Then how would you check which messages are failed to send? You still need to do something with the future that `sendAsync` returns. Kafka's `close` only looks good for those don't care about any error and think all API calls should always succeed.


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

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



[GitHub] [pulsar] BewareMyPower edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
BewareMyPower edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-928694041


   @lordcheng10 If you're going to submit a PIP, please follow the PIP template.
   
   But before submitting a PIP, it's better to send an email to dev@pulsar.apache.org to start a discussion so that more people know the context.


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

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



[GitHub] [pulsar] BewareMyPower commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   I've started a discussion, see https://lists.apache.org/thread.html/r8bfcb7ab28612d94d441ff5eadd996413346f0780b6f7b3484aaf7dc%40%3Cdev.pulsar.apache.org%3E


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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927322511


   Thank you for your attention!
   
   For similar producer close logic, we can refer to Kafka's processing: https://cwiki.apache.org/confluence/display/KAFKA/KIP-8+-+Add+a+flush+method+to+the+producer+API
   
   It may be more reasonable for the producer to flush during close. When closing, sending the unsent data to the server is a common demand of users. No user would be more willing to discard the data in the buffer.
   
   Of course, we can also provide a close method with a timeout to satisfy some users who do not want the close method to take too much time.
   
   @BewareMyPower @eolivelli @merlimat @sijie @hangc0276  


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

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



[GitHub] [pulsar] Shoothzj commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   LGTM @eolivelli, @codelipenghui, @BewareMyPower, @sijie, @hangc0276, @merlimat - PTAL, thanks.


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

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



[GitHub] [pulsar] BewareMyPower commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   IMO it's the right behavior. `close` or `closeAsync` should not be responsible to flush the internal messages.


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

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



[GitHub] [pulsar] Anonymitaet commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   @lordcheng10 feel free to ping me if you need a doc review.


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

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



[GitHub] [pulsar] Shoothzj commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   This method is `closeAsync`, I would think call `flushAsync` would be better.
   You can add a tests, the example is a little flaky, batch producer with batch time (like 5s) will be better


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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927269367


   > I mean that when the `flushAsync` future complete, then do the logic. So the `closeAsync` method will not block
   
   You are right, I will try to modify 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.

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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927322511


   Thank you for your attention!
   
   For similar producer close logic, we can refer to Kafka's processing: 
   ![image](https://user-images.githubusercontent.com/19296967/134814274-119a88e3-4880-4af7-8b72-3c144ce5d17c.png)
   
   
   It may be more reasonable for the producer to flush during close. When closing, sending the unsent data to the server is a common demand of users. No user would be more willing to discard the data in the buffer.
   
   Of course, we can also provide a close method with a timeout to satisfy some users who do not want the close method to take too much time.
   
   @BewareMyPower @eolivelli @merlimat @sijie @hangc0276  


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

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



[GitHub] [pulsar] BewareMyPower edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
BewareMyPower edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-928689470


   > Perhaps there is ambiguity for what constitutes a "pending write request"? 
   
   I also found the problem. I think it's an inconsistency between the JavaDocs and the actual implementation. But the definition of **pending write request** is ambiguous. Should it be the inflight `CommandSend` network requests or all the pending messages in producer's buffer?
   
   The description of `flushAsync` is
   
   >  Flush **all the messages buffered** in the client
   
   The description of `closeAsync` is
   
   > Waits until **all pending write request** are persisted.


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   > flushAsync
   
   Thank you for your attention! My understanding is that the original close cleanup logic can only be done after the flush is completed. If the flushAsync method is called, the flush is not guaranteed to be completed.


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

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



[GitHub] [pulsar] michaeljmarshall commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   https://github.com/apache/pulsar/blob/04604724dd89a4c8ff40b922e80c7045fedf112d/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Producer.java#L178-L186
   
   Given the Javadoc for `closeAsync`, I would expect the implementation to include a flush of all pending messages. Perhaps there is ambiguity for what constitutes a "pending write request"? I think a "pending write request" is any write request the producer has already accepted. This also aligns with the fact that the producer will not accept any more write requests.
   
   > First, we cannot assume pending messages are sent quickly. If your buffer memory is large enough, it might take long time to close. Assuming you have 100000 pending messages and in the timeout, only 20000 messages are persisted. What will you do now?
   
   If we add a timeout to the `closeAsync` logic, I think it is reasonable to fail all pending work that is not completed before the deadline and clean up all resources. It is up to the implementation to decide how long to wait for messages to deliver, and we already let users configure this via the `sendTimeoutMs` in the producer's config. This config is already used to fail messages that haven't delivered after some period of time.


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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927248432


   > flushAsync
   
   Thank you for your attention! The cleanup logic can only be done after the flush is completed. If the flushAsync method is used instead of the flush method, there is no guarantee that all the data has been sent to the server before the cleanup logic is completed;


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

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



[GitHub] [pulsar] eolivelli commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   I understand that current behaviour may be surprising for users.
    Probably it may be good to add a method 'flush and close' but this can be easily implemented by the application.
   
   We must ensure that the javadoc are clear about the behaviour.
   
   I suggest you to start a discussion on dev@pulsar.apache.org in order to reach out to a bigger audience.
   In any case this kind of changes will need a PIP because it would be an important behaviour change


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

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



[GitHub] [pulsar] liangyuanpeng commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   I believe that it will be resolved https://github.com/apache/pulsar/issues/11780 .


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   > I understand that current behaviour may be surprising for users.
   > Probably it may be good to add a method 'flush and close' but this can be easily implemented by the application.
   > 
   > We must ensure that the javadoc are clear about the behaviour.
   > 
   > I suggest you to start a discussion on [dev@pulsar.apache.org](mailto:dev@pulsar.apache.org) in order to reach out to a bigger audience.
   > In any case this kind of changes will need a PIP because it would be an important behaviour change
   
   I submitted a PIP: https://github.com/apache/pulsar/issues/12216  @eolivelli 


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

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



[GitHub] [pulsar] merlimat commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   The flushing on close was already the pre-existing behavior, though that got lost at some point. 


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

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



[GitHub] [pulsar] BewareMyPower commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   > The flushing on close was already the pre-existing behavior
   
   @merlimat I'm afraid not, in current `ProducerImpl#closeAsync`, before sending `CommandClose`, producer only cancels all timers, then call `clearPendingMessagesWhenClose` to complete all pending messages' callbacks with `AlreadyClosedException`.
   
   > Of course, we can also provide a close method with a timeout to satisfy some users who do not want the close method to take too much time.
   
   It's not so easy like you might think. We need to handle more corner cases if you added the flush semantics to `close`.
   
   First, we cannot assume pending messages are sent quickly. If your buffer memory is large enough, it might take long. Assuming you have 100000 pending messages and in the timeout, only 20000 messages are persisted. What should you do now?
   1. Continue sending the left 80000 messages. It means your application continues after `close`, but the background thread is still running. To ensure messages are not lost, you need to keep the connection open. Then it's interesting that a client is closed while the resources are still active, including the connection, pending messages, etc.
   2. Discard the left 80000 messages. Then data is lost. In addition, what's different from Kafka, Pulsar producer will send a `CommandCloseProducer` to notify the broker to close the producer. What's more, it make your guarantee that messages are not lost after calling synchronous `close` not work.
   
   At any case, when you choose `sendAsync`, you should always make use of the returned future to confirm the result of all messages. In Kafka, it's the send callback.
   
   Assuming now `close` will wait until all messages are persisted. Then you might write following pseudo code.
   
   ```java
   for (int i = 0; i < N; i++) {
       producer.sendAsync("message-i");
   }
   producer.close();
   ```
   
   What if `close()` failed? Then how would you check which messages are failed to send? You still need to do something with the future that `sendAsync` returns. Kafka's `close` only looks good for those don't care about any error and think all API calls should always succeed.


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


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

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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927322511


   Thank you for your attention!
   
   For similar producer close logic, we can refer to Kafka's processing: Close this producer. This method blocks until all previously sent requests complete.
   
   It may be more reasonable for the producer to flush during close. When closing, sending the unsent data to the server is a common demand of users. No user would be more willing to discard the data in the buffer.
   
   Of course, we can also provide a close method with a timeout to satisfy some users who do not want the close method to take too much time.
   
   @BewareMyPower @eolivelli @merlimat @sijie @hangc0276  


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

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



[GitHub] [pulsar] BewareMyPower edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
BewareMyPower edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-928689470






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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927322511


   Thank you for your attention!
   
   For similar producer close logic, we can refer to Kafka's processing: 
   ![image](https://user-images.githubusercontent.com/19296967/134814226-c6f14a38-ddcc-43c6-a83c-764e5e55d842.png)
   
   It may be more reasonable for the producer to flush during close. When closing, sending the unsent data to the server is a common demand of users. No user would be more willing to discard the data in the buffer.
   
   Of course, we can also provide a close method with a timeout to satisfy some users who do not want the close method to take too much time.
   
   @BewareMyPower @eolivelli @merlimat @sijie @hangc0276  


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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-928645314


   > I understand that current behaviour may be surprising for users.
   > Probably it may be good to add a method 'flush and close' but this can be easily implemented by the application.
   > 
   > We must ensure that the javadoc are clear about the behaviour.
   > 
   > I suggest you to start a discussion on [dev@pulsar.apache.org](mailto:dev@pulsar.apache.org) in order to reach out to a bigger audience.
   > In any case this kind of changes will need a PIP because it would be an important behaviour change
   
   As you describe, I created a PIP: https://github.com/apache/pulsar/issues/12216  @eolivelli 


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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-928645314


   > I understand that current behaviour may be surprising for users.
   > Probably it may be good to add a method 'flush and close' but this can be easily implemented by the application.
   > 
   > We must ensure that the javadoc are clear about the behaviour.
   > 
   > I suggest you to start a discussion on [dev@pulsar.apache.org](mailto:dev@pulsar.apache.org) in order to reach out to a bigger audience.
   > In any case this kind of changes will need a PIP because it would be an important behaviour change
   
   As you describe, I created a PIP: https://github.com/apache/pulsar/issues/12216  @eolivelli 


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

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



[GitHub] [pulsar] BewareMyPower commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   > Perhaps there is ambiguity for what constitutes a "pending write request"? 
   
   I also found the problem. I think it's an inconsistency between the JavaDocs and the actual implementation. But the definition of **pending write request** is ambiguous. Should it be the inflight `CommandSend` network requests or all the pending messages in producer's buffer? But


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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927322511


   Thank you for your attention!
   
   For similar producer close logic, we can refer to Kafka's processing: https://github.com/apache/kafka/commit/0636928d961a6ceaab46d908f9372d913c3e5faf
   
   It may be more reasonable for the producer to flush during close. When closing, sending the unsent data to the server is a common demand of users. No user would be more willing to discard the data in the buffer.
   
   Of course, we can also provide a close method with a timeout to satisfy some users who do not want the close method to take too much time.
   
   @BewareMyPower @eolivelli @merlimat @sijie @hangc0276  


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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927322511


   Thank you for your attention!
   
   For similar producer close logic, we can refer to Kafka's processing: https://cwiki.apache.org/confluence/display/KAFKA/KIP-8+-+Add+a+flush+method+to+the+producer+API
   
   It may be more reasonable for the producer to flush during close. When closing, sending the unsent data to the server is a common demand of users. No user would be more willing to discard the data in the buffer.
   
   Of course, we can also provide a close method with a timeout to satisfy some users who do not want the close method to take too much time.
   
   @BewareMyPower @eolivelli @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.

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

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



[GitHub] [pulsar] liangyuanpeng commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   I believe that it will be resolved https://github.com/apache/pulsar/issues/11780 .


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

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



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #12195: When the producer or client is normally closed, data will be lost

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



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java
##########
@@ -843,72 +843,78 @@ protected WriteInEventLoopCallback newObject(Handle<WriteInEventLoopCallback> ha
 
     @Override
     public CompletableFuture<Void> closeAsync() {
-        final State currentState = getAndUpdateState(state -> {
-            if (state == State.Closed) {
-                return state;
-            }
-            return State.Closing;
-        });
-
-        if (currentState == State.Closed || currentState == State.Closing) {
-            return CompletableFuture.completedFuture(null);
-        }
-
-        Timeout timeout = sendTimeout;
-        if (timeout != null) {
-            timeout.cancel();
-            sendTimeout = null;
-        }
+        CompletableFuture<Void> flushAndCloseFuture = new CompletableFuture<>();
+        flushAsync().thenRun(() -> {
+            final State currentState = getAndUpdateState(state -> {
+                if (state == State.Closed) {
+                    return state;
+                }
+                return State.Closing;
+            });

Review comment:
       I think we should set the state of the producer to `Closing` before triggering the flush. Otherwise, a message could be added to `batchMessageContainer` after the flush and before the state is set to `Closing`.

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java
##########
@@ -843,72 +843,78 @@ protected WriteInEventLoopCallback newObject(Handle<WriteInEventLoopCallback> ha
 
     @Override
     public CompletableFuture<Void> closeAsync() {
-        final State currentState = getAndUpdateState(state -> {
-            if (state == State.Closed) {
-                return state;
-            }
-            return State.Closing;
-        });
-
-        if (currentState == State.Closed || currentState == State.Closing) {
-            return CompletableFuture.completedFuture(null);
-        }
-
-        Timeout timeout = sendTimeout;
-        if (timeout != null) {
-            timeout.cancel();
-            sendTimeout = null;
-        }
+        CompletableFuture<Void> flushAndCloseFuture = new CompletableFuture<>();
+        flushAsync().thenRun(() -> {

Review comment:
       I am pretty sure we want to call `triggerFlush` instead of `flushAsync`. The future returned by `flushAsync` won't complete until the message has delivered and been acked. All we're woried about is ensuring that the buffered message has been sent on the TCP connection before sending the `CLOSE_PRODUCER` command. `triggerFlush` gives us these semantics.
   
   As a consequence, we won't need to wrap the majority of this method in a callback, and we won't need the `flushAndCloseFuture` variable.




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

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



[GitHub] [pulsar] lordcheng10 commented on a change in pull request #12195: When the producer or client is normally closed, data will be lost

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



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java
##########
@@ -843,72 +843,78 @@ protected WriteInEventLoopCallback newObject(Handle<WriteInEventLoopCallback> ha
 
     @Override
     public CompletableFuture<Void> closeAsync() {
-        final State currentState = getAndUpdateState(state -> {
-            if (state == State.Closed) {
-                return state;
-            }
-            return State.Closing;
-        });
-
-        if (currentState == State.Closed || currentState == State.Closing) {
-            return CompletableFuture.completedFuture(null);
-        }
-
-        Timeout timeout = sendTimeout;
-        if (timeout != null) {
-            timeout.cancel();
-            sendTimeout = null;
-        }
+        CompletableFuture<Void> flushAndCloseFuture = new CompletableFuture<>();
+        flushAsync().thenRun(() -> {
+            final State currentState = getAndUpdateState(state -> {
+                if (state == State.Closed) {
+                    return state;
+                }
+                return State.Closing;
+            });

Review comment:
       you are right!




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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   > @lordcheng10 Thanks for your contribution. For this PR, do we need to update docs?
   > 
   > (The [PR template contains info about doc](https://github.com/apache/pulsar/blob/master/.github/PULL_REQUEST_TEMPLATE.md#documentation), which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)
   
   Okay, when this discussion is completed, I will update docs。


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

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



[GitHub] [pulsar] liangyuanpeng edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
liangyuanpeng edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-928767615


   I believe that it will be resolve https://github.com/apache/pulsar/issues/11780 .


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

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



[GitHub] [pulsar] BewareMyPower commented on pull request #12195: When the producer or client is normally closed, data will be lost

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






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

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



[GitHub] [pulsar] lordcheng10 closed pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 closed pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195


   


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

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



[GitHub] [pulsar] Anonymitaet commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   @lordcheng10 Thanks for your contribution. For this PR, do we need to update docs?
   
   (The [PR template contains info about doc](https://github.com/apache/pulsar/blob/master/.github/PULL_REQUEST_TEMPLATE.md#documentation), which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks) 


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

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



[GitHub] [pulsar] merlimat commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   I think it's correct to flush everything on a (graceful) close, though I would not characterize it as "data lost" since the send futures will not be successful. 
   
   We should add testing for the behavior. 


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

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



[GitHub] [pulsar] BewareMyPower commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   @lordcheng10 If you're going to submit a PIP, please follow the PIP template.


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

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



[GitHub] [pulsar] lordcheng10 edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
lordcheng10 edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-927248432


   > flushAsync
   
   Thank you for your attention! I understand that the original close cleanup logic needs to be completed after the flush is completed. If the flushAsync method is used instead of the flush method, the data cannot be flushed before the original close cleanup logic is completed;


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   > I mean that when the `flushAsync` future complete, then do the logic. So the `closeAsync` method will not block
   
   You are correct, I will try to modify 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.

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

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



[GitHub] [pulsar] Anonymitaet commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   @lordcheng10 feel free to ping me if you need a doc review.


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

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



[GitHub] [pulsar] github-actions[bot] commented on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-1054902644


   The pr had no activity for 30 days, mark with Stale label.


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   For similar producer close logic, we can refer to Kafka's processing: https://cwiki.apache.org/confluence/display/KAFKA/KIP-8+-+Add+a+flush+method+to+the+producer+API
   
   It may be more reasonable for the producer to flush during close. When closing, sending the unsent data to the server is a common demand of users. No user would be more willing to discard the data in the buffer.
   
   Of course, we can also provide a close method with a timeout to satisfy some users who do not want the close method to take too much time.


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

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



[GitHub] [pulsar] Shoothzj commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   I mean that when the `flushAsync` future complete, then do the logic. So the `closeAsync` method will not block


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   > PIP template.
   
   OK, according to the PIP template, I modified it again。


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


   > I mean that when the `flushAsync` future complete, then do the logic. So the `closeAsync` method will not block
   
   @Shoothzj As we just discussed, I replaced flush with flushAsync and resubmitted the code.


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

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



[GitHub] [pulsar] lordcheng10 commented on pull request #12195: When the producer or client is normally closed, data will be lost

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


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

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

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



[GitHub] [pulsar] liangyuanpeng edited a comment on pull request #12195: When the producer or client is normally closed, data will be lost

Posted by GitBox <gi...@apache.org>.
liangyuanpeng edited a comment on pull request #12195:
URL: https://github.com/apache/pulsar/pull/12195#issuecomment-928767615


   I believe that it will be resolve https://github.com/apache/pulsar/issues/11780 .


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

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